You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Matthias J. Sax" <ma...@confluent.io> on 2019/04/11 08:47:39 UTC

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Thanks for driving the discussion of this KIP. It seems that everybody
agrees that the current branch() method using arrays is not optimal.

I had a quick look into the PR and I like the overall proposal. There
are some minor things we need to consider. I would recommend the
following renaming:

KStream#branch() -> #split()
KBranchedStream#addBranch() -> BranchingKStream#branch()
KBranchedStream#defaultBranch() -> BranchingKStream#default()

It's just a suggestion to get slightly shorter method names.

In the current PR, defaultBranch() does take an `Predicate` as argument,
but I think that is not required?

Also, we should consider KIP-307, that was recently accepted and is
currently implemented:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL

Ie, we should add overloads that accepted a `Named` parameter.


For the issue that the created `KStream` object are in different scopes:
could we extend `KBranchedStream` with a `get(int index)` method that
returns the corresponding "branched" result `KStream` object? Maybe, the
second argument of `addBranch()` should not be a `Consumer<KStream>` but
a `Function<KStream,KStream>` and `get()` could return whatever the
`Function` returns?


Finally, I would also suggest to update the KIP with the current
proposal. That makes it easier to review.


-Matthias



On 3/31/19 12:22 PM, Paul Whalen wrote:
> Ivan,
> 
> I'm a bit of a novice here as well, but I think it makes sense for you to
> revise the KIP and continue the discussion.  Obviously we'll need some
> buy-in from committers that have actual binding votes on whether the KIP
> could be adopted.  It would be great to hear if they think this is a good
> idea overall.  I'm not sure if that happens just by starting a vote, or if
> there is generally some indication of interest beforehand.
> 
> That being said, I'll continue the discussion a bit: assuming we do move
> forward the solution of "stream.branch() returns KBranchedStream", do we
> deprecate "stream.branch(...) returns KStream[]"?  I would favor
> deprecating, since having two mutually exclusive APIs that accomplish the
> same thing is confusing, especially when they're fairly similar anyway.  We
> just need to be sure we're not making something impossible/difficult that
> is currently possible/easy.
> 
> Regarding my PR - I think the general structure would work, it's just a
> little sloppy overall in terms of naming and clarity. In particular,
> passing in the "predicates" and "children" lists which get modified in
> KBranchedStream but read from all the way KStreamLazyBranch is a bit
> complicated to follow.
> 
> Thanks,
> Paul
> 
> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <ip...@mail.ru> wrote:
> 
>> Hi Paul!
>>
>> I read your code carefully and now I am fully convinced: your proposal
>> looks better and should work. We just have to document the crucial fact
>> that KStream consumers are invoked as they're added. And then it's all
>> going to be very nice.
>>
>> What shall we do now? I should re-write the KIP and resume the
>> discussion here, right?
>>
>> Why are you telling that your PR 'should not be even a starting point if
>> we go in this direction'? To me it looks like a good starting point. But
>> as a novice in this project I might miss some important details.
>>
>> Regards,
>>
>> Ivan
>>
>>
>> 28.03.2019 17:38, Paul Whalen пишет:
>>> Ivan,
>>>
>>> Maybe I’m missing the point, but I believe the stream.branch() solution
>> supports this. The couponIssuer::set* consumers will be invoked as they’re
>> added, not during streamsBuilder.build(). So the user still ought to be
>> able to call couponIssuer.coupons() afterward and depend on the branched
>> streams having been set.
>>>
>>> The issue I mean to point out is that it is hard to access the branched
>> streams in the same scope as the original stream (that is, not inside the
>> couponIssuer), which is a problem with both proposed solutions. It can be
>> worked around though.
>>>
>>> [Also, great to hear additional interest in 401, I’m excited to hear
>> your thoughts!]
>>>
>>> Paul
>>>
>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <ip...@mail.ru> wrote:
>>>>
>>>> Hi Paul!
>>>>
>>>> The idea to postpone the wiring of branches to the
>> streamsBuilder.build() also looked great for me at first glance, but ---
>>>>
>>>>> the newly branched streams are not available in the same scope as each
>> other.  That is, if we wanted to merge them back together again I don't see
>> a way to do that.
>>>>
>>>> You just took the words right out of my mouth, I was just going to
>> write in details about this issue.
>>>>
>>>> Consider the example from Bill's book, p. 101: say we need to identify
>> customers who have bought coffee and made a purchase in the electronics
>> store to give them coupons.
>>>>
>>>> This is the code I usually write under these circumstances using my
>> 'brancher' class:
>>>>
>>>> @Setter
>>>> class CouponIssuer{
>>>>    private KStream<....> coffePurchases;
>>>>    private KStream<....> electronicsPurchases;
>>>>
>>>>    KStream<...> coupons(){
>>>>        return coffePurchases.join(electronicsPurchases...)...whatever
>>>>
>>>>        /*In the real world the code here can be complex, so creation of
>> a separate CouponIssuer class is fully justified, in order to separate
>> classes' responsibilities.*/
>>>>
>>>>   }
>>>> }
>>>>
>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>
>>>> new KafkaStreamsBrancher<....>()
>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
>>>>      .branch(predicate2, couponIssuer::setElectronicsPurchases)
>>>>      .onTopOf(transactionStream);
>>>>
>>>> /*Alas, this won't work if we're going to wire up everything later,
>> without the terminal operation!!!*/
>>>> couponIssuer.coupons()...
>>>>
>>>> Does this make sense?  In order to properly initialize the CouponIssuer
>> we need the terminal operation to be called before streamsBuilder.build()
>> is called.
>>>>
>>>>
>>>> [BTW Paul, I just found out that your KIP-401 is essentially the next
>> KIP I was going to write here. I have some thoughts based on my experience,
>> so I will join the discussion on KIP-401 soon.]
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>> Ivan,
>>>>> I tried to make a very rough proof of concept of a fluent API based
>> off of
>>>>> KStream here (https://github.com/apache/kafka/pull/6512), and I think
>> I
>>>>> succeeded at removing both cons.
>>>>>     - Compatibility: I was incorrect earlier about compatibility
>> issues,
>>>>>     there aren't any direct ones.  I was unaware that Java is smart
>> enough to
>>>>>     distinguish between a branch(varargs...) returning one thing and
>> branch()
>>>>>     with no arguments returning another thing.
>>>>>     - Requiring a terminal method: We don't actually need it.  We can
>> just
>>>>>     build up the branches in the KBranchedStream who shares its state
>> with the
>>>>>     ProcessorSupplier that will actually do the branching.  It's not
>> terribly
>>>>>     pretty in its current form, but I think it demonstrates its
>> feasibility.
>>>>> To be clear, I don't think that pull request should be final or even a
>>>>> starting point if we go in this direction, I just wanted to see how
>>>>> challenging it would be to get the API working.
>>>>> I will say though, that I'm not sure the existing solution could be
>>>>> deprecated in favor of this, which I had originally suggested was a
>>>>> possibility.  The reason is that the newly branched streams are not
>>>>> available in the same scope as each other.  That is, if we wanted to
>> merge
>>>>> them back together again I don't see a way to do that.  The KIP
>> proposal
>>>>> has the same issue, though - all this means is that for either
>> solution,
>>>>> deprecating the existing branch(...) is not on the table.
>>>>> Thanks,
>>>>> Paul
>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <ip...@mail.ru>
>> wrote:
>>>>>> OK, let me summarize what we have discussed up to this point.
>>>>>>
>>>>>> First, it seems that it's commonly agreed that branch API needs
>>>>>> improvement. Motivation is given in the KIP.
>>>>>>
>>>>>> There are two potential ways to do it:
>>>>>>
>>>>>> 1. (as origianlly proposed)
>>>>>>
>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>     .branch(predicate1, ks ->..)
>>>>>>     .branch(predicate2, ks->..)
>>>>>>     .defaultBranch(ks->..) //optional
>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns its argument
>>>>>>
>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't make sense
>> until
>>>>>> all the necessary ingredients are provided.
>>>>>>
>>>>>> CONS: The need to create a KafkaStreamsBrancher instance contrasts the
>>>>>> fluency of other KStream methods.
>>>>>>
>>>>>> 2. (as Paul proposes)
>>>>>>
>>>>>> stream
>>>>>>     .branch(predicate1, ks ->...)
>>>>>>     .branch(predicate2, ks->...)
>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both defaultBranch(..)
>> and
>>>>>> noDefault() return void
>>>>>>
>>>>>> PROS: Generally follows the way KStreams interface is defined.
>>>>>>
>>>>>> CONS: We need to define two terminal methods (defaultBranch(ks->) and
>>>>>> noDefault()). And for a user it is very easy to miss the fact that one
>>>>>> of the terminal methods should be called. If these methods are not
>>>>>> called, we can throw an exception in runtime.
>>>>>>
>>>>>> Colleagues, what are your thoughts? Can we do better?
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>
>>>>>>>
>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>> Paul,
>>>>>>>>
>>>>>>>> I see your point when you are talking about
>>>>>>>> stream..branch..branch...default..
>>>>>>>>
>>>>>>>> Still, I believe that this cannot not be implemented the easy way.
>>>>>>>> Maybe we all should think further.
>>>>>>>>
>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>
>>>>>>>>> user could specify a terminal method that assumes nothing will
>> reach
>>>>>>>>> the default branch,
>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>>
>>>>>>>> 1) OK, apparently this should not be the only option besides
>>>>>>>> `default`, because there are scenarios when we want to just silently
>>>>>>>> drop the messages that didn't match any predicate. 2) Throwing an
>>>>>>>> exception in the middle of data flow processing looks like a bad
>> idea.
>>>>>>>> In stream processing paradigm, I would prefer to emit a special
>>>>>>>> message to a dedicated stream. This is exactly where `default` can
>> be
>>>>>>>> used.
>>>>>>>>
>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to track
>>>>>>>>> dangling
>>>>>>>> branches that haven't been terminated and raise a clear error
>> before it
>>>>>>>> becomes an issue.
>>>>>>>>
>>>>>>>> You mean a runtime exception, when the program is compiled and run?
>>>>>>>> Well,  I'd prefer an API that simply won't compile if used
>>>>>>>> incorrectly. Can we build such an API as a method chain starting
>> from
>>>>>>>> KStream object? There is a huge cost difference between runtime and
>>>>>>>> compile-time errors. Even if a failure uncovers instantly on unit
>>>>>>>> tests, it costs more for the project than a compilation failure.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>>
>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>> Ivan,
>>>>>>>>>
>>>>>>>>> Good point about the terminal operation being required.  But is
>> that
>>>>>>>>> really
>>>>>>>>> such a bad thing?  If the user doesn't want a defaultBranch they
>> can
>>>>>>>>> call
>>>>>>>>> some other terminal method (noDefaultBranch()?) just as easily.  In
>>>>>>>>> fact I
>>>>>>>>> think it creates an opportunity for a nicer API - a user could
>> specify
>>>>>> a
>>>>>>>>> terminal method that assumes nothing will reach the default branch,
>>>>>>>>> throwing an exception if such a case occurs.  That seems like an
>>>>>>>>> improvement over the current branch() API, which allows for the
>> more
>>>>>>>>> subtle
>>>>>>>>> behavior of records unexpectedly getting dropped.
>>>>>>>>>
>>>>>>>>> The need for a terminal operation certainly has to be well
>>>>>>>>> documented, but
>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to track
>>>>>>>>> dangling
>>>>>>>>> branches that haven't been terminated and raise a clear error
>> before it
>>>>>>>>> becomes an issue.  Especially now that there is a "build step"
>> where
>>>>>> the
>>>>>>>>> topology is actually wired up, when StreamsBuilder.build() is
>> called.
>>>>>>>>>
>>>>>>>>> Regarding onTopOf() returning its argument, I agree that it's
>>>>>>>>> critical to
>>>>>>>>> allow users to do other operations on the input stream.  With the
>>>>>> fluent
>>>>>>>>> solution, it ought to work the same way all other operations do -
>> if
>>>>>> you
>>>>>>>>> want to process off the original KStream multiple times, you just
>>>>>>>>> need the
>>>>>>>>> stream as a variable so you can call as many operations on it as
>> you
>>>>>>>>> desire.
>>>>>>>>>
>>>>>>>>> Thoughts?
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Paul
>>>>>>>>>
>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <iponomarev@mail.ru
>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hello Paul,
>>>>>>>>>>
>>>>>>>>>> I afraid this won't work because we do not always need the
>>>>>>>>>> defaultBranch. And without a terminal operation we don't know
>> when to
>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>>
>>>>>>>>>> In my proposal, onTopOf returns its argument, so we can do
>> something
>>>>>>>>>> more with the original branch after branching.
>>>>>>>>>>
>>>>>>>>>> I understand your point that the need of special object
>> construction
>>>>>>>>>> contrasts the fluency of most KStream methods. But here we have a
>>>>>>>>>> special case: we build the switch to split the flow, so I think
>> this
>>>>>> is
>>>>>>>>>> still idiomatic.
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>>
>>>>>>>>>> Ivan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>> Ivan,
>>>>>>>>>>>
>>>>>>>>>>> I think it's a great idea to improve this API, but I find the
>>>>>>>>>>> onTopOff()
>>>>>>>>>>> mechanism a little confusing since it contrasts the fluency of
>> other
>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call a method on
>> the
>>>>>>>>>> stream
>>>>>>>>>>> so it still reads top to bottom if the branch cases are defined
>>>>>>>>>>> fluently.
>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very nice and the
>>>>>>>>>>> right
>>>>>>>>>> way
>>>>>>>>>>> to do things, but what if we flipped around how we specify the
>> source
>>>>>>>>>>> stream.
>>>>>>>>>>>
>>>>>>>>>>> Like:
>>>>>>>>>>>
>>>>>>>>>>> stream.branch()
>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
>>>>>>>>>>>            .defaultBranch(this::handleDefault);
>>>>>>>>>>>
>>>>>>>>>>> Where branch() returns a KBranchedStreams or KStreamBrancher or
>>>>>>>>>> something,
>>>>>>>>>>> which is added to by addBranch() and terminated by
>> defaultBranch()
>>>>>>>>>>> (which
>>>>>>>>>>> returns void).  This is obviously incompatible with the current
>>>>>>>>>>> API, so
>>>>>>>>>> the
>>>>>>>>>>> new stream.branch() would have to have a different name, but that
>>>>>>>>>>> seems
>>>>>>>>>>> like a fairly small problem - we could call it something like
>>>>>>>>>>> branched()
>>>>>>>>>> or
>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>>>>>
>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It seems like it
>>>>>>>>>>> does to
>>>>>>>>>>> me, allowing for clear in-line branching while also allowing you
>> to
>>>>>>>>>>> dynamically build of branches off of KBranchedStreams if desired.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Paul
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>
>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>
>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>
>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
>>>>>>>>>>>>            ks.filter(....).mapValues(...)
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> ......
>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
>>>>>>>>>>>>       .onTopOf(....)
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Ivan
>>>>>>>>>>>>
>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher takes a Consumer
>> as a
>>>>>>>>>>>> second
>>>>>>>>>>>>> argument which returns nothing, and the example in the KIP
>> shows
>>>>>>>>>>>>> each
>>>>>>>>>>>>> stream from the branch using a terminal node (KafkaStreams#to()
>>>>>>>>>>>>> in this
>>>>>>>>>>>>> case).
>>>>>>>>>>>>>
>>>>>>>>>>>>> Maybe I've missed something, but how would we handle the case
>>>>>>>>>>>>> where the
>>>>>>>>>>>>> user has created a branch but wants to continue processing and
>> not
>>>>>>>>>>>>> necessarily use a terminal node on the branched stream
>> immediately?
>>>>>>>>>>>>>
>>>>>>>>>>>>> For example, using today's logic as is if we had something like
>>>>>>>>>>>>> this:
>>>>>>>>>>>>>
>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <bbejeck@gmail.com
>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP- 418.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418. Please take a
>> look
>>>>>> at
>>>>>>>>>> the
>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>
>>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>
>>>>>>>>>>>>>> JIRA KAFKA-5488:
>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> PR#6164: https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Hi Ivan,

Welcome back! Thanks for the update. This KIP looks really nice. 

Thanks also for taking into account the grammar. I think you’ve done a good job of balancing the proposed grammar’s objectives with the current API’s idioms. 

I have just a few comments remaining:

1. Can you propose to deprecate (but not remove) the existing ‘branch’ method? Since the proposed API is presented as a superior alternative, we may as well prune so that the total surface area remains constant. 

2. There’s one example (the enum one) that says “this is why this API is superior to branching directly off of the parent KStream”, but it doesn’t show what it would look like to branch off the parent KStream. Can you add that counterexample? My one reservation to the idea of a branch operator at all is that you can implicitly branch off of any KTable or KStream. I think it would be nice to devote some text to explaining when either one or the other is preferable. 

3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you were talking about the kafka Consumer interface (which doesn’t make sense, of course). I get that you were referring to the java Consumer interface, but we should still probably to to avoid the ambiguity. Just throwing out a suggestion, how about ‘withSink’?

4. More of an aside, it seems slightly convoluted to have withChain and withConsumer, since the sink operation could theoretically just go on the end of the chain. But I can see that you wanted to capture the chain and return it in the KStream-valued Map, which is incompatible with a terminal chain. It seems like there are really two disjoint use cases: EITHER using chain and the result map OR using just the sink. The API doesn’t codify this, though , which might be confusing. I really have no idea of what to do differently, though. Unless you’re feeling more creative than I am, maybe we’ll just plan to clarify this in the JavaDoc. 

Thanks again!
-John

On Wed, Apr 22, 2020, at 18:45, Ivan Ponomarev wrote:
> Hi,
> 
> I have read the John's "DSL design principles" and have completely 
> rewritten the KIP, see 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> 
> 
> This version includes all the previous discussion results and follows 
> the design principles, with one exception.
> 
> The exception is
> 
> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> 
> which formally violates 'no more than one parameter' rule, but I think 
> here it is justified.
> 
> We must provide a predicate for a branch and don't need to provide one 
> for the default branch. Thus for both operations we may use a single 
> Branched parameter class, with an extra method parameter for `branch`.
> 
> Since predicate is a natural, necessary part of a branch, no 
> 'proliferation of overloads, deprecations, etc.' is expected here as it 
> is said in the rationale for the 'single parameter rule'.
> 
> WDYT, is this KIP mature enough to begin voting?
> 
> Regards,
> 
> Ivan
> 
> 21.04.2020 2:09, Matthias J. Sax пишет:
> > Ivan,
> > 
> > no worries about getting side tracked. Glad to have you back!
> > 
> > The DSL improved further in the meantime and we already have a `Named`
> > config object to name operators. It seems reasonable to me to build on this.
> > 
> > Furthermore, John did a writeup about "DSL design principles" that we
> > want to follow:
> > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> > -- might be worth to checkout.
> > 
> > 
> > -Matthias
> > 
> > 
> > On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >> Hi everyone!
> >>
> >> Let me revive the discussion of this KIP.
> >>
> >> I'm very sorry for stopping my participation in the discussion in June
> >> 2019. My project work was very intensive then and it didn't leave me
> >> spare time. But I think I must finish this, because we invested
> >> substantial effort into this discussion and I'm not feel entitled to
> >> propose other things before this one is finalized.
> >>
> >> During these months I proceeded with writing and reviewing Kafka
> >> Streams-related code. Every time I needed branching, Spring-Kafka's
> >> KafkaStreamBrancher class of my invention (the original idea for this
> >> KIP) worked for me -- that's another reason why I gave up pushing the
> >> KIP forward. When I was coming across the problem with the scope of
> >> branches, I worked around it this way:
> >>
> >> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >> new KafkaStreamBrancher<....>()
> >>      .branch(....)
> >>      .defaultBranch(result::set)
> >>      .onTopOf(someStream);
> >> result.get()...
> >>
> >>
> >> And yes, of course I don't feel very happy with this approach.
> >>
> >> I think that Matthias came up with a bright solution in his post from
> >> May, 24th 2019. Let me quote it:
> >>
> >> KStream#split() -> KBranchedStream
> >> // branch is not easily accessible in current scope
> >> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>    -> KBranchedStream
> >> // assign a name to the branch and
> >> // return the sub-stream to the current scope later
> >> //
> >> // can be simple as `#branch(p, s->s, "name")`
> >> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>    -> KBranchedStream
> >> // default branch is not easily accessible
> >> // return map of all named sub-stream into current scope
> >> KBranchedStream#default(Cosumer<KStream>)
> >>    -> Map<String,KStream>
> >> // assign custom name to default-branch
> >> // return map of all named sub-stream into current scope
> >> KBranchedStream#default(Function<KStream,KStream>, String)
> >>    -> Map<String,KStream>
> >> // assign a default name for default
> >> // return map of all named sub-stream into current scope
> >> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>    -> Map<String,KStream>
> >> // return map of all names sub-stream into current scope
> >> KBranchedStream#noDefaultBranch()
> >>    -> Map<String,KStream>
> >>
> >> I believe this would satisfy everyone. Optional names seems to be a good
> >> idea: when you don't need to have the branches in the same scope, you
> >> just don't use names and you don't risk making your code brittle. Or,
> >> you might want to add names just for debugging purposes. Or, finally,
> >> you might use the returned Map to have the named branches in the
> >> original scope.
> >>
> >> There also was an input from John Roesler on June 4th, 2019, who
> >> suggested using Named class. I can't comment on this. The idea seems
> >> reasonable, but in this matter I'd rather trust people who are more
> >> familiar with Streams API design principles than me.
> >>
> >> Regards,
> >>
> >> Ivan
> >>
> >>
> >>
> >> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>> I am moving this KIP into "inactive status". Feel free to resume the KIP
> >>> at any point.
> >>>
> >>> If anybody else is interested in picking up this KIP, feel free to do so.
> >>>
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>> Ivan,
> >>>>
> >>>> did you see my last reply? What do you think about my proposal to mix
> >>>> both approaches and try to get best-of-both worlds?
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>> Thanks for the input John!
> >>>>>
> >>>>>> under your suggestion, it seems that the name is required
> >>>>>
> >>>>> If you want to get the `KStream` as part of the `Map` back using a
> >>>>> `Function`, yes. If you follow the "embedded chaining" pattern using a
> >>>>> `Consumer`, no.
> >>>>>
> >>>>> Allowing for a default name via `split()` can of course be done.
> >>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>
> >>>>> I wanted to sketch out a high level proposal to merge both patterns
> >>>>> only. Your suggestions to align the new API with the existing API make
> >>>>> totally sense.
> >>>>>
> >>>>>
> >>>>>
> >>>>> One follow up question: Would `Named` be optional or required in
> >>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>
> >>>>> If both are mandatory, what do we gain by it? The returned `Map` only
> >>>>> contains the corresponding branches, so why should we prefix all of
> >>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>> `split()`, the same question raises?
> >>>>>
> >>>>> Requiring `Named` in `split()` seems only to make sense, if `Named` is
> >>>>> optional in `branch()` and we generate `-X` suffix using a counter for
> >>>>> different branch name. However, this might lead to the problem of
> >>>>> changing names if branches are added/removed. Also, how would the names
> >>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>> returned in the `Map`).
> >>>>>
> >>>>> If `Named` is optional for both, it could happen that a user misses to
> >>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>
> >>>>>
> >>>>> Hence, I am actually in favor to not allow a default name but keep
> >>>>> `split()` without parameter and make `Named` in `branch()` required
> >>>>> if a
> >>>>> `Function` is used. This makes it explicit to the user that
> >>>>> specifying a
> >>>>> name is required if a `Function` is used.
> >>>>>
> >>>>>
> >>>>>
> >>>>> About
> >>>>>
> >>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>
> >>>>> I don't think that the branching predicate is a configuration and hence
> >>>>> would not include it in a configuration object.
> >>>>>
> >>>>>>       withChain(...);
> >>>>>
> >>>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
> >>>>> seem to be a configuration. We can also not prevent a user to call
> >>>>> `withName()` in combination of `withChain()` what does not make sense
> >>>>> IMHO. We could of course throw an RTE but not have a compile time check
> >>>>> seems less appealing. Also, it could happen that neither `withChain()`
> >>>>> not `withName()` is called and the branch is missing in the returned
> >>>>> `Map` what lead to runtime issues, too.
> >>>>>
> >>>>> Hence, I don't think that we should add `BranchConfig`. A config object
> >>>>> is helpful if each configuration can be set independently of all
> >>>>> others,
> >>>>> but this seems not to be the case here. If we add new configuration
> >>>>> later, we can also just move forward by deprecating the methods that
> >>>>> accept `Named` and add new methods that accepted `BranchConfig` (that
> >>>>> would of course implement `Named`).
> >>>>>
> >>>>>
> >>>>> Thoughts?
> >>>>>
> >>>>>
> >>>>> @Ivan, what do you think about the general idea to blend the two main
> >>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>>
> >>>>>
> >>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
> >>>>>> everyone. Returning the map from the terminal operations also solves
> >>>>>> the problem of merging/joining the branched streams, if we want to add
> >>>>>> support for the compliment later on.
> >>>>>>
> >>>>>> Under your suggestion, it seems that the name is required. Otherwise,
> >>>>>> we wouldn't have keys for the map to return. I this this is actually
> >>>>>> not too bad, since experience has taught us that, although names for
> >>>>>> operations are not required to define stream processing logic, it does
> >>>>>> significantly improve the operational experience when you can map the
> >>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>> wouldn't (have to) reference the name to chain extra processing onto
> >>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>
> >>>>>> In the current implementation of Branch, you can name the branch
> >>>>>> operator itself, and then all the branches get index-suffixed names
> >>>>>> built from the branch operator name. I guess under this proposal, we
> >>>>>> could naturally append the branch name to the branching operator name,
> >>>>>> like this:
> >>>>>>
> >>>>>>      stream.split(Named.withName("mysplit")) //creates node "mysplit"
> >>>>>>                 .branch(..., ..., "abranch") // creates node
> >>>>>> "mysplit-abranch"
> >>>>>>                 .defaultBranch(...) // creates node "mysplit-default"
> >>>>>>
> >>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>
> >>>>>> We don't have a defined grammar, so there's plenty of room to debate
> >>>>>> the "best" syntax in the context of each operation, but in general,
> >>>>>> the KStream DSL operators follow this pattern:
> >>>>>>
> >>>>>>       operator(function, config_object?) OR operator(config_object)
> >>>>>>
> >>>>>> where config_object is often just Named in the "function" variant.
> >>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>
> >>>>>> Here, we're introducing a totally different pattern:
> >>>>>>
> >>>>>>     operator(function, function, string)
> >>>>>>
> >>>>>> where the string is the name.
> >>>>>> My first question is whether the name should instead be specified with
> >>>>>> the NamedOperation interface.
> >>>>>>
> >>>>>> My second question is whether we should just roll all these arguments
> >>>>>> up into a config object like:
> >>>>>>
> >>>>>>      KBranchedStream#branch(BranchConfig)
> >>>>>>
> >>>>>>      interface BranchConfig extends NamedOperation {
> >>>>>>       withPredicate(...);
> >>>>>>       withChain(...);
> >>>>>>       withName(...);
> >>>>>>     }
> >>>>>>
> >>>>>> Although I guess we'd like to call BranchConfig something more like
> >>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>
> >>>>>> This makes the source code a little noisier, but it also makes us more
> >>>>>> future-proof, as we can deal with a wide range of alternatives purely
> >>>>>> in the config interface, and never have to deal with adding overloads
> >>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>
> >>>>>> WDYT?
> >>>>>>
> >>>>>> Thanks,
> >>>>>> -John
> >>>>>>
> >>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>> <mi...@confluent.io> wrote:
> >>>>>>>
> >>>>>>> Matthias: I think that's pretty reasonable from my point of view.
> >>>>>>> Good
> >>>>>>> suggestion.
> >>>>>>>
> >>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>> <ma...@confluent.io>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Interesting discussion.
> >>>>>>>>
> >>>>>>>> I am wondering, if we cannot unify the advantage of both approaches:
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>
> >>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>     -> KBranchedStream
> >>>>>>>>
> >>>>>>>> // assign a name to the branch and
> >>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>> //
> >>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>>     -> KBranchedStream
> >>>>>>>>
> >>>>>>>> // default branch is not easily accessible
> >>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>     -> Map<String,KStream>
> >>>>>>>>
> >>>>>>>> // assign custom name to default-branch
> >>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>     -> Map<String,KStream>
> >>>>>>>>
> >>>>>>>> // assign a default name for default
> >>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>     -> Map<String,KStream>
> >>>>>>>>
> >>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>     -> Map<String,KStream>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Hence, for each sub-stream, the user can pick to add a name and
> >>>>>>>> return
> >>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>> implementation can
> >>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>> returned
> >>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>
> >>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>
> >>>>>>>> Thoughts?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>> Ivan,
> >>>>>>>>>
> >>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>> dynamic case.
> >>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>> necessity.
> >>>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
> >>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>
> >>>>>>>>> Separately, I'm interested to see where the present discussion
> >>>>>>>>> leads.
> >>>>>>>>> I've written enough Javascript code in my life to be suspicious of
> >>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>> references (or
> >>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>> that this
> >>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>> logic when
> >>>>>>>>> their nested closure situation got out of hand. Unfortunately, it's
> >>>>>>>>> replacing nesting with redirection, both of which disrupt code
> >>>>>>>>> readability (but in different ways for different reasons). In other
> >>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>> solution if
> >>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>
> >>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>> references aren't
> >>>>>>>>> the end of the story either, and you can see that by observing that
> >>>>>>>>> there have been two follow-on eras, as they continue trying to
> >>>>>>>>> cope with
> >>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>> First, you
> >>>>>>>>> have Futures/Promises, which essentially let you convert nested
> >>>>>>>>> code to
> >>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>> this).
> >>>>>>>>> Most lately, you have async/await, which is an effort to apply
> >>>>>>>>> language
> >>>>>>>>> (not just API) syntax to the problem, and offer the "flattest"
> >>>>>>>>> possible
> >>>>>>>>> programming style to solve the problem (because you get back to
> >>>>>>>>> just one
> >>>>>>>>> code block per functional unit).
> >>>>>>>>>
> >>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>> nowhere
> >>>>>>>>> near as callback heavy as JS, so I don't think we have to take
> >>>>>>>>> the JS
> >>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>> valuable
> >>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>> bringing this
> >>>>>>>>> up to inspire further/deeper discussion. At the same time, just
> >>>>>>>>> like JS,
> >>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>
> >>>>>>>>> Separately again, I'm interested in the post-branch merge (and
> >>>>>>>>> I'd also
> >>>>>>>>> add join) problem that Paul brought up. We can clearly punt on
> >>>>>>>>> it, by
> >>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>> there a DSL
> >>>>>>>>> way to do it?
> >>>>>>>>>
> >>>>>>>>> Thanks again for your driving this,
> >>>>>>>>> -John
> >>>>>>>>>
> >>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
> >>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>
> >>>>>>>>>       Ivan, I’ll definitely forfeit my point on the clumsiness of
> >>>>>>>>> the
> >>>>>>>>>       branch(predicate, consumer) solution, I don’t see any real
> >>>>>>>>> drawbacks
> >>>>>>>>>       for the dynamic case.
> >>>>>>>>>
> >>>>>>>>>       IMO the one trade off to consider at this point is the scope
> >>>>>>>>>       question. I don’t know if I totally agree that “we rarely
> >>>>>>>>> need them
> >>>>>>>>>       in the same scope” since merging the branches back together
> >>>>>>>>> later
> >>>>>>>>>       seems like a perfectly plausible use case that can be a lot
> >>>>>>>>> nicer
> >>>>>>>>>       when the branched streams are in the same scope. That being
> >>>>>>>>> said,
> >>>>>>>>>       for the reasons Ivan listed, I think it is overall the better
> >>>>>>>>>       solution - working around the scope thing is easy enough if
> >>>>>>>>> you need
> >>>>>>>>>       to.
> >>>>>>>>>
> >>>>>>>>>       > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>       <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>       >
> >>>>>>>>>       > Hello everyone, thank you all for joining the discussion!
> >>>>>>>>>       >
> >>>>>>>>>       > Well, I don't think the idea of named branches, be it a
> >>>>>>>>>       LinkedHashMap (no other Map will do, because order of
> >>>>>>>>> definition
> >>>>>>>>>       matters) or `branch` method  taking name and Consumer has more
> >>>>>>>>>       advantages than drawbacks.
> >>>>>>>>>       >
> >>>>>>>>>       > In my opinion, the only real positive outcome from Michael's
> >>>>>>>>>       proposal is that all the returned branches are in the same
> >>>>>>>>> scope.
> >>>>>>>>>       But 1) we rarely need them in the same scope 2) there is a
> >>>>>>>>>       workaround for the scope problem, described in the KIP.
> >>>>>>>>>       >
> >>>>>>>>>       > 'Inlining the complex logic' is not a problem, because we
> >>>>>>>>> can use
> >>>>>>>>>       method references instead of lambdas. In real world
> >>>>>>>>> scenarios you
> >>>>>>>>>       tend to split the complex logic to methods anyway, so the
> >>>>>>>>> code is
> >>>>>>>>>       going to be clean.
> >>>>>>>>>       >
> >>>>>>>>>       > The drawbacks are strong. The cohesion between predicates
> >>>>>>>>> and
> >>>>>>>>>       handlers is lost. We have to define predicates in one
> >>>>>>>>> place, and
> >>>>>>>>>       handlers in another. This opens the door for bugs:
> >>>>>>>>>       >
> >>>>>>>>>       > - what if we forget to define a handler for a name? or a
> >>>>>>>>> name for
> >>>>>>>>>       a handler?
> >>>>>>>>>       > - what if we misspell a name?
> >>>>>>>>>       > - what if we copy-paste and duplicate a name?
> >>>>>>>>>       >
> >>>>>>>>>       > What Michael propose would have been totally OK if we had
> >>>>>>>>> been
> >>>>>>>>>       writing the API in Lua, Ruby or Python. In those languages the
> >>>>>>>>>       "dynamic naming" approach would have looked most concise and
> >>>>>>>>>       beautiful. But in Java we expect all the problems related to
> >>>>>>>>>       identifiers to be eliminated in compile time.
> >>>>>>>>>       >
> >>>>>>>>>       > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>       >
> >>>>>>>>>       > And if we do, what advantage are we supposed to get
> >>>>>>>>> besides having
> >>>>>>>>>       all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>> missing your
> >>>>>>>>>       point?
> >>>>>>>>>       >
> >>>>>>>>>       > ---
> >>>>>>>>>       >
> >>>>>>>>>       > Earlier in this discussion John Roesler also proposed to do
> >>>>>>>>>       without "start branching" operator, and later Paul
> >>>>>>>>> mentioned that in
> >>>>>>>>>       the case when we have to add a dynamic number of branches, the
> >>>>>>>>>       current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>> solution. Let
> >>>>>>>>>       me address both comments here.
> >>>>>>>>>       >
> >>>>>>>>>       > 1) "Start branching" operator (I think that *split* is a
> >>>>>>>>> good name
> >>>>>>>>>       for it indeed) is critical when we need to do a dynamic
> >>>>>>>>> branching,
> >>>>>>>>>       see example below.
> >>>>>>>>>       >
> >>>>>>>>>       > 2) No, dynamic branching in current KIP is not clumsy at
> >>>>>>>>> all.
> >>>>>>>>>       Imagine a real-world scenario when you need one branch per
> >>>>>>>>> enum
> >>>>>>>>>       value (say, RecordType). You can have something like this:
> >>>>>>>>>       >
> >>>>>>>>>       > /*John:if we had to start with stream.branch(...) here,
> >>>>>>>>> it would
> >>>>>>>>>       have been much messier.*/
> >>>>>>>>>       > KBranchedStream branched = stream.split();
> >>>>>>>>>       >
> >>>>>>>>>       > /*Not clumsy at all :-)*/
> >>>>>>>>>       > for (RecordType recordType : RecordType.values())
> >>>>>>>>>       >             branched = branched.branch((k, v) ->
> >>>>>>>>> v.getRecType() ==
> >>>>>>>>>       recordType,
> >>>>>>>>>       >                     recordType::processRecords);
> >>>>>>>>>       >
> >>>>>>>>>       > Regards,
> >>>>>>>>>       >
> >>>>>>>>>       > Ivan
> >>>>>>>>>       >
> >>>>>>>>>       >
> >>>>>>>>>       > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>       >> I also agree with Michael's observation about the core
> >>>>>>>>> problem of
> >>>>>>>>>       >> current `branch()` implementation.
> >>>>>>>>>       >>
> >>>>>>>>>       >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>> object. My
> >>>>>>>>>       thinking
> >>>>>>>>>       >> was more aligned with Paul's proposal to just add a name
> >>>>>>>>> to each
> >>>>>>>>>       >> `branch()` statement and return a `Map<String,KStream>`.
> >>>>>>>>>       >>
> >>>>>>>>>       >> It makes the code easier to read, and also make the
> >>>>>>>>> order of
> >>>>>>>>>       >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>       >>
> >>>>>>>>>       >>>>>> Map<String, KStream<K, V>> branches = stream.split()
> >>>>>>>>>       >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>       >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>       >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>       >> An open question is the case for which no
> >>>>>>>>> defaultBranch() should
> >>>>>>>> be
> >>>>>>>>>       >> specified. Atm, `split()` and `branch()` would return
> >>>>>>>>>       `BranchedKStream`
> >>>>>>>>>       >> and the call to `defaultBranch()` that returns the `Map` is
> >>>>>>>> mandatory
> >>>>>>>>>       >> (what is not the case atm). Or is this actually not a real
> >>>>>>>> problem,
> >>>>>>>>>       >> because users can just ignore the branch returned by
> >>>>>>>>>       `defaultBranch()`
> >>>>>>>>>       >> in the result `Map` ?
> >>>>>>>>>       >>
> >>>>>>>>>       >>
> >>>>>>>>>       >> About "inlining": So far, it seems to be a matter of
> >>>>>>>>> personal
> >>>>>>>>>       >> preference. I can see arguments for both, but no "killer
> >>>>>>>>>       argument" yet
> >>>>>>>>>       >> that clearly make the case for one or the other.
> >>>>>>>>>       >>
> >>>>>>>>>       >>
> >>>>>>>>>       >> -Matthias
> >>>>>>>>>       >>
> >>>>>>>>>       >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>       >>> Perhaps inlining is the wrong terminology. It doesn’t
> >>>>>>>>> require
> >>>>>>>>>       that a lambda with the full downstream topology be defined
> >>>>>>>>> inline -
> >>>>>>>>>       it can be a method reference as with Ivan’s original
> >>>>>>>>> suggestion.
> >>>>>>>>>       The advantage of putting the predicate and its downstream
> >>>>>>>>> logic
> >>>>>>>>>       (Consumer) together in branch() is that they are required
> >>>>>>>>> to be near
> >>>>>>>>>       to each other.
> >>>>>>>>>       >>>
> >>>>>>>>>       >>> Ultimately the downstream code has to live somewhere,
> >>>>>>>>> and deep
> >>>>>>>>>       branch trees will be hard to read regardless.
> >>>>>>>>>       >>>
> >>>>>>>>>       >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>       <michael.drogalis@confluent.io
> >>>>>>>>>       <ma...@confluent.io>> wrote:
> >>>>>>>>>       >>>>
> >>>>>>>>>       >>>> I'm less enthusiastic about inlining the branch logic
> >>>>>>>>> with its
> >>>>>>>>>       downstream
> >>>>>>>>>       >>>> functionality. Programs that have deep branch trees will
> >>>>>>>>>       quickly become
> >>>>>>>>>       >>>> harder to read as a single unit.
> >>>>>>>>>       >>>>
> >>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>       <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Also +1 on the issues/goals as Michael outlined them,
> >>>>>>>>> I think
> >>>>>>>>>       that sets a
> >>>>>>>>>       >>>>> great framework for the discussion.
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Regarding the SortedMap solution, my understanding is
> >>>>>>>>> that the
> >>>>>>>>>       current
> >>>>>>>>>       >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>> (pending naming
> >>>>>>>>>       decisions) is
> >>>>>>>>>       >>>>> roughly this:
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> stream.split()
> >>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >>>>>>>>>       >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Obviously some ordering is necessary, since branching
> >>>>>>>>> as a
> >>>>>>>>>       construct
> >>>>>>>>>       >>>>> doesn't work without it, but this solution seems like it
> >>>>>>>>>       provides as much
> >>>>>>>>>       >>>>> associativity as the SortedMap solution, because each
> >>>>>>>>> branch()
> >>>>>>>>>       call
> >>>>>>>>>       >>>>> directly associates the "conditional" with the "code
> >>>>>>>>> block."
> >>>>>>>>>       The value it
> >>>>>>>>>       >>>>> provides over the KIP solution is the accessing of
> >>>>>>>>> streams in
> >>>>>>>>>       the same
> >>>>>>>>>       >>>>> scope.
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> The KIP solution is less "dynamic" than the SortedMap
> >>>>>>>>> solution
> >>>>>>>>>       in the sense
> >>>>>>>>>       >>>>> that it is slightly clumsier to add a dynamic number of
> >>>>>>>>>       branches, but it is
> >>>>>>>>>       >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>> should favor
> >>>>>>>>>       the "static"
> >>>>>>>>>       >>>>> case anyway, and should make it simple and readable to
> >>>>>>>>>       fluently declare and
> >>>>>>>>>       >>>>> access your branches in-line.  It also makes it
> >>>>>>>>> impossible to
> >>>>>>>>>       ignore a
> >>>>>>>>>       >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>> identical
> >>>>>>>>>       SortedMap
> >>>>>>>>>       >>>>> solution on top of it.
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> I could also see a middle ground where instead of a raw
> >>>>>>>>>       SortedMap being
> >>>>>>>>>       >>>>> taken in, branch() takes a name and not a Consumer.
> >>>>>>>>> Something
> >>>>>>>>>       like this:
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Map<String, KStream<K, V>> branches = stream.split()
> >>>>>>>>>       >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>       >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>       >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Pros for that solution:
> >>>>>>>>>       >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>       >>>>> - no double brace initialization, hopefully slightly
> >>>>>>>>> more
> >>>>>>>>>       readable than
> >>>>>>>>>       >>>>> SortedMap
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Cons
> >>>>>>>>>       >>>>> - downstream branch logic cannot be specified inline
> >>>>>>>>> which
> >>>>>>>>>       makes it harder
> >>>>>>>>>       >>>>> to read top to bottom (like existing API and
> >>>>>>>>> SortedMap, but
> >>>>>>>>>       unlike the KIP)
> >>>>>>>>>       >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>> streams (like
> >>>>>>>>>       existing
> >>>>>>>>>       >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> (KBranchedStreams could even work *both* ways but
> >>>>>>>>> perhaps
> >>>>>>>>>       that's overdoing
> >>>>>>>>>       >>>>> it).
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Overall I'm curious how important it is to be able to
> >>>>>>>>> easily
> >>>>>>>>>       access the
> >>>>>>>>>       >>>>> branched KStream in the same scope as the original.
> >>>>>>>>> It's
> >>>>>>>>>       possible that it
> >>>>>>>>>       >>>>> doesn't need to be handled directly by the API, but
> >>>>>>>>> instead
> >>>>>>>>>       left up to the
> >>>>>>>>>       >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> Paul
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
> >>>>>>>>>       <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>       >>>>> wrote:
> >>>>>>>>>       >>>>>
> >>>>>>>>>       >>>>>> I'd like to +1 what Michael said about the issues
> >>>>>>>>> with the
> >>>>>>>>>       existing
> >>>>>>>>>       >>>>> branch
> >>>>>>>>>       >>>>>> method, I agree with what he's outlined and I think
> >>>>>>>>> we should
> >>>>>>>>>       proceed by
> >>>>>>>>>       >>>>>> trying to alleviate these problems. Specifically it
> >>>>>>>>> seems
> >>>>>>>>>       important to be
> >>>>>>>>>       >>>>>> able to cleanly access the individual branches (eg
> >>>>>>>>> by mapping
> >>>>>>>>>       >>>>>> name->stream), which I thought was the original
> >>>>>>>>> intention of
> >>>>>>>>>       this KIP.
> >>>>>>>>>       >>>>>>
> >>>>>>>>>       >>>>>> That said, I don't think we should so easily give in
> >>>>>>>>> to the
> >>>>>>>>>       double brace
> >>>>>>>>>       >>>>>> anti-pattern or force ours users into it if at all
> >>>>>>>>> possible to
> >>>>>>>>>       >>>>> avoid...just
> >>>>>>>>>       >>>>>> my two cents.
> >>>>>>>>>       >>>>>>
> >>>>>>>>>       >>>>>> Cheers,
> >>>>>>>>>       >>>>>> Sophie
> >>>>>>>>>       >>>>>>
> >>>>>>>>>       >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
> >>>>>>>>>       >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>       <ma...@confluent.io>> wrote:
> >>>>>>>>>       >>>>>>
> >>>>>>>>>       >>>>>>> I’d like to propose a different way of thinking
> >>>>>>>>> about this.
> >>>>>>>>>       To me,
> >>>>>>>>>       >>>>> there
> >>>>>>>>>       >>>>>>> are three problems with the existing branch signature:
> >>>>>>>>>       >>>>>>>
> >>>>>>>>>       >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>> raises unsafe
> >>>>>>>> type
> >>>>>>>>>       >>>>>> warnings.
> >>>>>>>>>       >>>>>>> 2. The way in which you use the stream branches is
> >>>>>>>>>       positionally coupled
> >>>>>>>>>       >>>>>> to
> >>>>>>>>>       >>>>>>> the ordering of the conditionals.
> >>>>>>>>>       >>>>>>> 3. It is brittle to extend existing branch calls with
> >>>>>>>>>       additional code
> >>>>>>>>>       >>>>>>> paths.
> >>>>>>>>>       >>>>>>>
> >>>>>>>>>       >>>>>>> Using associative constructs instead of relying on
> >>>>>>>>> ordered
> >>>>>>>>>       constructs
> >>>>>>>>>       >>>>>> would
> >>>>>>>>>       >>>>>>> be a stronger approach. Consider a signature that
> >>>>>>>>> instead
> >>>>>>>>>       looks like
> >>>>>>>>>       >>>>>> this:
> >>>>>>>>>       >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>       Predicate<?
> >>>>>>>>>       >>>>>>> super K,? super V>>);
> >>>>>>>>>       >>>>>>>
> >>>>>>>>>       >>>>>>> Branches are given names in a map, and as a result,
> >>>>>>>>> the API
> >>>>>>>>>       returns a
> >>>>>>>>>       >>>>>>> mapping of names to streams. The ordering of the
> >>>>>>>> conditionals is
> >>>>>>>>>       >>>>>> maintained
> >>>>>>>>>       >>>>>>> because it’s a sorted map. Insert order determines
> >>>>>>>>> the order
> >>>>>>>> of
> >>>>>>>>>       >>>>>> evaluation.
> >>>>>>>>>       >>>>>>> This solves problem 1 because there are no more
> >>>>>>>>> varargs. It
> >>>>>>>>>       solves
> >>>>>>>>>       >>>>>> problem
> >>>>>>>>>       >>>>>>> 2 because you no longer lean on ordering to access the
> >>>>>>>>>       branch you’re
> >>>>>>>>>       >>>>>>> interested in. It solves problem 3 because you can
> >>>>>>>>> introduce
> >>>>>>>>>       another
> >>>>>>>>>       >>>>>>> conditional by simply attaching another name to the
> >>>>>>>>>       structure, rather
> >>>>>>>>>       >>>>>> than
> >>>>>>>>>       >>>>>>> messing with the existing indices.
> >>>>>>>>>       >>>>>>>
> >>>>>>>>>       >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>> inline is
> >>>>>>>>>       historically
> >>>>>>>>>       >>>>>>> awkward in Java. I know it’s an anti-pattern to use
> >>>>>>>>>       voluminously, but
> >>>>>>>>>       >>>>>>> double brace initialization would clean up the
> >>>>>>>>> aesthetics.
> >>>>>>>>>       >>>>>>>
> >>>>>>>>>       >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>       <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>       >>>>> wrote:
> >>>>>>>>>       >>>>>>>> Hi Ivan,
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>> Thanks for the update.
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>> FWIW, I agree with Matthias that the current "start
> >>>>>>>> branching"
> >>>>>>>>>       >>>>> operator
> >>>>>>>>>       >>>>>>> is
> >>>>>>>>>       >>>>>>>> confusing when named the same way as the actual
> >>>>>>>>> branches.
> >>>>>>>>>       "Split"
> >>>>>>>>>       >>>>> seems
> >>>>>>>>>       >>>>>>>> like a good name. Alternatively, we can do without
> >>>>>>>>> a "start
> >>>>>>>>>       >>>>> branching"
> >>>>>>>>>       >>>>>>>> operator at all, and just do:
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>> stream
> >>>>>>>>>       >>>>>>>>      .branch(Predicate)
> >>>>>>>>>       >>>>>>>>      .branch(Predicate)
> >>>>>>>>>       >>>>>>>>      .defaultBranch();
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>> Tentatively, I think that this branching operation
> >>>>>>>>> should be
> >>>>>>>>>       >>>>> terminal.
> >>>>>>>>>       >>>>>>> That
> >>>>>>>>>       >>>>>>>> way, we don't create ambiguity about how to use
> >>>>>>>>> it. That
> >>>>>>>>>       is, `branch`
> >>>>>>>>>       >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>> `defaultBranch` is
> >>>>>>>>>       `void`, to
> >>>>>>>>>       >>>>>>>> enforce that it comes last, and that there is only
> >>>>>>>>> one
> >>>>>>>>>       definition of
> >>>>>>>>>       >>>>>> the
> >>>>>>>>>       >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>> warning if
> >>>>>>>>>       there's no
> >>>>>>>>>       >>>>>>> default,
> >>>>>>>>>       >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>> exception) if a
> >>>>>>>>>       record
> >>>>>>>>>       >>>>>> falls
> >>>>>>>>>       >>>>>>>> though with no default.
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>> Thoughts?
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>> Thanks,
> >>>>>>>>>       >>>>>>>> -John
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
> >>>>>>>>>       >>>>> matthias@confluent.io <ma...@confluent.io>
> >>>>>>>>>       >>>>>>>> wrote:
> >>>>>>>>>       >>>>>>>>
> >>>>>>>>>       >>>>>>>>> Thanks for updating the KIP and your answers.
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> this is to make the name similar to String#split
> >>>>>>>>>       >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>       >>>>>>>>> The intend was to avoid name duplication. The
> >>>>>>>>> return type
> >>>>>>>>>       should
> >>>>>>>>>       >>>>>> _not_
> >>>>>>>>>       >>>>>>>>> be an array.
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> The current proposal is
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> stream.branch()
> >>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>       >>>>>>>>>      .defaultBranch();
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> IMHO, this reads a little odd, because the first
> >>>>>>>>>       `branch()` does
> >>>>>>>>>       >>>>> not
> >>>>>>>>>       >>>>>>>>> take any parameters and has different semantics
> >>>>>>>>> than the
> >>>>>>>> later
> >>>>>>>>>       >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>> snippet above,
> >>>>>>>> it's
> >>>>>>>>>       >>>>> hidden
> >>>>>>>>>       >>>>>>>>> that the first call is `KStream#branch()` while
> >>>>>>>>> the others
> >>>>>>>> are
> >>>>>>>>>       >>>>>>>>> `KBranchedStream#branch()` what makes reading the
> >>>>>>>>> code
> >>>>>>>> harder.
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> Because I suggested to rename `addBranch()` ->
> >>>>>>>>> `branch()`,
> >>>>>>>>>       I though
> >>>>>>>>>       >>>>>> it
> >>>>>>>>>       >>>>>>>>> might be better to also rename `KStream#branch()`
> >>>>>>>>> to avoid
> >>>>>>>> the
> >>>>>>>>>       >>>>> naming
> >>>>>>>>>       >>>>>>>>> overlap that seems to be confusing. The following
> >>>>>>>>> reads
> >>>>>>>> much
> >>>>>>>>>       >>>>> cleaner
> >>>>>>>>>       >>>>>> to
> >>>>>>>>>       >>>>>>>> me:
> >>>>>>>>>       >>>>>>>>> stream.split()
> >>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>       >>>>>>>>>      .defaultBranch();
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> Maybe there is a better alternative to `split()`
> >>>>>>>>> though to
> >>>>>>>>>       avoid
> >>>>>>>>>       >>>>> the
> >>>>>>>>>       >>>>>>>>> naming overlap.
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>> unfortunately
> >>>>>>>> we
> >>>>>>>>>       >>>>> cannot
> >>>>>>>>>       >>>>>>> have
> >>>>>>>>>       >>>>>>>>> a method with such name :-)
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> Bummer. Didn't consider this. Maybe we can still
> >>>>>>>>> come up
> >>>>>>>>>       with a
> >>>>>>>>>       >>>>> short
> >>>>>>>>>       >>>>>>>> name?
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> Can you add the interface `KBranchedStream` to
> >>>>>>>>> the KIP
> >>>>>>>>>       with all
> >>>>>>>>>       >>>>> it's
> >>>>>>>>>       >>>>>>>>> methods? It will be part of public API and should be
> >>>>>>>>>       contained in
> >>>>>>>>>       >>>>> the
> >>>>>>>>>       >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>> return type of
> >>>>>>>>>       >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>       `KBranchedStream#get(int
> >>>>>>>>>       >>>>>>> index)
> >>>>>>>>>       >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>       branched-KStreams. Would
> >>>>>>>>>       >>>>>> be
> >>>>>>>>>       >>>>>>>>> nice to get your feedback about it. It seems you
> >>>>>>>>> suggest
> >>>>>>>>>       that users
> >>>>>>>>>       >>>>>>>>> would need to write custom utility code
> >>>>>>>>> otherwise, to
> >>>>>>>>>       access them.
> >>>>>>>>>       >>>>> We
> >>>>>>>>>       >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>> approaches. It
> >>>>>>>> feels
> >>>>>>>>>       >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>> built-in support
> >>>>>>>>>       to get
> >>>>>>>>>       >>>>> the
> >>>>>>>>>       >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>> -Matthias
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>       >>>>>>>>>> Hi all!
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> I have updated the KIP-418 according to the new
> >>>>>>>>> vision.
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>       >>>>>>>>>> I can see your point: this is to make the name
> >>>>>>>>> similar to
> >>>>>>>>>       >>>>>>> String#split
> >>>>>>>>>       >>>>>>>>>> that also returns an array, right? But is it
> >>>>>>>>> worth the
> >>>>>>>>>       loss of
> >>>>>>>>>       >>>>>>>> backwards
> >>>>>>>>>       >>>>>>>>>> compatibility? We can have overloaded branch()
> >>>>>>>>> as well
> >>>>>>>>>       without
> >>>>>>>>>       >>>>>>>> affecting
> >>>>>>>>>       >>>>>>>>>> the existing code. Maybe the old array-based
> >>>>>>>>> `branch`
> >>>>>>>> method
> >>>>>>>>>       >>>>> should
> >>>>>>>>>       >>>>>>> be
> >>>>>>>>>       >>>>>>>>>> deprecated, but this is a subject for discussion.
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>       >>>>> BranchingKStream#branch(),
> >>>>>>>>>       >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>> BranchingKStream#default()
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> Totally agree with 'addBranch->branch' rename.
> >>>>>>>>> 'default'
> >>>>>>>> is,
> >>>>>>>>>       >>>>>>> however, a
> >>>>>>>>>       >>>>>>>>>> reserved word, so unfortunately we cannot have a
> >>>>>>>>> method
> >>>>>>>>>       with such
> >>>>>>>>>       >>>>>>> name
> >>>>>>>>>       >>>>>>>>> :-)
> >>>>>>>>>       >>>>>>>>>>> defaultBranch() does take an `Predicate` as
> >>>>>>>>> argument,
> >>>>>>>> but I
> >>>>>>>>>       >>>>> think
> >>>>>>>>>       >>>>>>> that
> >>>>>>>>>       >>>>>>>>>> is not required?
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> Absolutely! I think that was just copy-paste
> >>>>>>>>> error or
> >>>>>>>>>       something.
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> Dear colleagues,
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> please revise the new version of the KIP and
> >>>>>>>>> Paul's PR
> >>>>>>>>>       >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> Ivan
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>       >>>>>>>>>>> Thanks for driving the discussion of this KIP.
> >>>>>>>>> It seems
> >>>>>>>> that
> >>>>>>>>>       >>>>>>> everybody
> >>>>>>>>>       >>>>>>>>>>> agrees that the current branch() method using
> >>>>>>>>> arrays is
> >>>>>>>> not
> >>>>>>>>>       >>>>>> optimal.
> >>>>>>>>>       >>>>>>>>>>> I had a quick look into the PR and I like the
> >>>>>>>>> overall
> >>>>>>>>>       proposal.
> >>>>>>>>>       >>>>>>> There
> >>>>>>>>>       >>>>>>>>>>> are some minor things we need to consider. I would
> >>>>>>>>>       recommend the
> >>>>>>>>>       >>>>>>>>>>> following renaming:
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>       >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>       >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>       BranchingKStream#default()
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> It's just a suggestion to get slightly shorter
> >>>>>>>>> method
> >>>>>>>> names.
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> In the current PR, defaultBranch() does take an
> >>>>>>>>>       `Predicate` as
> >>>>>>>>>       >>>>>>>> argument,
> >>>>>>>>>       >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>> recently
> >>>>>>>>>       accepted and
> >>>>>>>>>       >>>>>> is
> >>>>>>>>>       >>>>>>>>>>> currently implemented:
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>
> >>>>>>>>>
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> Ie, we should add overloads that accepted a
> >>>>>>>>> `Named`
> >>>>>>>>>       parameter.
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> For the issue that the created `KStream` object
> >>>>>>>>> are in
> >>>>>>>>>       different
> >>>>>>>>>       >>>>>>>> scopes:
> >>>>>>>>>       >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
> >>>>>>>>>       index)` method
> >>>>>>>>>       >>>>>>> that
> >>>>>>>>>       >>>>>>>>>>> returns the corresponding "branched" result
> >>>>>>>>> `KStream`
> >>>>>>>>>       object?
> >>>>>>>>>       >>>>>> Maybe,
> >>>>>>>>>       >>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>> second argument of `addBranch()` should not be a
> >>>>>>>>>       >>>>>> `Consumer<KStream>`
> >>>>>>>>>       >>>>>>>> but
> >>>>>>>>>       >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could
> >>>>>>>>> return
> >>>>>>>>>       whatever
> >>>>>>>>>       >>>>>> the
> >>>>>>>>>       >>>>>>>>>>> `Function` returns?
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> Finally, I would also suggest to update the KIP
> >>>>>>>>> with the
> >>>>>>>>>       current
> >>>>>>>>>       >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>> -Matthias
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>       >>>>>>>>>>>> Ivan,
> >>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>> think it
> >>>>>>>>>       makes sense
> >>>>>>>>>       >>>>>> for
> >>>>>>>>>       >>>>>>>> you
> >>>>>>>>>       >>>>>>>>> to
> >>>>>>>>>       >>>>>>>>>>>> revise the KIP and continue the discussion.
> >>>>>>>>> Obviously
> >>>>>>>>>       we'll
> >>>>>>>>>       >>>>> need
> >>>>>>>>>       >>>>>>>> some
> >>>>>>>>>       >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>> binding votes on
> >>>>>>>>>       >>>>> whether
> >>>>>>>>>       >>>>>>> the
> >>>>>>>>>       >>>>>>>>> KIP
> >>>>>>>>>       >>>>>>>>>>>> could be adopted.  It would be great to hear
> >>>>>>>>> if they
> >>>>>>>>>       think this
> >>>>>>>>>       >>>>>> is
> >>>>>>>>>       >>>>>>> a
> >>>>>>>>>       >>>>>>>>> good
> >>>>>>>>>       >>>>>>>>>>>> idea overall.  I'm not sure if that happens
> >>>>>>>>> just by
> >>>>>>>>>       starting a
> >>>>>>>>>       >>>>>>> vote,
> >>>>>>>>>       >>>>>>>>> or if
> >>>>>>>>>       >>>>>>>>>>>> there is generally some indication of interest
> >>>>>>>> beforehand.
> >>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>> That being said, I'll continue the discussion
> >>>>>>>>> a bit:
> >>>>>>>>>       assuming
> >>>>>>>>>       >>>>> we
> >>>>>>>>>       >>>>>> do
> >>>>>>>>>       >>>>>>>>> move
> >>>>>>>>>       >>>>>>>>>>>> forward the solution of "stream.branch() returns
> >>>>>>>>>       >>>>>> KBranchedStream",
> >>>>>>>>>       >>>>>>> do
> >>>>>>>>>       >>>>>>>>> we
> >>>>>>>>>       >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>> KStream[]"?  I
> >>>>>>>> would
> >>>>>>>>>       >>>>> favor
> >>>>>>>>>       >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>> exclusive APIs
> >>>>>>>> that
> >>>>>>>>>       >>>>>>> accomplish
> >>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>> they're fairly
> >>>>>>>>>       similar
> >>>>>>>>>       >>>>>>>>> anyway.  We
> >>>>>>>>>       >>>>>>>>>>>> just need to be sure we're not making something
> >>>>>>>>>       >>>>>>> impossible/difficult
> >>>>>>>>>       >>>>>>>>> that
> >>>>>>>>>       >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>> structure would
> >>>>>>>> work,
> >>>>>>>>>       >>>>> it's
> >>>>>>>>>       >>>>>>>> just a
> >>>>>>>>>       >>>>>>>>>>>> little sloppy overall in terms of naming and
> >>>>>>>>> clarity. In
> >>>>>>>>>       >>>>>>> particular,
> >>>>>>>>>       >>>>>>>>>>>> passing in the "predicates" and "children"
> >>>>>>>>> lists which
> >>>>>>>> get
> >>>>>>>>>       >>>>>> modified
> >>>>>>>>>       >>>>>>>> in
> >>>>>>>>>       >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>       KStreamLazyBranch is
> >>>>>>>>>       >>>>> a
> >>>>>>>>>       >>>>>>> bit
> >>>>>>>>>       >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>> Thanks,
> >>>>>>>>>       >>>>>>>>>>>> Paul
> >>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
> >>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>       >>>>>>>>> wrote:
> >>>>>>>>>       >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>> I read your code carefully and now I am fully
> >>>>>>>>>       convinced: your
> >>>>>>>>>       >>>>>>>> proposal
> >>>>>>>>>       >>>>>>>>>>>>> looks better and should work. We just have to
> >>>>>>>>> document
> >>>>>>>> the
> >>>>>>>>>       >>>>>> crucial
> >>>>>>>>>       >>>>>>>>> fact
> >>>>>>>>>       >>>>>>>>>>>>> that KStream consumers are invoked as they're
> >>>>>>>>> added.
> >>>>>>>>>       And then
> >>>>>>>>>       >>>>>> it's
> >>>>>>>>>       >>>>>>>> all
> >>>>>>>>>       >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>> What shall we do now? I should re-write the
> >>>>>>>>> KIP and
> >>>>>>>>>       resume the
> >>>>>>>>>       >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>> Why are you telling that your PR 'should not
> >>>>>>>>> be even a
> >>>>>>>>>       >>>>> starting
> >>>>>>>>>       >>>>>>>> point
> >>>>>>>>>       >>>>>>>>> if
> >>>>>>>>>       >>>>>>>>>>>>> we go in this direction'? To me it looks like
> >>>>>>>>> a good
> >>>>>>>>>       starting
> >>>>>>>>>       >>>>>>> point.
> >>>>>>>>>       >>>>>>>>> But
> >>>>>>>>>       >>>>>>>>>>>>> as a novice in this project I might miss some
> >>>>>>>>> important
> >>>>>>>>>       >>>>> details.
> >>>>>>>>>       >>>>>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>> Ivan
> >>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>       >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>       >>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
> >>>>>>>>>       >>>>> stream.branch()
> >>>>>>>>>       >>>>>>>>> solution
> >>>>>>>>>       >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>> consumers will be
> >>>>>>>>>       >>>>> invoked
> >>>>>>>>>       >>>>>> as
> >>>>>>>>>       >>>>>>>>> they’re
> >>>>>>>>>       >>>>>>>>>>>>> added, not during streamsBuilder.build(). So
> >>>>>>>>> the user
> >>>>>>>>>       still
> >>>>>>>>>       >>>>>> ought
> >>>>>>>>>       >>>>>>> to
> >>>>>>>>>       >>>>>>>>> be
> >>>>>>>>>       >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward
> >>>>>>>>> and
> >>>>>>>>>       depend on
> >>>>>>>>>       >>>>> the
> >>>>>>>>>       >>>>>>>>> branched
> >>>>>>>>>       >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>       >>>>>>>>>>>>>> The issue I mean to point out is that it is
> >>>>>>>>> hard to
> >>>>>>>>>       access
> >>>>>>>>>       >>>>> the
> >>>>>>>>>       >>>>>>>>> branched
> >>>>>>>>>       >>>>>>>>>>>>> streams in the same scope as the original
> >>>>>>>>> stream (that
> >>>>>>>>>       is, not
> >>>>>>>>>       >>>>>>>> inside
> >>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>> couponIssuer), which is a problem with both
> >>>>>>>>> proposed
> >>>>>>>>>       >>>>> solutions.
> >>>>>>>>>       >>>>>> It
> >>>>>>>>>       >>>>>>>>> can be
> >>>>>>>>>       >>>>>>>>>>>>> worked around though.
> >>>>>>>>>       >>>>>>>>>>>>>> [Also, great to hear additional interest in
> >>>>>>>>> 401, I’m
> >>>>>>>>>       excited
> >>>>>>>>>       >>>>> to
> >>>>>>>>>       >>>>>>>> hear
> >>>>>>>>>       >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>       >>>>>>>>>>>>>> Paul
> >>>>>>>>>       >>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
> >>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>       >>>>>>>>> wrote:
> >>>>>>>>>       >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> The idea to postpone the wiring of branches
> >>>>>>>>> to the
> >>>>>>>>>       >>>>>>>>>>>>> streamsBuilder.build() also looked great for
> >>>>>>>>> me at
> >>>>>>>> first
> >>>>>>>>>       >>>>> glance,
> >>>>>>>>>       >>>>>>> but
> >>>>>>>>>       >>>>>>>>> ---
> >>>>>>>>>       >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>> available in the
> >>>>>>>>>       same
> >>>>>>>>>       >>>>>> scope
> >>>>>>>>>       >>>>>>> as
> >>>>>>>>>       >>>>>>>>> each
> >>>>>>>>>       >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
> >>>>>>>> together
> >>>>>>>>>       >>>>> again
> >>>>>>>>>       >>>>>> I
> >>>>>>>>>       >>>>>>>>> don't see
> >>>>>>>>>       >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>       >>>>>>>>>>>>>>> You just took the words right out of my
> >>>>>>>>> mouth, I was
> >>>>>>>>>       just
> >>>>>>>>>       >>>>>> going
> >>>>>>>>>       >>>>>>> to
> >>>>>>>>>       >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>       >>>>>>>>>>>>>>> Consider the example from Bill's book, p.
> >>>>>>>>> 101: say
> >>>>>>>>>       we need
> >>>>>>>>>       >>>>> to
> >>>>>>>>>       >>>>>>>>> identify
> >>>>>>>>>       >>>>>>>>>>>>> customers who have bought coffee and made a
> >>>>>>>>> purchase
> >>>>>>>>>       in the
> >>>>>>>>>       >>>>>>>>> electronics
> >>>>>>>>>       >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>       >>>>>>>>>>>>>>> This is the code I usually write under these
> >>>>>>>>>       circumstances
> >>>>>>>>>       >>>>>> using
> >>>>>>>>>       >>>>>>>> my
> >>>>>>>>>       >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>       >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>       >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
> >>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>       >>>>>>>>>>>>>>>       return
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>       >>>>>>>>>>>>>>>       /*In the real world the code here can be
> >>>>>>>>>       complex, so
> >>>>>>>>>       >>>>>>>>> creation of
> >>>>>>>>>       >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>> justified, in
> >>>>>>>>>       order to
> >>>>>>>>>       >>>>>>>> separate
> >>>>>>>>>       >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>       >>>>>>>>>>>>>>>  }
> >>>>>>>>>       >>>>>>>>>>>>>>> }
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>> CouponIssuer();
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>       >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>       >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to
> >>>>>>>>> wire up
> >>>>>>>>>       everything
> >>>>>>>>>       >>>>>>>> later,
> >>>>>>>>>       >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>       >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> Does this make sense?  In order to properly
> >>>>>>>>>       initialize the
> >>>>>>>>>       >>>>>>>>> CouponIssuer
> >>>>>>>>>       >>>>>>>>>>>>> we need the terminal operation to be called
> >>>>>>>>> before
> >>>>>>>>>       >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>       >>>>>>>>>>>>> is called.
> >>>>>>>>>       >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>> KIP-401 is
> >>>>>>>>>       essentially
> >>>>>>>>>       >>>>>> the
> >>>>>>>>>       >>>>>>>>> next
> >>>>>>>>>       >>>>>>>>>>>>> KIP I was going to write here. I have some
> >>>>>>>>> thoughts
> >>>>>>>>>       based on
> >>>>>>>>>       >>>>> my
> >>>>>>>>>       >>>>>>>>> experience,
> >>>>>>>>>       >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
> >>>>>>>>>       >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>       >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>       >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>> concept of a
> >>>>>>>>>       fluent
> >>>>>>>>>       >>>>> API
> >>>>>>>>>       >>>>>>>> based
> >>>>>>>>>       >>>>>>>>>>>>> off of
> >>>>>>>>>       >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>       (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>       >>>>>> and
> >>>>>>>>>       >>>>>>> I
> >>>>>>>>>       >>>>>>>>> think
> >>>>>>>>>       >>>>>>>>>>>>> I
> >>>>>>>>>       >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>       >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>> earlier about
> >>>>>>>>>       >>>>>>> compatibility
> >>>>>>>>>       >>>>>>>>>>>>> issues,
> >>>>>>>>>       >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was
> >>>>>>>>> unaware
> >>>>>>>>>       that Java
> >>>>>>>>>       >>>>> is
> >>>>>>>>>       >>>>>>>> smart
> >>>>>>>>>       >>>>>>>>>>>>> enough to
> >>>>>>>>>       >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
> >>>>>>>>>       returning one
> >>>>>>>>>       >>>>>>> thing
> >>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>       >>>>>>>>>>>>> branch()
> >>>>>>>>>       >>>>>>>>>>>>>>>>    with no arguments returning another thing.
> >>>>>>>>>       >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't
> >>>>>>>>> actually
> >>>>>>>>>       need
> >>>>>>>>>       >>>>> it.
> >>>>>>>>>       >>>>>>> We
> >>>>>>>>>       >>>>>>>>> can
> >>>>>>>>>       >>>>>>>>>>>>> just
> >>>>>>>>>       >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>> KBranchedStream who
> >>>>>>>>>       shares
> >>>>>>>>>       >>>>>> its
> >>>>>>>>>       >>>>>>>>> state
> >>>>>>>>>       >>>>>>>>>>>>> with the
> >>>>>>>>>       >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do
> >>>>>>>>> the
> >>>>>>>>>       branching.
> >>>>>>>>>       >>>>>>> It's
> >>>>>>>>>       >>>>>>>>> not
> >>>>>>>>>       >>>>>>>>>>>>> terribly
> >>>>>>>>>       >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
> >>>>>>>>>       demonstrates
> >>>>>>>>>       >>>>>> its
> >>>>>>>>>       >>>>>>>>>>>>> feasibility.
> >>>>>>>>>       >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>> request should
> >>>>>>>> be
> >>>>>>>>>       >>>>> final
> >>>>>>>>>       >>>>>> or
> >>>>>>>>>       >>>>>>>>> even a
> >>>>>>>>>       >>>>>>>>>>>>>>>> starting point if we go in this direction,
> >>>>>>>>> I just
> >>>>>>>>>       wanted to
> >>>>>>>>>       >>>>>> see
> >>>>>>>>>       >>>>>>>> how
> >>>>>>>>>       >>>>>>>>>>>>>>>> challenging it would be to get the API
> >>>>>>>>> working.
> >>>>>>>>>       >>>>>>>>>>>>>>>> I will say though, that I'm not sure the
> >>>>>>>>> existing
> >>>>>>>>>       solution
> >>>>>>>>>       >>>>>>> could
> >>>>>>>>>       >>>>>>>> be
> >>>>>>>>>       >>>>>>>>>>>>>>>> deprecated in favor of this, which I had
> >>>>>>>>> originally
> >>>>>>>>>       >>>>> suggested
> >>>>>>>>>       >>>>>>>> was a
> >>>>>>>>>       >>>>>>>>>>>>>>>> possibility.  The reason is that the newly
> >>>>>>>>> branched
> >>>>>>>>>       streams
> >>>>>>>>>       >>>>>> are
> >>>>>>>>>       >>>>>>>> not
> >>>>>>>>>       >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>> other.  That
> >>>>>>>>>       is, if we
> >>>>>>>>>       >>>>>>> wanted
> >>>>>>>>>       >>>>>>>>> to
> >>>>>>>>>       >>>>>>>>>>>>> merge
> >>>>>>>>>       >>>>>>>>>>>>>>>> them back together again I don't see a way
> >>>>>>>>> to do
> >>>>>>>>>       that.  The
> >>>>>>>>>       >>>>>> KIP
> >>>>>>>>>       >>>>>>>>>>>>> proposal
> >>>>>>>>>       >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>> means is that
> >>>>>>>> for
> >>>>>>>>>       >>>>>> either
> >>>>>>>>>       >>>>>>>>>>>>> solution,
> >>>>>>>>>       >>>>>>>>>>>>>>>> deprecating the existing branch(...) is
> >>>>>>>>> not on the
> >>>>>>>>>       table.
> >>>>>>>>>       >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>       >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>       >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan
> >>>>>>>>> Ponomarev <
> >>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
> >>>>>>>>>       >>>>>>>>>>>>> wrote:
> >>>>>>>>>       >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>> discussed up to
> >>>>>>>> this
> >>>>>>>>>       >>>>>> point.
> >>>>>>>>>       >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed
> >>>>>>>>> that
> >>>>>>>>>       branch API
> >>>>>>>>>       >>>>>>> needs
> >>>>>>>>>       >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> There are two potential ways to do it:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>       >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...)....
> >>>>>>>>> //onTopOf
> >>>>>>>>>       returns
> >>>>>>>>>       >>>>>> its
> >>>>>>>>>       >>>>>>>>> argument
> >>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2)
> >>>>>>>>> The code
> >>>>>>>> won't
> >>>>>>>>>       >>>>> make
> >>>>>>>>>       >>>>>>>> sense
> >>>>>>>>>       >>>>>>>>>>>>> until
> >>>>>>>>>       >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>       instance
> >>>>>>>>>       >>>>>>>>> contrasts the
> >>>>>>>>>       >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>> noDefault(). Both
> >>>>>>>>>       >>>>>>>>> defaultBranch(..)
> >>>>>>>>>       >>>>>>>>>>>>> and
> >>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams
> >>>>>>>>> interface
> >>>>>>>> is
> >>>>>>>>>       >>>>>> defined.
> >>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
> >>>>>>>>>       >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very
> >>>>>>>>> easy to
> >>>>>>>>>       miss the
> >>>>>>>>>       >>>>>> fact
> >>>>>>>>>       >>>>>>>>> that one
> >>>>>>>>>       >>>>>>>>>>>>>>>>> of the terminal methods should be called.
> >>>>>>>>> If these
> >>>>>>>>>       methods
> >>>>>>>>>       >>>>>> are
> >>>>>>>>>       >>>>>>>> not
> >>>>>>>>>       >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>> runtime.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can
> >>>>>>>>> we do
> >>>>>>>> better?
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> I see your point when you are talking
> >>>>>>>>> about
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
> >>>>>>>>>       implemented the
> >>>>>>>>>       >>>>>>> easy
> >>>>>>>>>       >>>>>>>>> way.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
> >>>>>>>> assumes
> >>>>>>>>>       >>>>> nothing
> >>>>>>>>>       >>>>>>>> will
> >>>>>>>>>       >>>>>>>>>>>>> reach
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> throwing an exception if such a case
> >>>>>>>>> occurs.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be
> >>>>>>>>> the only
> >>>>>>>> option
> >>>>>>>>>       >>>>>> besides
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios
> >>>>>>>>> when we
> >>>>>>>>>       want to
> >>>>>>>>>       >>>>>> just
> >>>>>>>>>       >>>>>>>>> silently
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
> >>>>>>>>>       predicate. 2)
> >>>>>>>>>       >>>>>>> Throwing
> >>>>>>>>>       >>>>>>>>> an
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> exception in the middle of data flow
> >>>>>>>>> processing
> >>>>>>>>>       looks
> >>>>>>>>>       >>>>>> like a
> >>>>>>>>>       >>>>>>>> bad
> >>>>>>>>>       >>>>>>>>>>>>> idea.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would
> >>>>>>>>> prefer to
> >>>>>>>>>       emit a
> >>>>>>>>>       >>>>>>>> special
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is
> >>>>>>>>> exactly
> >>>>>>>> where
> >>>>>>>>>       >>>>>>> `default`
> >>>>>>>>>       >>>>>>>>> can
> >>>>>>>>>       >>>>>>>>>>>>> be
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>       >>>>> InternalTopologyBuilder
> >>>>>>>>>       >>>>>>> to
> >>>>>>>>>       >>>>>>>>> track
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> branches that haven't been terminated
> >>>>>>>>> and raise
> >>>>>>>>>       a clear
> >>>>>>>>>       >>>>>>> error
> >>>>>>>>>       >>>>>>>>>>>>> before it
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the
> >>>>>>>>> program is
> >>>>>>>>>       >>>>> compiled
> >>>>>>>>>       >>>>>>> and
> >>>>>>>>>       >>>>>>>>> run?
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
> >>>>>>>>>       compile if
> >>>>>>>>>       >>>>> used
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
> >>>>>>>>>       method chain
> >>>>>>>>>       >>>>>>>> starting
> >>>>>>>>>       >>>>>>>>>>>>> from
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost
> >>>>>>>>> difference
> >>>>>>>>>       between
> >>>>>>>>>       >>>>>>>> runtime
> >>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure
> >>>>>>>>> uncovers
> >>>>>>>>>       >>>>> instantly
> >>>>>>>>>       >>>>>> on
> >>>>>>>>>       >>>>>>>>> unit
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> tests, it costs more for the project
> >>>>>>>>> than a
> >>>>>>>>>       compilation
> >>>>>>>>>       >>>>>>>> failure.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>> operation being
> >>>>>>>>>       required.
> >>>>>>>>>       >>>>>>> But
> >>>>>>>>>       >>>>>>>> is
> >>>>>>>>>       >>>>>>>>>>>>> that
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't
> >>>>>>>>> want a
> >>>>>>>>>       >>>>>> defaultBranch
> >>>>>>>>>       >>>>>>>>> they
> >>>>>>>>>       >>>>>>>>>>>>> can
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>       just as
> >>>>>>>>>       >>>>>>>>> easily.  In
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a
> >>>>>>>>> nicer API
> >>>>>>>> - a
> >>>>>>>>>       >>>>> user
> >>>>>>>>>       >>>>>>>> could
> >>>>>>>>>       >>>>>>>>>>>>> specify
> >>>>>>>>>       >>>>>>>>>>>>>>>>> a
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing
> >>>>>>>>> will reach
> >>>>>>>> the
> >>>>>>>>>       >>>>>> default
> >>>>>>>>>       >>>>>>>>> branch,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case
> >>>>>>>>> occurs.
> >>>>>>>> That
> >>>>>>>>>       >>>>> seems
> >>>>>>>>>       >>>>>>> like
> >>>>>>>>>       >>>>>>>>> an
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> improvement over the current branch()
> >>>>>>>>> API,
> >>>>>>>>>       which allows
> >>>>>>>>>       >>>>>> for
> >>>>>>>>>       >>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>> more
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
> >>>>>>>> dropped.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>> certainly has
> >>>>>>>>>       to be
> >>>>>>>>>       >>>>>> well
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>       >>>>> InternalTopologyBuilder
> >>>>>>>>>       >>>>>>> to
> >>>>>>>>>       >>>>>>>>> track
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated
> >>>>>>>>> and raise
> >>>>>>>>>       a clear
> >>>>>>>>>       >>>>>>> error
> >>>>>>>>>       >>>>>>>>>>>>> before it
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that
> >>>>>>>>> there is
> >>>>>>>> a
> >>>>>>>>>       >>>>> "build
> >>>>>>>>>       >>>>>>>> step"
> >>>>>>>>>       >>>>>>>>>>>>> where
> >>>>>>>>>       >>>>>>>>>>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
> >>>>>>>>>       >>>>>> StreamsBuilder.build()
> >>>>>>>>>       >>>>>>> is
> >>>>>>>>>       >>>>>>>>>>>>> called.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>> argument, I
> >>>>>>>> agree
> >>>>>>>>>       >>>>> that
> >>>>>>>>>       >>>>>>> it's
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> allow users to do other operations on
> >>>>>>>>> the input
> >>>>>>>>>       stream.
> >>>>>>>>>       >>>>>>> With
> >>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same
> >>>>>>>>> way all
> >>>>>>>> other
> >>>>>>>>>       >>>>>>> operations
> >>>>>>>>>       >>>>>>>>> do -
> >>>>>>>>>       >>>>>>>>>>>>> if
> >>>>>>>>>       >>>>>>>>>>>>>>>>> you
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
> >>>>>>>> multiple
> >>>>>>>>>       >>>>> times,
> >>>>>>>>>       >>>>>>> you
> >>>>>>>>>       >>>>>>>>> just
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call
> >>>>>>>>> as many
> >>>>>>>>>       operations
> >>>>>>>>>       >>>>>> on
> >>>>>>>>>       >>>>>>> it
> >>>>>>>>>       >>>>>>>>> as
> >>>>>>>>>       >>>>>>>>>>>>> you
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan
> >>>>>>>>> Ponomarev <
> >>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we
> >>>>>>>>> do not
> >>>>>>>>>       always need
> >>>>>>>>>       >>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
> >>>>>>>> operation we
> >>>>>>>>>       >>>>> don't
> >>>>>>>>>       >>>>>>>> know
> >>>>>>>>>       >>>>>>>>>>>>> when to
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its
> >>>>>>>>> argument,
> >>>>>>>>>       so we
> >>>>>>>>>       >>>>> can
> >>>>>>>>>       >>>>>> do
> >>>>>>>>>       >>>>>>>>>>>>> something
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> more with the original branch after
> >>>>>>>>> branching.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
> >>>>>>>> special
> >>>>>>>>>       >>>>> object
> >>>>>>>>>       >>>>>>>>>>>>> construction
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream
> >>>>>>>>> methods.
> >>>>>>>> But
> >>>>>>>>>       >>>>> here
> >>>>>>>>>       >>>>>> we
> >>>>>>>>>       >>>>>>>>> have a
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to
> >>>>>>>>> split the
> >>>>>>>>>       flow,
> >>>>>>>>>       >>>>> so
> >>>>>>>>>       >>>>>> I
> >>>>>>>>>       >>>>>>>>> think
> >>>>>>>>>       >>>>>>>>>>>>> this
> >>>>>>>>>       >>>>>>>>>>>>>>>>> is
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve
> >>>>>>>>> this
> >>>>>>>>>       API, but I
> >>>>>>>>>       >>>>>> find
> >>>>>>>>>       >>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
> >>>>>>>>>       contrasts the
> >>>>>>>>>       >>>>>>> fluency
> >>>>>>>>>       >>>>>>>>> of
> >>>>>>>>>       >>>>>>>>>>>>> other
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd
> >>>>>>>>> like to
> >>>>>>>>>       just call
> >>>>>>>>>       >>>>> a
> >>>>>>>>>       >>>>>>>>> method on
> >>>>>>>>>       >>>>>>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if
> >>>>>>>>> the branch
> >>>>>>>>>       cases
> >>>>>>>>>       >>>>> are
> >>>>>>>>>       >>>>>>>>> defined
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate,
> >>>>>>>>> handleCase)
> >>>>>>>>>       is very
> >>>>>>>>>       >>>>>> nice
> >>>>>>>>>       >>>>>>>>> and the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped
> >>>>>>>>> around
> >>>>>>>>>       how we
> >>>>>>>>>       >>>>>>> specify
> >>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>> source
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
> >>>>>>>> this::handle1)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
> >>>>>>>> this::handle2)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>> KBranchedStreams or
> >>>>>>>>>       >>>>>>>> KStreamBrancher
> >>>>>>>>>       >>>>>>>>> or
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
> >>>>>>>>>       terminated by
> >>>>>>>>>       >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
> >>>>>>>>>       incompatible with
> >>>>>>>>>       >>>>> the
> >>>>>>>>>       >>>>>>>>> current
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to
> >>>>>>>>> have a
> >>>>>>>>>       different
> >>>>>>>>>       >>>>>> name,
> >>>>>>>>>       >>>>>>>> but
> >>>>>>>>>       >>>>>>>>> that
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we
> >>>>>>>>> could call it
> >>>>>>>>>       >>>>>> something
> >>>>>>>>>       >>>>>>>> like
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the
> >>>>>>>>> old API.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of
> >>>>>>>>> your
> >>>>>>>>>       KIP?  It
> >>>>>>>>>       >>>>>> seems
> >>>>>>>>>       >>>>>>>>> like it
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>> branching
> >>>>>>>>>       while also
> >>>>>>>>>       >>>>>>>> allowing
> >>>>>>>>>       >>>>>>>>> you
> >>>>>>>>>       >>>>>>>>>>>>> to
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
> >>>>>>>>>       KBranchedStreams
> >>>>>>>>>       >>>>>> if
> >>>>>>>>>       >>>>>>>>> desired.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan
> >>>>>>>>> Ponomarev
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>       ks){
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
> >>>>>>>>>       String> ks){
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String,
> >>>>>>>>> String>()
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>       this::handleFirstCase)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>       this::handleSecondCase)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>       >>>>> takes a
> >>>>>>>>>       >>>>>>>>> Consumer
> >>>>>>>>>       >>>>>>>>>>>>> as a
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing,
> >>>>>>>>> and the
> >>>>>>>>>       example in
> >>>>>>>>>       >>>>>> the
> >>>>>>>>>       >>>>>>>> KIP
> >>>>>>>>>       >>>>>>>>>>>>> shows
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a
> >>>>>>>>> terminal node
> >>>>>>>>>       >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but
> >>>>>>>>> how would
> >>>>>>>> we
> >>>>>>>>>       >>>>> handle
> >>>>>>>>>       >>>>>>> the
> >>>>>>>>>       >>>>>>>>> case
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>> wants to
> >>>>>>>> continue
> >>>>>>>>>       >>>>>>>> processing
> >>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>       >>>>>>>>>>>>> not
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on
> >>>>>>>>> the
> >>>>>>>> branched
> >>>>>>>>>       >>>>>> stream
> >>>>>>>>>       >>>>>>>>>>>>> immediately?
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic
> >>>>>>>>> as is if
> >>>>>>>>>       we had
> >>>>>>>>>       >>>>>>>> something
> >>>>>>>>>       >>>>>>>>> like
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM
> >>>>>>>>> Bill Bejeck
> >>>>>>>> <
> >>>>>>>>>       >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>> discussion for
> >>>>>>>> KIP-
> >>>>>>>>>       >>>>> 418.
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
> >>>>>>>> KIP-418.
> >>>>>>>>>       >>>>> Please
> >>>>>>>>>       >>>>>>>> take
> >>>>>>>>>       >>>>>>>>> a
> >>>>>>>>>       >>>>>>>>>>>>> look
> >>>>>>>>>       >>>>>>>>>>>>>>>>> at
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>> appreciate any
> >>>>>>>>>       feedback :)
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>       >>>>>
> >>>>>>>>>
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>       >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>       >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>       >>>>>>>>>
> >>>>>>>>>       >
> >>>>>>>>>
> >>>>>>>>
> >>>>>
> >>>>
> >>>
> >>
> > 
> 
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi all!

OK, we have two concerns:

1. John's proposal to add an overloaded version of Branched.with(name?, 
Consumer<KStream>) (my opinion: yes, it must be a good idea).

2. To skip or not to skip null values if they are returned by provided 
function (my current opinion: to skip).

First, let me reason from functional programming point of view. Well, in 
fact, I don't know any of 'pure FP' languages, but I'm somewhat literate 
in set theory and category theory. So actually I'm going to reason from 
mathematical point of view :-)

Java's consumer is a function with a codomain equals to {0}, aka Unit. A 
usual Java function with a nullable result of type T is a function with 
a codomain T ∪ {0}. So, in 'pure functional world', Java Consumer should 
be type compatible with Java Function: Consumer is just a function that 
maps everything to an empty set, while a 'nullable function' can map to 
either an empty set or something from its return type.

In Java's type system Consumer and Function are incompatible. But we can 
add an overloaded version of `with` per John's suggestion and make them 
compatible at least for our API.

This reasoning implies that for the sake of uniformity we should treat 
both returned nulls and consumers the same way, i.e. either put nulls 
for Consumers, or not put nulls at all. The second seems more reasonable!

Besides 'mathematical' arguments in favour of skipping nulls in the Map, 
there are 'programming' arguments that John mentioned in his reply (many 
Map implementations don't allow nulls etc), and I can't agree with him 
more. And yes, there is also a practical argument: what null values in 
the map are good for besides getting an NPE?

So, I would like to add an overloaded `with` to the KIP and don't change 
the rest of it.

Where can I be wrong?

Regards,

Ivan



29.05.2020 23:20, John Roesler пишет:
> Hi all,
> 
> Right, that was my suggestion; sorry for the confusion. I was thinking
> that adding null as a value to maps is generally dubious, for which
> reason a lot of Map implementations actually don't allow it at all.
> 
> The reason is that it creates ambiguity, since a lot of code treats
> "map.get(key) == null" as meaning the key is not in the map, but
> if we place a null value in the map (presuming we don't immediately
> get an exception), then you face weird contradictions, like
> get(key) == null, but containsKey(key) can be either true or false, and
> you may or may not see it while iterating. Which would violate the
> expectations of many Java programs. You can just take a look at
> the AK codebase, and you'll find many occurrences where we
> assume a null mapping means the map doesn't contain the key.
> 
> Of course, the practical question, which Ivan brought up, is also
> a good one. What would you actually do with these null values
> in the map, besides get an NPE ?
> 
> I'm not sure why I didn't think of this before, but an alternative
> to this debate is to go back to adding Consumer into the API, but
> with the same method name, withChain(Consumer<KStream<K,V>>).
> 
> Now that we have only static methods, it's less confusing because it's
> not possible to choose _both_ the Function<KStream,KStream> _and_
> the Consumer<KStream> in the same branch (which was one of the
> main drawbacks of the original plan wrt withChain and
> withJavaConsumer as builder methods.
> 
> I.e., I was previously concerned about:
> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2)
> It's just a little confusing trying to reason about how this is all going to wire up.
> 
> But with only the static methods, users have only three methods,
> and they are all clear:
> 1. Branched.as(name): just names the branch, the branch-predicated stream is the value of the map
> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the branch, applies the transformation after the branch predicate, and the transformed stream is the value of the map (null is not permitted)
> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch, passes the predicated stream in to the Consumer function and omits the branch from the map
> 
> Sorry for digging this option up again, but it's starting to look more
> attractive after we dropped the builder functions, and I think it also
> resolves the "null" concern.
> 
> Thoughts?
> -John
> 
> 
> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>> Hi Matthias!
>>
>> Thanks for your reply!
>>
>> (1)
>>
>>   > Do you imply that there won't be a `name -> null` entry in the `Map`
>> for the branch? If yes, I am wondering why?
>>
>> Ah, I re-read the discussion and found out that omitting null entries
>> was John's idea, not yours :-)
>>
>> Anyway. As you remember, I was against the 'swallowing' of branches by
>> Consumer. I had a feeling that there are cases when I would like both to
>> consume a branch and post-process it in a Map handler.
>>
>> But then the idea of getting rid of `withJavaConsumer` came up. And then
>> we realized that we should handle nulls somehow.
>>
>> So it looked like taking the best from both approaches: if we want to
>> omit a branch in the resulting map, we just return null, thus emulating
>> a 'swallowing consumer'. If we want to include a branch, we return it.
>> It provides the full control for a user, and it spares the user from
>> null-checking in their code.
>>
>>   > My reasoning is, that it's very clear that `name -> null` would be
>> there, because the user code did execute `return null`.
>>
>> My reasoning is following. The way such lambdas are written, it's hard
>> to return null unintentionally. And if a user returns null
>> intentionally, what `name->null` in the resulting Map can be good for?
>> It's either NPE (which is clearly not intended) or something that should
>> be null-checked and skipped. Ok, let's do it!
>>
>> But, I'm ready to be convinced to drop this rule. It doesn't seem to me
>> to be a principal thing.
>>
>> (2)
>>
>>   > I think it would be good if the KIP could explains the intended
>> default naming schema. This is important because the naming schema must
>> be part of the API contract; otherwise, users cannot rely on the naming
>> when trying to use the returned `Map`.
>>
>> Well, in fact, I intentionally didn't include any naming schema in the
>> KIP. And it's done exactly for the purpose that a user won't be able to
>> rely on default naming. If they want to use certain branches from the
>> Map, they will devise their own naming strategy that will never change
>> and will never let them down.
>>
>> If they don't mind what's in a Map, if they are looking up the Map just
>> for debugging, or if they want to treat a Map just like a Collection of
>> values (a possible use case!), it doesn't matter how the branches are named.
>>
>> This way we are giving ourselves the full freedom to change the naming
>> schema afterwards without violating the API contract.
>>
>>
>> Regards,
>>
>> Ivan
>>
>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>> Thanks for updating the KIP!
>>>
>>> Using covariant generics is a good idea! I am also fine with only using
>>> static method in `Branched` for now, as it's only two parameters and
>>> thus not too many overloads.
>>>
>>>
>>> (1) What is unclear to me is, what you exaclty mean by:
>>>
>>>> If a function returns `null`, its result is omitted.
>>>
>>> Do you imply that there won't be a `name -> null` entry in the `Map` for
>>> the branch? If yes, I am wondering why?
>>>
>>> My previous argument was to omit an entry only for the
>>> `withJavaConsumer()` case, because the return type is "void" (and thus
>>> it's unclear what should be added and it would introduce inconsistencies
>>> in the `Map`).
>>>
>>> However, for a `Function` that returns a `KStream`, we could actually
>>> just add a `name -> null` entry if `null` is returned. My reasoning is,
>>> that it's very clear that `name -> null` would be there, because the
>>> user code did execute `return null`. Adding a `name -> null` entry for
>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>> exceptional case for handling `null`).
>>>
>>> Thoughts?
>>>
>>>
>>> (2) For `Branched.as()` the KIP says:
>>>
>>>> sets the name of the branch (auto-generated by default, when split operation is named, then the names are index-suffixed)
>>>
>>> I think it would be good if the KIP could explains the intended default
>>> naming schema. This is important because the naming schema must be part
>>> of the API contract; otherwise, users cannot rely on the naming when
>>> trying to use the returned `Map`.
>>>
>>> Similalry, it should be explained how names are generated if
>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations how the
>>> API can be mixed and matched and it's unclear atm how it would work in
>>> detail.
>>>
>>>
>>> Thanks!
>>>
>>>
>>> -Matthias
>>>
>>>
>>>
>>>
>>>
>>>
>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>> Hi Ivan,
>>>>
>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>> that have been raised in the discussion so far have been
>>>> addressed. And it's been a while since anyone raised a new
>>>> concern. At this point, it seems like a good time to start
>>>> the VOTE thread.
>>>>
>>>> Sometimes, the vote thread will trigger new people to look
>>>> into the KIP, and they may raise new concerns, but it's not a
>>>> problem. We'll just address those lingering concerns if there
>>>> are any, until you have all the votes you need.
>>>>
>>>> Thanks again for the contribution!
>>>>
>>>> -John
>>>>
>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>
>>>>> 1. Per Mathhias's suggestion, if a chain function returns null, the
>>>>> respective result if omitted in the resulting Map.
>>>>>
>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>
>>>>> 3. `Branched` class has only three static methods with all the possible
>>>>> combinations of parameters.
>>>>>
>>>>> 4. Chain function is defined 'fully covariant', let's see if we can
>>>>> implement it this way :-))
>>>>>
>>>>> + code example updates and minor edits.
>>>>>
>>>>>
>>>>> Since this is my first KIP, I'm not sure what should I do next. I feel
>>>>> that we talked over all the details and the consensus is reached. Is it
>>>>> OK to call for VOTE now or is it better to wait for more feedback?
>>>>>
>>>>> Regards,
>>>>>
>>>>> Ivan
>>>>>
>>>>>
>>>>>
>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>> Sounds good to me, Ivan!
>>>>>> -John
>>>>>>
>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>> John,
>>>>>>>
>>>>>>> ---------------------
>>>>>>>
>>>>>>>     > I'm sorry things have been dragging out a little, but I have the
>>>>>>> sense we're very close to the end of this discussion, which is exciting.
>>>>>>>
>>>>>>> We are certainly moving forward!  And I'm not in a hurry at all. As I
>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>> KafkaStreamBrancher -- the implementation of the first, rejected version
>>>>>>> of this KIP. It's inferior to what we are discussing here, but it does
>>>>>>> its work. So it's worth to design this KIP really, really well. And by
>>>>>>> the way, from this discussion I'm learning the good API designing
>>>>>>> process. For me it has a value per se :-))
>>>>>>>
>>>>>>> ----------------------
>>>>>>>
>>>>>>>     >>   > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>>     >>   > Do they provide any benefit over just using the right static
>>>>>>> factory?
>>>>>>>     >
>>>>>>>     > I don't have a strong feeling, either. It seems nice to offer a better
>>>>>>>     > type inference experience than what we get with Materialized, by
>>>>>>>     > offering the static method that takes both name and chain.
>>>>>>>     > Given that, there doesn't seem to be a good reason to also offer the
>>>>>>>     > non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>>>>
>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I found out
>>>>>>> an interesting fact about type inference rules in Java. Funny thing is
>>>>>>> that although we need to explicitly set types in a chain like this
>>>>>>>
>>>>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>>>>>>
>>>>>>> (otherwise it won't compile), the composition of static method calls
>>>>>>> works just fine, all the types are being calculated correctly:
>>>>>>>
>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>>>>>>
>>>>>>> As I was told, for type inference there is difference between qualifiers
>>>>>>> and arguments, 'you go up if you are argument but stop if you are
>>>>>>> qualifier'. And it also seems that we should not bet on any future
>>>>>>> improvements in Java type inference here.
>>>>>>>
>>>>>>> So,
>>>>>>>
>>>>>>> 1) I think we that in this KIP we should provide three static methods
>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and
>>>>>>> drop any non-static ones.
>>>>>>>
>>>>>>> 2) If anything else will be ever needed, we can easily add anything.
>>>>>>> Maybe this can be done in a process of refinement of all the parameter
>>>>>>> classes.
>>>>>>>
>>>>>>> ----------------------------
>>>>>>>
>>>>>>>     > we may as well hope for the best, and propose the "fully
>>>>>>>     > covariant" definition for now.
>>>>>>>
>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>
>>>>>>> ----------------------------
>>>>>>>
>>>>>>>     >> Good question, I already thought about it and rejected the idea....
>>>>>>>     > (I cut off your quote; the rest is in the chain below)
>>>>>>>     >  "Worst" case scenario: someone
>>>>>>>     > else will wish the return type is something different, and we'll go
>>>>>>>     > through a painless deprecation transition to change it later.
>>>>>>>
>>>>>>> Of course, we cant' predict all the ways people are going to use it.
>>>>>>>     From my own humble experience with Kafka Streams, the worst scenario is
>>>>>>> unlikely. Split is split, transform is transform, too much flexibility
>>>>>>> is often evil.
>>>>>>>
>>>>>>> ------------------------
>>>>>>>
>>>>>>> So it seems that we are close to the consensus. Two things to be altered
>>>>>>> in the current version of KIP:
>>>>>>>
>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>
>>>>>>> Any ideas / questions / objections?
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>
>>>>>>>> I'm sorry things have been dragging out a little, but I have the sense
>>>>>>>> we're very close to the end of this discussion, which is exciting.
>>>>>>>>
>>>>>>>>>      > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>>>>      > Do they provide any benefit over just using the right static factory?
>>>>>>>>>
>>>>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>>>>> be omitted, I agree!
>>>>>>>>
>>>>>>>> I don't have a strong feeling, either. It seems nice to offer a better
>>>>>>>> type inference experience than what we get with Materialized, by
>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>
>>>>>>>> Given that, there doesn't seem to be a good reason to also offer the
>>>>>>>> non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>>>>>
>>>>>>>> I'll defer to Matthias, if he has a chance to consider whether it's more
>>>>>>>> valuable to stick with the existing pattern or break the pattern to offer
>>>>>>>> a better experience.
>>>>>>>>
>>>>>>>>>      > You might as well propose the “ideal” API in the KIP, which is the
>>>>>>>>>      > covariant typed function
>>>>>>>>>
>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>>>>> and unit testing?
>>>>>>>>
>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been that we just
>>>>>>>> won't know what works until we really try it, and try it in several different
>>>>>>>> ways. Still, we may as well hope for the best, and propose the "fully
>>>>>>>> covariant" definition for now.
>>>>>>>>
>>>>>>>>>      > Is it necessary to restrict the result key and value types to be the
>>>>>>>>>      > same as the inputs?
>>>>>>>>>
>>>>>>>>> Good question, I already thought about it and rejected the idea....
>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>
>>>>>>>> That's fair! It's your KIP, after all. I think I might have made a different
>>>>>>>> call here, but I think this choice is fine. "Worst" case scenario: someone
>>>>>>>> else will wish the return type is something different, and we'll go
>>>>>>>> through a painless deprecation transition to change it later. Thanks
>>>>>>>> to the clean design of your API, this doesn't seem to bad. And, of
>>>>>>>> course, you've actually been using similar functionality already, so it
>>>>>>>> seems we should trust your intuition.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> -John
>>>>>>>>
>>>>>>>>
>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>> Hi John,
>>>>>>>>>
>>>>>>>>>      > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>>>> Do they provide any benefit over just using the right static factory?
>>>>>>>>>
>>>>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>>>>> be omitted, I agree!
>>>>>>>>>
>>>>>>>>>      > You might as well propose the “ideal” API in the KIP, which is the
>>>>>>>>> covariant typed function
>>>>>>>>>
>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>>>>> and unit testing?
>>>>>>>>>
>>>>>>>>>      > Is it necessary to restrict the result key and value types to be the
>>>>>>>>> same as the inputs?
>>>>>>>>>
>>>>>>>>> Good question, I already thought about it and rejected the idea.
>>>>>>>>>
>>>>>>>>> Look, if we want to keep `withChain`'s function optional, then we must
>>>>>>>>> keep the result key and value types the same. Because for now, the
>>>>>>>>> default value for the 'chain function' is Function.identity().
>>>>>>>>>
>>>>>>>>> Of course, we can make the 'chain function' required. But I think this
>>>>>>>>> is not what `split` method is for. `split` is for splitting, not
>>>>>>>>> transforming, and `chainFunction` in most of the cases should be either
>>>>>>>>> a consumer or the identity function.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>
>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
>>>>>>>>>>
>>>>>>>>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
>>>>>>>>>>
>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
>>>>>>>>>>
>>>>>>>>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
>>>>>>>>>>
>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
>>>>>>>>>>
>>>>>>>>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> John
>>>>>>>>>>
>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>> Hello John,
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 1.
>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>
>>>>>>>>>>>       > Perhaps it would be better to stick with "as" for now
>>>>>>>>>>>       > and just file a Jira to switch them all at the same time [for
>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>
>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
>>>>>>>>>>> standard workaround
>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>>>>>>> So actually this should be a very low priority issue, if an issue at
>>>>>>>>>>> all.
>>>>>>>>>>>
>>>>>>>>>>>       > I don't understand how your new proposed
>>>>>>>>>>>       > methods would work any differently than the ones you already
>>>>>>>>>>>       > had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>>>>       > the generic type parameters on the first static factory call. Can you
>>>>>>>>>>>       > explain how your new interface proposal differs from the existing KIP?
>>>>>>>>>>>
>>>>>>>>>>> In the KIP, I didn't clarify what methods should be static. Now I
>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>
>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>
>>>>>>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>>>>>>
>>>>>>>>>>> The overloaded `with` version that provides both Function and name can
>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>
>>>>>>>>>>> 2.
>>>>>>>>>>> ----------------------------
>>>>>>>>>>>
>>>>>>>>>>>       > Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>>>>       > be nice, but I'm not sure it will actually work.
>>>>>>>>>>>
>>>>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>>>>
>>>>>>>>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
>>>>>>>>>>>               return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>           s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>                      //Won't compile!!
>>>>>>>>>>>                      Branched.with(Me::func));
>>>>>>>>>>>
>>>>>>>>>>> The simple workaround here is to change `func`'s return type from
>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>
>>>>>>>>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
>>>>>>>>>>> `Branched`, so during code migration I will have to modify my functions'
>>>>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>>>>>>
>>>>>>>>>>>       >  the map you're returning is Map<K,V>, and of course a K is not the
>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>
>>>>>>>>>>> I think what you actually meant here is that KStream<? extends K, ?
>>>>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
>>>>>>>>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
>>>>>>>>>>>
>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
>>>>>>>>>>> for now it's better to just admit that API is not absolutely perfect and
>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>
>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that "as" would
>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from convention
>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>
>>>>>>>>>>>> The convention is that "as(String name)" is used for the static factory
>>>>>>>>>>>> method, whereas "withName(String name)" is an instance method
>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
>>>>>>>>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
>>>>>>>>>>>> although it's somewhat dubious, since all the other configuration
>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
>>>>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>>>>
>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
>>>>>>>>>>>> configuraion objects. The problem is that Java infers the type
>>>>>>>>>>>> based on the _first_ method in the chain. While it does consider what
>>>>>>>>>>>> the recipient of the method result wants, it only considers the _next_
>>>>>>>>>>>> recipient.
>>>>>>>>>>>>
>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>>>>>>> Branched<String,String> variable, java infers the type correctly. But
>>>>>>>>>>>> when the "next recipient" is a chained method call, like "withChain",
>>>>>>>>>>>> then the chained method doesn't bound the type (by definition,
>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java will take
>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
>>>>>>>>>>>>
>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
>>>>>>>>>>>> that hard to recursively consider the entire context when inferring the
>>>>>>>>>>>> generic type parameters, but this is what we have to work with.
>>>>>>>>>>>>
>>>>>>>>>>>> To be honest, though, I don't understand how your new proposed
>>>>>>>>>>>> methods would work any differently than the ones you already
>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>>>>> the generic type parameters on the first static factory call. Can you
>>>>>>>>>>>> explain how your new interface proposal differs from the existing KIP?
>>>>>>>>>>>>
>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
>>>>>>>>>>>> try. In the past, we've run into soe truly strange interactions between the
>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>
>>>>>>>>>>>> Another issue is that the value type of the map you're returning is
>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>> -John
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to implement examples
>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1. Having to return something from lambda is not a very big deal.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use method references
>>>>>>>>>>>>> for already written stream consumers, but then I realized that I can
>>>>>>>>>>>>> just change my methods from returning void to returning the input
>>>>>>>>>>>>> parameter and use references to them. Not very convenient, but passable.
>>>>>>>>>>>>>
>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>>>>>>>>>>>>> function returns null, we don't insert it into the resulting map.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but workable solution as
>>>>>>>>>>>>> a first approximation. And later we can always add to `Branched`
>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
>>>>>>>>>>>>> should be built? First of all, it seems that `as` now is more preferred
>>>>>>>>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>
>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>
>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>           Branched.as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>
>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>           Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>
>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>
>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>
>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>
>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 4. And one more. What do you think, do we need that flexibility:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>>>>>>>>>>>>
>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>
>>>>>>>>>>>>> ??
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>>>>>>>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         (1) split a stream and return the substreams for futher processing
>>>>>>>>>>>>>>>         (2) split a stream and modify the substreams with in-place method chaining
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          -> split a stream, modify the substreams, and return the _modified_
>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         - using `withChain` I get the modified sub-stream
>>>>>>>>>>>>>>>         - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        From my understanding the original idea of `withJavaConsumer` was to
>>>>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>          s.to();
>>>>>>>>>>>>>>>          return null;
>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Following this train of through, and if we want to allow the "return
>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>          s.to();
>>>>>>>>>>>>>>>          return s;
>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>>>>>>>>>>>>> while for the first proposal it adds new functionality (if `return null`
>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we follow your
>>>>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>>>>>>>>>>>>> implementation would be to not add the "branch" to the result map if
>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the returned map
>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>>>>>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>>>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>>>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>              .branch(....)
>>>>>>>>>>>>>>>>>>>>>>              .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>              .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>               withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>              stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>                         .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>                         .defaultBranch(...) // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>               operator(function, config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             operator(function, function, string)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>              KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>              interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>               withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>               withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>               withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>             }
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>             -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>             -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               me address both comments here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> An open question is the case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>           ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        ��      >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>>
>>>
>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <mj...@apache.org>.
I am happy with the KIP. Was actually waiting for John to reply :)


On 6/26/20 11:06 AM, John Roesler wrote:
> Hello Ivan,
> 
> I’m sorry for the delay. I was waiting to see if Matthias wanted to ask any follow-up questions.
> 
> It seems like there was just one last minor question from him, and I think your answer makes perfect sense. 
> 
> I’m really happy with this proposal, thanks for working with us on it. I’m ready to vote now.
> 
> Thanks,
> John
> 
> On Sun, Jun 21, 2020, at 01:03, Ivan Ponomarev wrote:
>> Hello!
>>
>> Just to remind: I am ready to make further improvements for this KIP and 
>> the voting is also open.
>>
>> Regards,
>>
>> Ivan
>>
>> 12.06.2020 21:07, Ivan Ponomarev пишет:
>>> Hi Matthias, what do you think? Shall I update the KIP with another 
>>> naming scheme?
>>>
>>> Regards, Ivan
>>>
>>> 05.06.2020 13:57, Ivan Ponomarev пишет:
>>>>  > Can you maybe explain your reasoning? Why not just start with "0" and
>>>>  > the default branch gets the next index as any other branch
>>>>
>>>> My reasoning was the following: when we add a new branch, all the 
>>>> following branches will be renumbered. If the default branch gets the 
>>>> maximum index, its handler should be modified each time a new branch 
>>>> is added. If the default branch always has "0" index, we'll have to 
>>>> modify less code when adding a new branch to the end of the list.
>>>>
>>>>  > that's a minor question.
>>>>
>>>> I agree that it's a minor question, although maybe for a different 
>>>> reason :-)
>>>>
>>>> As a programmer, I don't believe in default branch naming, I'm not going
>>>> to use it myself and I'm going to discourage others from using it.
>>>>
>>>> The code relying on default naming scheme is going to be brittle. For 
>>>> example, if we want to reorder branches, we will also have to 
>>>> synchronously change the code that extracts branches from the Map for 
>>>> handling, with lots of opportunities to make a mistake.
>>>>
>>>> Low cohesion between branch predicates and branch handlers is one of 
>>>> the problems of current branching API that this KIP was intended to 
>>>> fight.
>>>>
>>>> And this is why my first goal was to avoid explicit contract for 
>>>> default branch naming in KIP/JavaDoc.
>>>>
>>>> Note that I agree that some form of default branch naming is needed. I 
>>>> can imagine cases when I'm going to use the resulting Map without 
>>>> explicit knowledge of its keys (e. g. operating on map.values() only).
>>>>
>>>> So if for some reason you feel that other indexing scheme is more 
>>>> preferable, I'm absolutely open for any proposals, because I don't 
>>>> think that it is an important matter :-)
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>> 05.06.2020 3:18, Matthias J. Sax пишет:
>>>>> Thanks for updating the KIP!
>>>>>
>>>>> I am overall happy with it. The proposed default branch numbering (ie,
>>>>> start with "1" and use "0" for the default branch) is not super
>>>>> intuitive, but that's a minor question.
>>>>>
>>>>> Can you maybe explain your reasoning? Why not just start with "0" and
>>>>> the default branch gets the next index as any other branch (ie, this is
>>>>> how the currently returned array indexes the branches, too)?
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 6/1/20 5:29 AM, Ivan Ponomarev wrote:
>>>>>> Hi all,
>>>>>>
>>>>>> I updated the KIP with what we have discussed, see 'How the resulting
>>>>>> Map is formed' section. Four bullet points to define the rules for Map
>>>>>> keys, and four bullet points to define the rules for Map values.
>>>>>>
>>>>>> I also added the overloads to `Branched` in order to accept 
>>>>>> Consumers as
>>>>>> parameters.
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>>
>>>>>> 01.06.2020 8:56, Matthias J. Sax пишет:
>>>>>>> For naming:
>>>>>>>
>>>>>>> (1) _If_ people specify a name for split() _and_ branch(), we _must_
>>>>>>> have a pulic API contract. Otherwise it would be very bad user
>>>>>>> experience if we are allowed to change the names (eg, do from "we 
>>>>>>> don't
>>>>>>> insert a `-` to we insert a `-`). -- We also need to clarify what
>>>>>>> happens is only split() _or_ branch() has a name specified, ie, is 
>>>>>>> there
>>>>>>> a contract for this case or not?
>>>>>>>
>>>>>>> (2) If we have a contract for how split() and branch() names are put
>>>>>>> together, we can also apply it to auto-generated names (this seems 
>>>>>>> to be
>>>>>>> just consistent).
>>>>>>>
>>>>>>> (3) Do we want to have a contract for auto-generated names? This is
>>>>>>> debatable, but I personally don't see any harm in having a contract.
>>>>>>> Also, if people want to access the Map, they can easily do it without
>>>>>>> specifying names if there is a contract -- if there is no contract, we
>>>>>>> force people to specify names to use the Map, what seems to make 
>>>>>>> the API
>>>>>>> harder to use.
>>>>>>>
>>>>>>> But we can of course discuss furhter on the dev list.
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 5/30/20 3:53 PM, Ivan Ponomarev wrote:
>>>>>>>> Hi Mathhias,
>>>>>>>>
>>>>>>>> I received your letter just after I sent mine concerning nulls and
>>>>>>>> reintroducing Consumer overload. So, if you're fine with omitting 
>>>>>>>> nulls
>>>>>>>> and introducing a with(Consumer) overload, it's great!
>>>>>>>>
>>>>>>>> Concerning the naming algorithm. Well, actually what you propose 
>>>>>>>> is very
>>>>>>>> clear and this is what I was going to implement anyway. But are 
>>>>>>>> you sure
>>>>>>>> that we will never want to change this algorithm? Here I trust your
>>>>>>>> judgement, since you understand the KStreams API's evolution much 
>>>>>>>> more
>>>>>>>> than I do.
>>>>>>>>
>>>>>>>>
>>>>>>>> 31.05.2020 0:50, Matthias J. Sax пишет:
>>>>>>>>> Ivan, John,
>>>>>>>>>
>>>>>>>>> thanks for the details.
>>>>>>>>>
>>>>>>>>> I guess I can be convinced in either direction with regard to `null`
>>>>>>>>> handling. But I am less worried about it, because I assume that a 
>>>>>>>>> Java
>>>>>>>>> programmer understand the ambiguty of `get(key) == null` -- 
>>>>>>>>> that's why
>>>>>>>>> there is `containsKey()`. In the end it might be a minor detail 
>>>>>>>>> and as
>>>>>>>>> long as it's well documented all discussed approaches seem fine.
>>>>>>>>> Including the idea to re-introduce the `Consumer` overload.
>>>>>>>>>
>>>>>>>>> It's Ivan's KIP so I am fine with whatever he picks :)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> About the naming: Not sure why we don't want to make the naming a
>>>>>>>>> public
>>>>>>>>> contract? I guess for split() we could default to the empty 
>>>>>>>>> String, and
>>>>>>>>> for `branch()` we could just add a counter by default?
>>>>>>>>>
>>>>>>>>> With no passed names at all, we just generte branche names "1", "2",
>>>>>>>>> "3", etc... With only`split(Named.as("foo-"))` (but not branch
>>>>>>>>> names) we
>>>>>>>>> generate "foo-1", "foo-2", etc.
>>>>>>>>>
>>>>>>>>> If any branch has a dedicated name, the counting is preserved but 
>>>>>>>>> just
>>>>>>>>> not used int he name for this brach:
>>>>>>>>>
>>>>>>>>> split("foo-")
>>>>>>>>>      .brach()
>>>>>>>>>      .brach(Branched.as("bar")
>>>>>>>>>      .defaultBranch()
>>>>>>>>>
>>>>>>>>> would generate "foo-1", "foo-bar", "foo-3"?
>>>>>>>>>
>>>>>>>>> Thoughts?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> On 5/29/20 1:20 PM, John Roesler wrote:
>>>>>>>>>> Hi all,
>>>>>>>>>>
>>>>>>>>>> Right, that was my suggestion; sorry for the confusion. I was 
>>>>>>>>>> thinking
>>>>>>>>>> that adding null as a value to maps is generally dubious, for which
>>>>>>>>>> reason a lot of Map implementations actually don't allow it at all.
>>>>>>>>>>
>>>>>>>>>> The reason is that it creates ambiguity, since a lot of code treats
>>>>>>>>>> "map.get(key) == null" as meaning the key is not in the map, but
>>>>>>>>>> if we place a null value in the map (presuming we don't immediately
>>>>>>>>>> get an exception), then you face weird contradictions, like
>>>>>>>>>> get(key) == null, but containsKey(key) can be either true or false,
>>>>>>>>>> and
>>>>>>>>>> you may or may not see it while iterating. Which would violate the
>>>>>>>>>> expectations of many Java programs. You can just take a look at
>>>>>>>>>> the AK codebase, and you'll find many occurrences where we
>>>>>>>>>> assume a null mapping means the map doesn't contain the key.
>>>>>>>>>>
>>>>>>>>>> Of course, the practical question, which Ivan brought up, is also
>>>>>>>>>> a good one. What would you actually do with these null values
>>>>>>>>>> in the map, besides get an NPE ?
>>>>>>>>>>
>>>>>>>>>> I'm not sure why I didn't think of this before, but an alternative
>>>>>>>>>> to this debate is to go back to adding Consumer into the API, but
>>>>>>>>>> with the same method name, withChain(Consumer<KStream<K,V>>).
>>>>>>>>>>
>>>>>>>>>> Now that we have only static methods, it's less confusing 
>>>>>>>>>> because it's
>>>>>>>>>> not possible to choose _both_ the Function<KStream,KStream> _and_
>>>>>>>>>> the Consumer<KStream> in the same branch (which was one of the
>>>>>>>>>> main drawbacks of the original plan wrt withChain and
>>>>>>>>>> withJavaConsumer as builder methods.
>>>>>>>>>>
>>>>>>>>>> I.e., I was previously concerned about:
>>>>>>>>>> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2) 
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> It's just a little confusing trying to reason about how this is all
>>>>>>>>>> going to wire up.
>>>>>>>>>>
>>>>>>>>>> But with only the static methods, users have only three methods,
>>>>>>>>>> and they are all clear:
>>>>>>>>>> 1. Branched.as(name): just names the branch, the branch-predicated
>>>>>>>>>> stream is the value of the map
>>>>>>>>>> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the
>>>>>>>>>> branch, applies the transformation after the branch predicate, and
>>>>>>>>>> the transformed stream is the value of the map (null is not 
>>>>>>>>>> permitted)
>>>>>>>>>> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch,
>>>>>>>>>> passes the predicated stream in to the Consumer function and omits
>>>>>>>>>> the branch from the map
>>>>>>>>>>
>>>>>>>>>> Sorry for digging this option up again, but it's starting to 
>>>>>>>>>> look more
>>>>>>>>>> attractive after we dropped the builder functions, and I think 
>>>>>>>>>> it also
>>>>>>>>>> resolves the "null" concern.
>>>>>>>>>>
>>>>>>>>>> Thoughts?
>>>>>>>>>> -John
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>>>>>>>>>>> Hi Matthias!
>>>>>>>>>>>
>>>>>>>>>>> Thanks for your reply!
>>>>>>>>>>>
>>>>>>>>>>> (1)
>>>>>>>>>>>
>>>>>>>>>>>     > Do you imply that there won't be a `name -> null` entry 
>>>>>>>>>>> in the
>>>>>>>>>>> `Map`
>>>>>>>>>>> for the branch? If yes, I am wondering why?
>>>>>>>>>>>
>>>>>>>>>>> Ah, I re-read the discussion and found out that omitting null 
>>>>>>>>>>> entries
>>>>>>>>>>> was John's idea, not yours :-)
>>>>>>>>>>>
>>>>>>>>>>> Anyway. As you remember, I was against the 'swallowing' of
>>>>>>>>>>> branches by
>>>>>>>>>>> Consumer. I had a feeling that there are cases when I would like
>>>>>>>>>>> both to
>>>>>>>>>>> consume a branch and post-process it in a Map handler.
>>>>>>>>>>>
>>>>>>>>>>> But then the idea of getting rid of `withJavaConsumer` came up. 
>>>>>>>>>>> And
>>>>>>>>>>> then
>>>>>>>>>>> we realized that we should handle nulls somehow.
>>>>>>>>>>>
>>>>>>>>>>> So it looked like taking the best from both approaches: if we 
>>>>>>>>>>> want to
>>>>>>>>>>> omit a branch in the resulting map, we just return null, thus
>>>>>>>>>>> emulating
>>>>>>>>>>> a 'swallowing consumer'. If we want to include a branch, we return
>>>>>>>>>>> it.
>>>>>>>>>>> It provides the full control for a user, and it spares the user 
>>>>>>>>>>> from
>>>>>>>>>>> null-checking in their code.
>>>>>>>>>>>
>>>>>>>>>>>     > My reasoning is, that it's very clear that `name -> null`
>>>>>>>>>>> would be
>>>>>>>>>>> there, because the user code did execute `return null`.
>>>>>>>>>>>
>>>>>>>>>>> My reasoning is following. The way such lambdas are written, it's
>>>>>>>>>>> hard
>>>>>>>>>>> to return null unintentionally. And if a user returns null
>>>>>>>>>>> intentionally, what `name->null` in the resulting Map can be good
>>>>>>>>>>> for?
>>>>>>>>>>> It's either NPE (which is clearly not intended) or something that
>>>>>>>>>>> should
>>>>>>>>>>> be null-checked and skipped. Ok, let's do it!
>>>>>>>>>>>
>>>>>>>>>>> But, I'm ready to be convinced to drop this rule. It doesn't seem
>>>>>>>>>>> to me
>>>>>>>>>>> to be a principal thing.
>>>>>>>>>>>
>>>>>>>>>>> (2)
>>>>>>>>>>>
>>>>>>>>>>>     > I think it would be good if the KIP could explains the 
>>>>>>>>>>> intended
>>>>>>>>>>> default naming schema. This is important because the naming schema
>>>>>>>>>>> must
>>>>>>>>>>> be part of the API contract; otherwise, users cannot rely on the
>>>>>>>>>>> naming
>>>>>>>>>>> when trying to use the returned `Map`.
>>>>>>>>>>>
>>>>>>>>>>> Well, in fact, I intentionally didn't include any naming schema in
>>>>>>>>>>> the
>>>>>>>>>>> KIP. And it's done exactly for the purpose that a user won't be
>>>>>>>>>>> able to
>>>>>>>>>>> rely on default naming. If they want to use certain branches 
>>>>>>>>>>> from the
>>>>>>>>>>> Map, they will devise their own naming strategy that will never
>>>>>>>>>>> change
>>>>>>>>>>> and will never let them down.
>>>>>>>>>>>
>>>>>>>>>>> If they don't mind what's in a Map, if they are looking up the Map
>>>>>>>>>>> just
>>>>>>>>>>> for debugging, or if they want to treat a Map just like a
>>>>>>>>>>> Collection of
>>>>>>>>>>> values (a possible use case!), it doesn't matter how the branches
>>>>>>>>>>> are named.
>>>>>>>>>>>
>>>>>>>>>>> This way we are giving ourselves the full freedom to change the
>>>>>>>>>>> naming
>>>>>>>>>>> schema afterwards without violating the API contract.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>
>>>>>>>>>>>> Using covariant generics is a good idea! I am also fine with only
>>>>>>>>>>>> using
>>>>>>>>>>>> static method in `Branched` for now, as it's only two 
>>>>>>>>>>>> parameters and
>>>>>>>>>>>> thus not too many overloads.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (1) What is unclear to me is, what you exaclty mean by:
>>>>>>>>>>>>
>>>>>>>>>>>>> If a function returns `null`, its result is omitted.
>>>>>>>>>>>>
>>>>>>>>>>>> Do you imply that there won't be a `name -> null` entry in the
>>>>>>>>>>>> `Map` for
>>>>>>>>>>>> the branch? If yes, I am wondering why?
>>>>>>>>>>>>
>>>>>>>>>>>> My previous argument was to omit an entry only for the
>>>>>>>>>>>> `withJavaConsumer()` case, because the return type is "void" (and
>>>>>>>>>>>> thus
>>>>>>>>>>>> it's unclear what should be added and it would introduce
>>>>>>>>>>>> inconsistencies
>>>>>>>>>>>> in the `Map`).
>>>>>>>>>>>>
>>>>>>>>>>>> However, for a `Function` that returns a `KStream`, we could
>>>>>>>>>>>> actually
>>>>>>>>>>>> just add a `name -> null` entry if `null` is returned. My 
>>>>>>>>>>>> reasoning
>>>>>>>>>>>> is,
>>>>>>>>>>>> that it's very clear that `name -> null` would be there, 
>>>>>>>>>>>> because the
>>>>>>>>>>>> user code did execute `return null`. Adding a `name -> null`
>>>>>>>>>>>> entry for
>>>>>>>>>>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>>>>>>>>>>> exceptional case for handling `null`).
>>>>>>>>>>>>
>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (2) For `Branched.as()` the KIP says:
>>>>>>>>>>>>
>>>>>>>>>>>>> sets the name of the branch (auto-generated by default, when 
>>>>>>>>>>>>> split
>>>>>>>>>>>>> operation is named, then the names are index-suffixed)
>>>>>>>>>>>>
>>>>>>>>>>>> I think it would be good if the KIP could explains the intended
>>>>>>>>>>>> default
>>>>>>>>>>>> naming schema. This is important because the naming schema 
>>>>>>>>>>>> must be
>>>>>>>>>>>> part
>>>>>>>>>>>> of the API contract; otherwise, users cannot rely on the 
>>>>>>>>>>>> naming when
>>>>>>>>>>>> trying to use the returned `Map`.
>>>>>>>>>>>>
>>>>>>>>>>>> Similalry, it should be explained how names are generated if
>>>>>>>>>>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations
>>>>>>>>>>>> how the
>>>>>>>>>>>> API can be mixed and matched and it's unclear atm how it would
>>>>>>>>>>>> work in
>>>>>>>>>>>> detail.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>>>>>>>>>>> that have been raised in the discussion so far have been
>>>>>>>>>>>>> addressed. And it's been a while since anyone raised a new
>>>>>>>>>>>>> concern. At this point, it seems like a good time to start
>>>>>>>>>>>>> the VOTE thread.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Sometimes, the vote thread will trigger new people to look
>>>>>>>>>>>>> into the KIP, and they may raise new concerns, but it's not a
>>>>>>>>>>>>> problem. We'll just address those lingering concerns if there
>>>>>>>>>>>>> are any, until you have all the votes you need.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks again for the contribution!
>>>>>>>>>>>>>
>>>>>>>>>>>>> -John
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>>>>>>>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1. Per Mathhias's suggestion, if a chain function returns null,
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> respective result if omitted in the resulting Map.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3. `Branched` class has only three static methods with all the
>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>> combinations of parameters.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 4. Chain function is defined 'fully covariant', let's see if we
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>> implement it this way :-))
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> + code example updates and minor edits.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Since this is my first KIP, I'm not sure what should I do 
>>>>>>>>>>>>>> next. I
>>>>>>>>>>>>>> feel
>>>>>>>>>>>>>> that we talked over all the details and the consensus is 
>>>>>>>>>>>>>> reached.
>>>>>>>>>>>>>> Is it
>>>>>>>>>>>>>> OK to call for VOTE now or is it better to wait for more 
>>>>>>>>>>>>>> feedback?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>>>>>>>>>>> Sounds good to me, Ivan!
>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ---------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>       > I'm sorry things have been dragging out a little, 
>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>> have the
>>>>>>>>>>>>>>>> sense we're very close to the end of this discussion, 
>>>>>>>>>>>>>>>> which is
>>>>>>>>>>>>>>>> exciting.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> We are certainly moving forward!  And I'm not in a hurry at
>>>>>>>>>>>>>>>> all. As I
>>>>>>>>>>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>>>>>>>>>>> KafkaStreamBrancher -- the implementation of the first,
>>>>>>>>>>>>>>>> rejected version
>>>>>>>>>>>>>>>> of this KIP. It's inferior to what we are discussing here, 
>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> it does
>>>>>>>>>>>>>>>> its work. So it's worth to design this KIP really, really 
>>>>>>>>>>>>>>>> well.
>>>>>>>>>>>>>>>> And by
>>>>>>>>>>>>>>>> the way, from this discussion I'm learning the good API
>>>>>>>>>>>>>>>> designing
>>>>>>>>>>>>>>>> process. For me it has a value per se :-))
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ----------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>       >>   > I'd wonder whether we need the non-static 
>>>>>>>>>>>>>>>> builders
>>>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>>>       >>   > Do they provide any benefit over just using the
>>>>>>>>>>>>>>>> right static
>>>>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>>>>       >
>>>>>>>>>>>>>>>>       > I don't have a strong feeling, either. It seems 
>>>>>>>>>>>>>>>> nice to
>>>>>>>>>>>>>>>> offer a better
>>>>>>>>>>>>>>>>       > type inference experience than what we get with
>>>>>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>>>>>       > offering the static method that takes both name and
>>>>>>>>>>>>>>>> chain.
>>>>>>>>>>>>>>>>       > Given that, there doesn't seem to be a good reason to
>>>>>>>>>>>>>>>> also offer the
>>>>>>>>>>>>>>>>       > non-static builder-style methods, so I guess I'd 
>>>>>>>>>>>>>>>> prefer
>>>>>>>>>>>>>>>> to drop them.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>>>>>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) 
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> found out
>>>>>>>>>>>>>>>> an interesting fact about type inference rules in Java. Funny
>>>>>>>>>>>>>>>> thing is
>>>>>>>>>>>>>>>> that although we need to explicitly set types in a chain like
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> foo.branch(..., 
>>>>>>>>>>>>>>>> Branched.<...,...>named("foo").withChain(...));
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> (otherwise it won't compile), the composition of static 
>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>> works just fine, all the types are being calculated 
>>>>>>>>>>>>>>>> correctly:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), 
>>>>>>>>>>>>>>>> ...));
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> As I was told, for type inference there is difference between
>>>>>>>>>>>>>>>> qualifiers
>>>>>>>>>>>>>>>> and arguments, 'you go up if you are argument but stop if you
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> qualifier'. And it also seems that we should not bet on any
>>>>>>>>>>>>>>>> future
>>>>>>>>>>>>>>>> improvements in Java type inference here.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1) I think we that in this KIP we should provide three static
>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function,
>>>>>>>>>>>>>>>> String)`, and
>>>>>>>>>>>>>>>> drop any non-static ones.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2) If anything else will be ever needed, we can easily add
>>>>>>>>>>>>>>>> anything.
>>>>>>>>>>>>>>>> Maybe this can be done in a process of refinement of all the
>>>>>>>>>>>>>>>> parameter
>>>>>>>>>>>>>>>> classes.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>       > we may as well hope for the best, and propose the 
>>>>>>>>>>>>>>>> "fully
>>>>>>>>>>>>>>>>       > covariant" definition for now.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>       >> Good question, I already thought about it and 
>>>>>>>>>>>>>>>> rejected
>>>>>>>>>>>>>>>> the idea....
>>>>>>>>>>>>>>>>       > (I cut off your quote; the rest is in the chain 
>>>>>>>>>>>>>>>> below)
>>>>>>>>>>>>>>>>       >  "Worst" case scenario: someone
>>>>>>>>>>>>>>>>       > else will wish the return type is something 
>>>>>>>>>>>>>>>> different,
>>>>>>>>>>>>>>>> and we'll go
>>>>>>>>>>>>>>>>       > through a painless deprecation transition to 
>>>>>>>>>>>>>>>> change it
>>>>>>>>>>>>>>>> later.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Of course, we cant' predict all the ways people are going to
>>>>>>>>>>>>>>>> use it.
>>>>>>>>>>>>>>>>       From my own humble experience with Kafka Streams, 
>>>>>>>>>>>>>>>> the worst
>>>>>>>>>>>>>>>> scenario is
>>>>>>>>>>>>>>>> unlikely. Split is split, transform is transform, too much
>>>>>>>>>>>>>>>> flexibility
>>>>>>>>>>>>>>>> is often evil.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> So it seems that we are close to the consensus. Two things to
>>>>>>>>>>>>>>>> be altered
>>>>>>>>>>>>>>>> in the current version of KIP:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>>>>>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Any ideas / questions / objections?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>>>>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'm sorry things have been dragging out a little, but I have
>>>>>>>>>>>>>>>>> the sense
>>>>>>>>>>>>>>>>> we're very close to the end of this discussion, which is
>>>>>>>>>>>>>>>>> exciting.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>>>>>        > Do they provide any benefit over just using the 
>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>> static factory?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I don't have a strong feeling, either. It seems nice to 
>>>>>>>>>>>>>>>>> offer
>>>>>>>>>>>>>>>>> a better
>>>>>>>>>>>>>>>>> type inference experience than what we get with
>>>>>>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Given that, there doesn't seem to be a good reason to also
>>>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>>>> non-static builder-style methods, so I guess I'd prefer to
>>>>>>>>>>>>>>>>> drop them.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'll defer to Matthias, if he has a chance to consider 
>>>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>> it's more
>>>>>>>>>>>>>>>>> valuable to stick with the existing pattern or break the
>>>>>>>>>>>>>>>>> pattern to offer
>>>>>>>>>>>>>>>>> a better experience.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in 
>>>>>>>>>>>>>>>>>> the KIP,
>>>>>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>>>>>>        > covariant typed function
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
>>>>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in 
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been 
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> we just
>>>>>>>>>>>>>>>>> won't know what works until we really try it, and try it in
>>>>>>>>>>>>>>>>> several different
>>>>>>>>>>>>>>>>> ways. Still, we may as well hope for the best, and 
>>>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>>>> "fully
>>>>>>>>>>>>>>>>> covariant" definition for now.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
>>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>>>>>>        > same as the inputs?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>>>>>> idea....
>>>>>>>>>>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> That's fair! It's your KIP, after all. I think I might have
>>>>>>>>>>>>>>>>> made a different
>>>>>>>>>>>>>>>>> call here, but I think this choice is fine. "Worst" case
>>>>>>>>>>>>>>>>> scenario: someone
>>>>>>>>>>>>>>>>> else will wish the return type is something different, and
>>>>>>>>>>>>>>>>> we'll go
>>>>>>>>>>>>>>>>> through a painless deprecation transition to change it 
>>>>>>>>>>>>>>>>> later.
>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>> to the clean design of your API, this doesn't seem to bad.
>>>>>>>>>>>>>>>>> And, of
>>>>>>>>>>>>>>>>> course, you've actually been using similar functionality
>>>>>>>>>>>>>>>>> already, so it
>>>>>>>>>>>>>>>>> seems we should trust your intuition.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>>>>> Do they provide any benefit over just using the right 
>>>>>>>>>>>>>>>>>> static
>>>>>>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in 
>>>>>>>>>>>>>>>>>> the KIP,
>>>>>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>>>>>> covariant typed function
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
>>>>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in 
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
>>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>>>>>> same as the inputs?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Look, if we want to keep `withChain`'s function optional,
>>>>>>>>>>>>>>>>>> then we must
>>>>>>>>>>>>>>>>>> keep the result key and value types the same. Because for
>>>>>>>>>>>>>>>>>> now, the
>>>>>>>>>>>>>>>>>> default value for the 'chain function' is 
>>>>>>>>>>>>>>>>>> Function.identity().
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Of course, we can make the 'chain function' required. But I
>>>>>>>>>>>>>>>>>> think this
>>>>>>>>>>>>>>>>>> is not what `split` method is for. `split` is for
>>>>>>>>>>>>>>>>>> splitting, not
>>>>>>>>>>>>>>>>>> transforming, and `chainFunction` in most of the cases 
>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> be either
>>>>>>>>>>>>>>>>>> a consumer or the identity function.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past
>>>>>>>>>>>>>>>>>>> that generic type inference problem is to offer a static
>>>>>>>>>>>>>>>>>>> factory method that takes all the options (both name and
>>>>>>>>>>>>>>>>>>> function).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I’m happy with adding that option, although if we have all
>>>>>>>>>>>>>>>>>>> possible combinations available as static methods, then 
>>>>>>>>>>>>>>>>>>> I’d
>>>>>>>>>>>>>>>>>>> wonder whether we need the non-static builders (like
>>>>>>>>>>>>>>>>>>> withChain). Do they provide any benefit over just using 
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> right static factory?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function 
>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>> bounds works algebraically, but I’m not sure whether the
>>>>>>>>>>>>>>>>>>> limitations of Java type inference will actually let you
>>>>>>>>>>>>>>>>>>> pass in all the different functions you would want to.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I recall Bruno and I wrestling with this when he added
>>>>>>>>>>>>>>>>>>> Transform operators. In retrospect, this is a problem 
>>>>>>>>>>>>>>>>>>> we can
>>>>>>>>>>>>>>>>>>> iron out when we write tests in the PR. You might as well
>>>>>>>>>>>>>>>>>>> propose the “ideal” API in the KIP, which is the covariant
>>>>>>>>>>>>>>>>>>> typed function:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>>>>>>> KStream<? extends K, ? extends V>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 3. Actually, I have a new question about the types: Is it
>>>>>>>>>>>>>>>>>>> necessary to restrict the result key and value types to be
>>>>>>>>>>>>>>>>>>> the same as the inputs? I.e., shouldn’t the result 
>>>>>>>>>>>>>>>>>>> types be
>>>>>>>>>>>>>>>>>>> K1,V1?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>> Hello John,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>         > Perhaps it would be better to stick with "as"
>>>>>>>>>>>>>>>>>>>> for now
>>>>>>>>>>>>>>>>>>>>         > and just file a Jira to switch them all at 
>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>> time [for
>>>>>>>>>>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin
>>>>>>>>>>>>>>>>>>>> they have a
>>>>>>>>>>>>>>>>>>>> standard workaround
>>>>>>>>>>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin). 
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> So actually this should be a very low priority issue, 
>>>>>>>>>>>>>>>>>>>> if an
>>>>>>>>>>>>>>>>>>>> issue at
>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>         > I don't understand how your new proposed
>>>>>>>>>>>>>>>>>>>>         > methods would work any differently than the 
>>>>>>>>>>>>>>>>>>>> ones
>>>>>>>>>>>>>>>>>>>> you already
>>>>>>>>>>>>>>>>>>>>         > had proposed in the KIP. It seems like you'd 
>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>> have to provide
>>>>>>>>>>>>>>>>>>>>         > the generic type parameters on the first static
>>>>>>>>>>>>>>>>>>>> factory call. Can you
>>>>>>>>>>>>>>>>>>>>         > explain how your new interface proposal differs
>>>>>>>>>>>>>>>>>>>> from the existing KIP?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> In the KIP, I didn't clarify what methods should be 
>>>>>>>>>>>>>>>>>>>> static.
>>>>>>>>>>>>>>>>>>>> Now I
>>>>>>>>>>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> static: as(String), with(Function), with(Function, 
>>>>>>>>>>>>>>>>>>>> String).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The overloaded `with` version that provides both Function
>>>>>>>>>>>>>>>>>>>> and name can
>>>>>>>>>>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>         > Regarding making the K,V types covariant 
>>>>>>>>>>>>>>>>>>>> also, yes,
>>>>>>>>>>>>>>>>>>>> that would indeed
>>>>>>>>>>>>>>>>>>>>         > be nice, but I'm not sure it will actually 
>>>>>>>>>>>>>>>>>>>> work.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> What I'm keeping in mind is the following example: 
>>>>>>>>>>>>>>>>>>>> imagine
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> static KStream<String, Integer> func(KStream<String,
>>>>>>>>>>>>>>>>>>>> Number> s) {
>>>>>>>>>>>>>>>>>>>>                 return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>>>>>>>>>>             s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>>>>>>>>>>                        //Won't compile!!
>>>>>>>>>>>>>>>>>>>>                        Branched.with(Me::func));
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The simple workaround here is to change `func`'s return
>>>>>>>>>>>>>>>>>>>> type from
>>>>>>>>>>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [On the other hand, we already agreed to remove
>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` from
>>>>>>>>>>>>>>>>>>>> `Branched`, so during code migration I will have to 
>>>>>>>>>>>>>>>>>>>> modify
>>>>>>>>>>>>>>>>>>>> my functions'
>>>>>>>>>>>>>>>>>>>> return types anyway -- I mean, from `void` to 
>>>>>>>>>>>>>>>>>>>> `KStream`!! ]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>         >  the map you're returning is Map<K,V>, and of
>>>>>>>>>>>>>>>>>>>> course a K is not the
>>>>>>>>>>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I think what you actually meant here is that KStream<?
>>>>>>>>>>>>>>>>>>>> extends K, ?
>>>>>>>>>>>>>>>>>>>> extends V> is not fit as a value for Map<String, 
>>>>>>>>>>>>>>>>>>>> KStream<K,
>>>>>>>>>>>>>>>>>>>> V>>. This
>>>>>>>>>>>>>>>>>>>> particularly is not a problem, since KStream<? extends 
>>>>>>>>>>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>>>>> extends V>
>>>>>>>>>>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be 
>>>>>>>>>>>>>>>>>>>> put
>>>>>>>>>>>>>>>>>>>> to the map.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard
>>>>>>>>>>>>>>>>>>>> types. So maybe
>>>>>>>>>>>>>>>>>>>> for now it's better to just admit that API is not
>>>>>>>>>>>>>>>>>>>> absolutely perfect and
>>>>>>>>>>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that
>>>>>>>>>>>>>>>>>>>>> "as" would
>>>>>>>>>>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart 
>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>> convention
>>>>>>>>>>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The convention is that "as(String name)" is used for the
>>>>>>>>>>>>>>>>>>>>> static factory
>>>>>>>>>>>>>>>>>>>>> method, whereas "withName(String name)" is an instance
>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to
>>>>>>>>>>>>>>>>>>>>> avoid "as"
>>>>>>>>>>>>>>>>>>>>> for compatibility with Kotlin, I might suggest
>>>>>>>>>>>>>>>>>>>>> "fromName(String name)",
>>>>>>>>>>>>>>>>>>>>> although it's somewhat dubious, since all the other
>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick 
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> "as" for now
>>>>>>>>>>>>>>>>>>>>> and just file a Jira to switch them all at the same 
>>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>>>>>>>>>>> Regarding the type inference problem, yes, it's a 
>>>>>>>>>>>>>>>>>>>>> blemish
>>>>>>>>>>>>>>>>>>>>> on all of our
>>>>>>>>>>>>>>>>>>>>> configuraion objects. The problem is that Java infers 
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>> based on the _first_ method in the chain. While it does
>>>>>>>>>>>>>>>>>>>>> consider what
>>>>>>>>>>>>>>>>>>>>> the recipient of the method result wants, it only
>>>>>>>>>>>>>>>>>>>>> considers the _next_
>>>>>>>>>>>>>>>>>>>>> recipient.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it 
>>>>>>>>>>>>>>>>>>>>> to a
>>>>>>>>>>>>>>>>>>>>> Branched<String,String> variable, java infers the type
>>>>>>>>>>>>>>>>>>>>> correctly. But
>>>>>>>>>>>>>>>>>>>>> when the "next recipient" is a chained method call, like
>>>>>>>>>>>>>>>>>>>>> "withChain",
>>>>>>>>>>>>>>>>>>>>> then the chained method doesn't bound the type (by
>>>>>>>>>>>>>>>>>>>>> definition,
>>>>>>>>>>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so 
>>>>>>>>>>>>>>>>>>>>> Java
>>>>>>>>>>>>>>>>>>>>> will take
>>>>>>>>>>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be
>>>>>>>>>>>>>>>>>>>>> revised anymore.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, 
>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>> doesn't seem
>>>>>>>>>>>>>>>>>>>>> that hard to recursively consider the entire context 
>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> inferring the
>>>>>>>>>>>>>>>>>>>>> generic type parameters, but this is what we have to 
>>>>>>>>>>>>>>>>>>>>> work
>>>>>>>>>>>>>>>>>>>>> with.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> To be honest, though, I don't understand how your new
>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>> methods would work any differently than the ones you
>>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still 
>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>> the generic type parameters on the first static factory
>>>>>>>>>>>>>>>>>>>>> call. Can you
>>>>>>>>>>>>>>>>>>>>> explain how your new interface proposal differs from the
>>>>>>>>>>>>>>>>>>>>> existing KIP?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that
>>>>>>>>>>>>>>>>>>>>> would indeed
>>>>>>>>>>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You 
>>>>>>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>>>> want to give it a
>>>>>>>>>>>>>>>>>>>>> try. In the past, we've run into soe truly strange
>>>>>>>>>>>>>>>>>>>>> interactions between the
>>>>>>>>>>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner
>>>>>>>>>>>>>>>>>>>>> classes) in
>>>>>>>>>>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Another issue is that the value type of the map you're
>>>>>>>>>>>>>>>>>>>>> returning is
>>>>>>>>>>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? 
>>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>> K", so it
>>>>>>>>>>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that
>>>>>>>>>>>>>>>>>>>>>> getting rid of
>>>>>>>>>>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as
>>>>>>>>>>>>>>>>>>>>>> possible, but not
>>>>>>>>>>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to
>>>>>>>>>>>>>>>>>>>>>> implement examples
>>>>>>>>>>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1. Having to return something from lambda is not a very
>>>>>>>>>>>>>>>>>>>>>> big deal.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use
>>>>>>>>>>>>>>>>>>>>>> method references
>>>>>>>>>>>>>>>>>>>>>> for already written stream consumers, but then I 
>>>>>>>>>>>>>>>>>>>>>> realized
>>>>>>>>>>>>>>>>>>>>>> that I can
>>>>>>>>>>>>>>>>>>>>>> just change my methods from returning void to returning
>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>> parameter and use references to them. Not very
>>>>>>>>>>>>>>>>>>>>>> convenient, but passable.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no
>>>>>>>>>>>>>>>>>>>>>> consumer 2) when
>>>>>>>>>>>>>>>>>>>>>> function returns null, we don't insert it into the
>>>>>>>>>>>>>>>>>>>>>> resulting map.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but
>>>>>>>>>>>>>>>>>>>>>> workable solution as
>>>>>>>>>>>>>>>>>>>>>> a first approximation. And later we can always add to
>>>>>>>>>>>>>>>>>>>>>> `Branched`
>>>>>>>>>>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes
>>>>>>>>>>>>>>>>>>>>>> like Branched
>>>>>>>>>>>>>>>>>>>>>> should be built? First of all, it seems that `as` 
>>>>>>>>>>>>>>>>>>>>>> now is
>>>>>>>>>>>>>>>>>>>>>> more preferred
>>>>>>>>>>>>>>>>>>>>>> than `withName` (although as you probably know it 
>>>>>>>>>>>>>>>>>>>>>> clashes
>>>>>>>>>>>>>>>>>>>>>> with Kotlin's
>>>>>>>>>>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out 
>>>>>>>>>>>>>>>>>>>>>> that my
>>>>>>>>>>>>>>>>>>>>>> Java cannot
>>>>>>>>>>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>>>>>             Branched.as("foo").withChain(s ->
>>>>>>>>>>>>>>>>>>>>>> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>>>>>             Branched.<String, 
>>>>>>>>>>>>>>>>>>>>>> String>as("foo").withChain(s
>>>>>>>>>>>>>>>>>>>>>> -> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of
>>>>>>>>>>>>>>>>>>>>>> `Branched` methods
>>>>>>>>>>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, 
>>>>>>>>>>>>>>>>>>>>>> V>,
>>>>>>>>>>>>>>>>>>>>>> ? extends
>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 4. And one more. What do you think, do we need that
>>>>>>>>>>>>>>>>>>>>>> flexibility:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, 
>>>>>>>>>>>>>>>>>>>>>> V>>
>>>>>>>>>>>>>>>>>>>>>> chain
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? 
>>>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>>> KStream<?
>>>>>>>>>>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ??
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be 
>>>>>>>>>>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>>>>>>>>>>> map, because there’s only one thing that could be 
>>>>>>>>>>>>>>>>>>>>>>> there,
>>>>>>>>>>>>>>>>>>>>>>> which is whatever is returned from the chain function.
>>>>>>>>>>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the 
>>>>>>>>>>>>>>>>>>>>>>> extensible way
>>>>>>>>>>>>>>>>>>>>>>> this KIP is designed, it would be trivially easy to 
>>>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>> the “terminal” chain later. As you say, fewer concepts
>>>>>>>>>>>>>>>>>>>>>>> leads to an API that is easier to learn.
>>>>>>>>>>>>>>>>>>>>>>> 3. We get to side-step the naming of this method.
>>>>>>>>>>>>>>>>>>>>>>> Although I didn’t complain about withJavaConsumer, it
>>>>>>>>>>>>>>>>>>>>>>> was only because I couldn’t think of a better name.
>>>>>>>>>>>>>>>>>>>>>>> Still, it’s somewhat unsatisfying to name a method 
>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>> its argument type, since this provides no 
>>>>>>>>>>>>>>>>>>>>>>> information at
>>>>>>>>>>>>>>>>>>>>>>> all about what the method does. I was willing to 
>>>>>>>>>>>>>>>>>>>>>>> accept
>>>>>>>>>>>>>>>>>>>>>>> it because I didn’t have an alternative, but I 
>>>>>>>>>>>>>>>>>>>>>>> would be
>>>>>>>>>>>>>>>>>>>>>>> happy to skip this method for now to avoid the problem
>>>>>>>>>>>>>>>>>>>>>>> until we have more inspiration.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> The only con I see is that it makes the code a little
>>>>>>>>>>>>>>>>>>>>>>> less ergonomic to write when you don’t want to return
>>>>>>>>>>>>>>>>>>>>>>> the result of the chain (such as when the chain is
>>>>>>>>>>>>>>>>>>>>>>> terminal), since I’m your example, you have to 
>>>>>>>>>>>>>>>>>>>>>>> declare a
>>>>>>>>>>>>>>>>>>>>>>> block with a return statement at the end. It’s not
>>>>>>>>>>>>>>>>>>>>>>> ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with 
>>>>>>>>>>>>>>>>>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>> a null result, which would just remove the branch from
>>>>>>>>>>>>>>>>>>>>>>> the returned map. It seems nicer than forcing 
>>>>>>>>>>>>>>>>>>>>>>> people to
>>>>>>>>>>>>>>>>>>>>>>> pick a stream to return when their chain is 
>>>>>>>>>>>>>>>>>>>>>>> terminal and
>>>>>>>>>>>>>>>>>>>>>>> they don’t want to use the result later.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I guess the only open question is about
>>>>>>>>>>>>>>>>>>>>>>>> `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>           (1) split a stream and return the 
>>>>>>>>>>>>>>>>>>>>>>>> substreams
>>>>>>>>>>>>>>>>>>>>>>>> for futher processing
>>>>>>>>>>>>>>>>>>>>>>>>           (2) split a stream and modify the 
>>>>>>>>>>>>>>>>>>>>>>>> substreams
>>>>>>>>>>>>>>>>>>>>>>>> with in-place method chaining
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>            -> split a stream, modify the 
>>>>>>>>>>>>>>>>>>>>>>>> substreams, and
>>>>>>>>>>>>>>>>>>>>>>>> return the _modified_
>>>>>>>>>>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>>>>>> split the topology graph at any point. Technically,
>>>>>>>>>>>>>>>>>>>>>>>>> it's OK to do both: feed the KStream to a
>>>>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>>>>>>> doesn't need the stream in the Map, one simply does
>>>>>>>>>>>>>>>>>>>>>>>>> not extract it from there
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> That is of course possible. However, it introduces 
>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>> "hidded" semantics:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>           - using `withChain` I get the modified
>>>>>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>>>>>           - using `withJavaConsumer` I get the 
>>>>>>>>>>>>>>>>>>>>>>>> unmodifed
>>>>>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>          From my understanding the original idea of
>>>>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` was to
>>>>>>>>>>>>>>>>>>>>>>>> model a terminal operation, ie, it should be 
>>>>>>>>>>>>>>>>>>>>>>>> similar to:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>>>>>            return null;
>>>>>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> However, I am not sure if we should even allow
>>>>>>>>>>>>>>>>>>>>>>>> `withChain()` to return
>>>>>>>>>>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this
>>>>>>>>>>>>>>>>>>>>>>>> case to avoid a `key
>>>>>>>>>>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Following this train of through, and if we want to
>>>>>>>>>>>>>>>>>>>>>>>> allow the "return
>>>>>>>>>>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>>>>>> that does not add
>>>>>>>>>>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Following your proposal, the semantics of
>>>>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` could also
>>>>>>>>>>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>>>>>            return s;
>>>>>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely
>>>>>>>>>>>>>>>>>>>>>>>> syntactic sugar,
>>>>>>>>>>>>>>>>>>>>>>>> while for the first proposal it adds new 
>>>>>>>>>>>>>>>>>>>>>>>> functionality
>>>>>>>>>>>>>>>>>>>>>>>> (if `return null`
>>>>>>>>>>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not 
>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>> "hide a
>>>>>>>>>>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need
>>>>>>>>>>>>>>>>>>>>>>>> to allow `return
>>>>>>>>>>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a
>>>>>>>>>>>>>>>>>>>>>>>> sub-stream in the Map.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we
>>>>>>>>>>>>>>>>>>>>>>>> follow your
>>>>>>>>>>>>>>>>>>>>>>>> proposal, I am wondering if we need 
>>>>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>>>>>> at all? Its
>>>>>>>>>>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API
>>>>>>>>>>>>>>>>>>>>>>>> is usually
>>>>>>>>>>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox,
>>>>>>>>>>>>>>>>>>>>>>>>> fortunately we have
>>>>>>>>>>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) 
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> existing ‘branch’
>>>>>>>>>>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration
>>>>>>>>>>>>>>>>>>>>>>>>> Plan" section.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to
>>>>>>>>>>>>>>>>>>>>>>>>>> branching directly
>>>>>>>>>>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic
>>>>>>>>>>>>>>>>>>>>>>>>> branching]
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic
>>>>>>>>>>>>>>>>>>>>>>>>> Branching' section.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be 
>>>>>>>>>>>>>>>>>>>>>>>>> confusing. I
>>>>>>>>>>>>>>>>>>>>>>>>> renamed this
>>>>>>>>>>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' 
>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use 
>>>>>>>>>>>>>>>>>>>>>>>>>> cases:
>>>>>>>>>>>>>>>>>>>>>>>>>> EITHER using
>>>>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`]
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' 
>>>>>>>>>>>>>>>>>>>>>>>>>> version of
>>>>>>>>>>>>>>>>>>>>>>>>>> the `branch`
>>>>>>>>>>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that
>>>>>>>>>>>>>>>>>>>>>>>>>> using both
>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of
>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()` will
>>>>>>>>>>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the
>>>>>>>>>>>>>>>>>>>>>>>>>> returned `Map` of
>>>>>>>>>>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a
>>>>>>>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>>>>>>>> implementation would be to not add the "branch" 
>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>> result map if
>>>>>>>>>>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>>>>>> split the topology
>>>>>>>>>>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both:
>>>>>>>>>>>>>>>>>>>>>>>>> feed the KStream to
>>>>>>>>>>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If 
>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>> doesn't need the
>>>>>>>>>>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it 
>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>> there :-)
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the
>>>>>>>>>>>>>>>>>>>>>>>>> returned map
>>>>>>>>>>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs
>>>>>>>>>>>>>>>>>>>>>>>>> explicitly set by the
>>>>>>>>>>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids.
>>>>>>>>>>>>>>>>>>>>>>>>> Dealing with this
>>>>>>>>>>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing
>>>>>>>>>>>>>>>>>>>>>>>>> exclusions to this
>>>>>>>>>>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by
>>>>>>>>>>>>>>>>>>>>>>>>> provided
>>>>>>>>>>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated.
>>>>>>>>>>>>>>>>>>>>>>>>> What if a user
>>>>>>>>>>>>>>>>>>>>>>>>> provides both the name of the branch and a
>>>>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer? What do they
>>>>>>>>>>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or
>>>>>>>>>>>>>>>>>>>>>>>>> save it to the map?
>>>>>>>>>>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this 
>>>>>>>>>>>>>>>>>>>>>>>>> map, so
>>>>>>>>>>>>>>>>>>>>>>>>> maybe just leave
>>>>>>>>>>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg,
>>>>>>>>>>>>>>>>>>>>>>>>>> CoGroupedKStream,
>>>>>>>>>>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the 
>>>>>>>>>>>>>>>>>>>>>>>>>> "legacy"
>>>>>>>>>>>>>>>>>>>>>>>>>> `KGroupedStream`
>>>>>>>>>>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename 
>>>>>>>>>>>>>>>>>>>>>>>>>> without a
>>>>>>>>>>>>>>>>>>>>>>>>>> breaking change...
>>>>>>>>>>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have
>>>>>>>>>>>>>>>>>>>>>>>>>>> overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a 
>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate`
>>>>>>>>>>>>>>>>>>>>>>>>>> and assume you
>>>>>>>>>>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe
>>>>>>>>>>>>>>>>>>>>>>>>>> rephrase it accordingly
>>>>>>>>>>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public
>>>>>>>>>>>>>>>>>>>>>>>>>> Interface" section) of
>>>>>>>>>>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces) 
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction
>>>>>>>>>>>>>>>>>>>>>>>>>> of `withConsumer()`
>>>>>>>>>>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. 
>>>>>>>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>>>> related to John's
>>>>>>>>>>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use
>>>>>>>>>>>>>>>>>>>>>>>>>>> cases: EITHER using
>>>>>>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and
>>>>>>>>>>>>>>>>>>>>>>>>>> `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the
>>>>>>>>>>>>>>>>>>>>>>>>>> result of `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is 
>>>>>>>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>> with the `Consumer`
>>>>>>>>>>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and
>>>>>>>>>>>>>>>>>>>>>>>>>> `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to 
>>>>>>>>>>>>>>>>>>>>>>>>>> not add
>>>>>>>>>>>>>>>>>>>>>>>>>> the "branch" to
>>>>>>>>>>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As 
>>>>>>>>>>>>>>>>>>>>>>>>>> long as
>>>>>>>>>>>>>>>>>>>>>>>>>> we clearly document
>>>>>>>>>>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’
>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing; I thought you
>>>>>>>>>>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface
>>>>>>>>>>>>>>>>>>>>>>>>>>> (which doesn’t make
>>>>>>>>>>>>>>>>>>>>>>>>>>> sense, of course). I get that you were 
>>>>>>>>>>>>>>>>>>>>>>>>>>> referring to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the java Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>> interface, but we should still probably to to 
>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>> the ambiguity.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about 
>>>>>>>>>>>>>>>>>>>>>>>>>>> ‘withSink’?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be
>>>>>>>>>>>>>>>>>>>>>>>>>> confused with a "sink
>>>>>>>>>>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less 
>>>>>>>>>>>>>>>>>>>>>>>>>> ambiguous?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I
>>>>>>>>>>>>>>>>>>>>>>>>>>> think it’s close,
>>>>>>>>>>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev 
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> will someone please take a look at the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> reworked KIP?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that now it follows design 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> principles and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes into account
>>>>>>>>>>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> principles" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have completely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This version includes all the previous 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> results and follows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to provide one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we may use a single
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch, no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is expected here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rule'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have you back!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> already have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable to me to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> principles" that we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> participation in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we invested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not feel entitled to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalized.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original idea for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I gave up pushing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem with the scope of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .branch(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution in his post
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names seems to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your code brittle.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purposes. Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The idea seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trust
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> people who are more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status".
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about my proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worlds?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `String`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to merge both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your example.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added/removed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allow a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default name but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explicit to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that neither
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> independently of all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> taught us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that, although
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Branch, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> guess
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split(Named.withName("mysplit"))
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                           .branch(..., 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ...,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> syntax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operator(function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               operator(function, function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> string)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                interface BranchConfig 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noisier,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but it also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> decide we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tells us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domain,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sink
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Ivan, I’ll definitely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch(predicate, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumer)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 IMO the one trade 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> off to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 question. I don’t 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the same scope” 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 seems like a perfectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 when the branched 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the reasons Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution - working 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > On May 2, 2019, at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7:00
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Hello everyone, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thank you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Well, I don't 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 LinkedHashMap (no 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 matters) or `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 advantages than 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > In my opinion, the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal is that all 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 But 1) we rarely 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 workaround for the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 'Inlining the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method references 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 tend to split the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > The drawbacks are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> strong.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers is lost. We 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers in another. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misspell a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > What Michael propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 writing the API in Lua,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "dynamic naming" 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 beautiful. But in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 identifiers to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Do we have to invent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > And if we do, what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 all the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Earlier in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without "start 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the case when we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current KIP is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'clumsier'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 me address both 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 1) "Start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for it indeed) is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 2) No, dynamic 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Imagine a real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 value (say, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordType).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*John:if we had 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 have been much 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > KBranchedStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*Not clumsy at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > for (RecordType
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >             
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 02.05.2019 14:40,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> I also agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> current `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> However, I also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> was more aligned 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `branch()` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statement and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> It makes the code 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `Predicates` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (that is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        ��        >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> An open question 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> specified. Atm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> and the call to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> (what is not the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> because users can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> in the result 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> About "inlining": So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> preference. I can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> that clearly make 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> On 5/1/19 6:26 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Perhaps inlining 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that a lambda with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the full
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 it can be a method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The advantage of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 (Consumer) together in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Ultimately the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch trees will be 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> On May 1, 2019, at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> I'm less 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enthusiastic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> functionality.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> harder to read 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    >>>>> Also +1 on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> great 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> framework for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Regarding the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> proposal in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(Consumer<KStream<K, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Obviously some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't work 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> associativity 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> directly 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> provides over 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> The KIP 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that it is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> certainly 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> case anyway, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> access your 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch, and it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solution on 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> top of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> I could also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> taken in, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Pros for that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - accessing 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - no double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - downstream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to read top to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - you can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> (KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Overall I'm 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> curious
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branched 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't need 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user.  I'm 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sort of in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <sophie@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I'd like to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +1 what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> method, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> trying to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> alleviate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> able to cleanly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name->stream), which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> That said, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> anti-pattern or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> On Tue, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I’d like to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> are three 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 1. If you 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2. The way 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the ordering 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 3. It is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> brittle to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Using 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be a stronger
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> super K,? super
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Branches are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> given
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> mapping of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> because it’s a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> This solves 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2 because 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> interested 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> conditional by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> messing with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> One of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> awkward in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> On Tue, Apr 30,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <john@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> FWIW, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> confusing when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like a good 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> operator at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Tentatively, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> way, we don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> should return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> enforce 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> default 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> additionally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> though with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> On Fri, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 26,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> this is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The intend 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> IMHO, this 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> take any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> `branch()` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Because I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> might be 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> overlap that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Maybe 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> naming 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 'default' 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a method with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Bummer. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Can you 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> methods? 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP. For 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> You did not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -> KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branched-KStreams. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> nice to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> would need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> should 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "incomplete" to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched-KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 4/13/19 2:13
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I have 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Matthias, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I can see 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Totally 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> reserved 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> word,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> is not 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Absolutely! I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Dear 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> please 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> revise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Any new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 11.04.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> agrees 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> I had a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> quick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> are some 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> It's just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> In the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Also, we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Ie, we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> For the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> could we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> second 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> `Function`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Finally, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On 3/31/19
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> I'm a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bit of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> revise 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> buy-in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> could be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> idea 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> forward 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> deprecate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> same 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> just 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> little 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sloppy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> passing 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream but read from all 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On Fri, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I read 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> looks 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> going 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> What 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shall we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Why 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we go 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Maybe 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> added, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> able 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> The 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> worked 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [Also, great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 28,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Hi 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> The 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a way 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> You 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> write in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consider
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> customers who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> store 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> This 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'brancher'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> class
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> classes'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*Alas,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Does 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> [BTW 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> KIP I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> so I will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tried to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512), 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeeded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              ��  >>>>>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> To be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> has 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Wed,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> OK, let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 1. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PROS: 1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 2. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> PROS:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Still,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) OK,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drop
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Well,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tests,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��      
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          �� >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    �� >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...).. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey()..... 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/6164 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Hello Ivan,

I’m sorry for the delay. I was waiting to see if Matthias wanted to ask any follow-up questions.

It seems like there was just one last minor question from him, and I think your answer makes perfect sense. 

I’m really happy with this proposal, thanks for working with us on it. I’m ready to vote now.

Thanks,
John

On Sun, Jun 21, 2020, at 01:03, Ivan Ponomarev wrote:
> Hello!
> 
> Just to remind: I am ready to make further improvements for this KIP and 
> the voting is also open.
> 
> Regards,
> 
> Ivan
> 
> 12.06.2020 21:07, Ivan Ponomarev пишет:
> > Hi Matthias, what do you think? Shall I update the KIP with another 
> > naming scheme?
> > 
> > Regards, Ivan
> > 
> > 05.06.2020 13:57, Ivan Ponomarev пишет:
> >>  > Can you maybe explain your reasoning? Why not just start with "0" and
> >>  > the default branch gets the next index as any other branch
> >>
> >> My reasoning was the following: when we add a new branch, all the 
> >> following branches will be renumbered. If the default branch gets the 
> >> maximum index, its handler should be modified each time a new branch 
> >> is added. If the default branch always has "0" index, we'll have to 
> >> modify less code when adding a new branch to the end of the list.
> >>
> >>  > that's a minor question.
> >>
> >> I agree that it's a minor question, although maybe for a different 
> >> reason :-)
> >>
> >> As a programmer, I don't believe in default branch naming, I'm not going
> >> to use it myself and I'm going to discourage others from using it.
> >>
> >> The code relying on default naming scheme is going to be brittle. For 
> >> example, if we want to reorder branches, we will also have to 
> >> synchronously change the code that extracts branches from the Map for 
> >> handling, with lots of opportunities to make a mistake.
> >>
> >> Low cohesion between branch predicates and branch handlers is one of 
> >> the problems of current branching API that this KIP was intended to 
> >> fight.
> >>
> >> And this is why my first goal was to avoid explicit contract for 
> >> default branch naming in KIP/JavaDoc.
> >>
> >> Note that I agree that some form of default branch naming is needed. I 
> >> can imagine cases when I'm going to use the resulting Map without 
> >> explicit knowledge of its keys (e. g. operating on map.values() only).
> >>
> >> So if for some reason you feel that other indexing scheme is more 
> >> preferable, I'm absolutely open for any proposals, because I don't 
> >> think that it is an important matter :-)
> >>
> >> Regards,
> >>
> >> Ivan
> >>
> >> 05.06.2020 3:18, Matthias J. Sax пишет:
> >>> Thanks for updating the KIP!
> >>>
> >>> I am overall happy with it. The proposed default branch numbering (ie,
> >>> start with "1" and use "0" for the default branch) is not super
> >>> intuitive, but that's a minor question.
> >>>
> >>> Can you maybe explain your reasoning? Why not just start with "0" and
> >>> the default branch gets the next index as any other branch (ie, this is
> >>> how the currently returned array indexes the branches, too)?
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 6/1/20 5:29 AM, Ivan Ponomarev wrote:
> >>>> Hi all,
> >>>>
> >>>> I updated the KIP with what we have discussed, see 'How the resulting
> >>>> Map is formed' section. Four bullet points to define the rules for Map
> >>>> keys, and four bullet points to define the rules for Map values.
> >>>>
> >>>> I also added the overloads to `Branched` in order to accept 
> >>>> Consumers as
> >>>> parameters.
> >>>>
> >>>> Regards,
> >>>>
> >>>> Ivan
> >>>>
> >>>>
> >>>> 01.06.2020 8:56, Matthias J. Sax пишет:
> >>>>> For naming:
> >>>>>
> >>>>> (1) _If_ people specify a name for split() _and_ branch(), we _must_
> >>>>> have a pulic API contract. Otherwise it would be very bad user
> >>>>> experience if we are allowed to change the names (eg, do from "we 
> >>>>> don't
> >>>>> insert a `-` to we insert a `-`). -- We also need to clarify what
> >>>>> happens is only split() _or_ branch() has a name specified, ie, is 
> >>>>> there
> >>>>> a contract for this case or not?
> >>>>>
> >>>>> (2) If we have a contract for how split() and branch() names are put
> >>>>> together, we can also apply it to auto-generated names (this seems 
> >>>>> to be
> >>>>> just consistent).
> >>>>>
> >>>>> (3) Do we want to have a contract for auto-generated names? This is
> >>>>> debatable, but I personally don't see any harm in having a contract.
> >>>>> Also, if people want to access the Map, they can easily do it without
> >>>>> specifying names if there is a contract -- if there is no contract, we
> >>>>> force people to specify names to use the Map, what seems to make 
> >>>>> the API
> >>>>> harder to use.
> >>>>>
> >>>>> But we can of course discuss furhter on the dev list.
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>> On 5/30/20 3:53 PM, Ivan Ponomarev wrote:
> >>>>>> Hi Mathhias,
> >>>>>>
> >>>>>> I received your letter just after I sent mine concerning nulls and
> >>>>>> reintroducing Consumer overload. So, if you're fine with omitting 
> >>>>>> nulls
> >>>>>> and introducing a with(Consumer) overload, it's great!
> >>>>>>
> >>>>>> Concerning the naming algorithm. Well, actually what you propose 
> >>>>>> is very
> >>>>>> clear and this is what I was going to implement anyway. But are 
> >>>>>> you sure
> >>>>>> that we will never want to change this algorithm? Here I trust your
> >>>>>> judgement, since you understand the KStreams API's evolution much 
> >>>>>> more
> >>>>>> than I do.
> >>>>>>
> >>>>>>
> >>>>>> 31.05.2020 0:50, Matthias J. Sax пишет:
> >>>>>>> Ivan, John,
> >>>>>>>
> >>>>>>> thanks for the details.
> >>>>>>>
> >>>>>>> I guess I can be convinced in either direction with regard to `null`
> >>>>>>> handling. But I am less worried about it, because I assume that a 
> >>>>>>> Java
> >>>>>>> programmer understand the ambiguty of `get(key) == null` -- 
> >>>>>>> that's why
> >>>>>>> there is `containsKey()`. In the end it might be a minor detail 
> >>>>>>> and as
> >>>>>>> long as it's well documented all discussed approaches seem fine.
> >>>>>>> Including the idea to re-introduce the `Consumer` overload.
> >>>>>>>
> >>>>>>> It's Ivan's KIP so I am fine with whatever he picks :)
> >>>>>>>
> >>>>>>>
> >>>>>>> About the naming: Not sure why we don't want to make the naming a
> >>>>>>> public
> >>>>>>> contract? I guess for split() we could default to the empty 
> >>>>>>> String, and
> >>>>>>> for `branch()` we could just add a counter by default?
> >>>>>>>
> >>>>>>> With no passed names at all, we just generte branche names "1", "2",
> >>>>>>> "3", etc... With only`split(Named.as("foo-"))` (but not branch
> >>>>>>> names) we
> >>>>>>> generate "foo-1", "foo-2", etc.
> >>>>>>>
> >>>>>>> If any branch has a dedicated name, the counting is preserved but 
> >>>>>>> just
> >>>>>>> not used int he name for this brach:
> >>>>>>>
> >>>>>>> split("foo-")
> >>>>>>>      .brach()
> >>>>>>>      .brach(Branched.as("bar")
> >>>>>>>      .defaultBranch()
> >>>>>>>
> >>>>>>> would generate "foo-1", "foo-bar", "foo-3"?
> >>>>>>>
> >>>>>>> Thoughts?
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> -Matthias
> >>>>>>>
> >>>>>>> On 5/29/20 1:20 PM, John Roesler wrote:
> >>>>>>>> Hi all,
> >>>>>>>>
> >>>>>>>> Right, that was my suggestion; sorry for the confusion. I was 
> >>>>>>>> thinking
> >>>>>>>> that adding null as a value to maps is generally dubious, for which
> >>>>>>>> reason a lot of Map implementations actually don't allow it at all.
> >>>>>>>>
> >>>>>>>> The reason is that it creates ambiguity, since a lot of code treats
> >>>>>>>> "map.get(key) == null" as meaning the key is not in the map, but
> >>>>>>>> if we place a null value in the map (presuming we don't immediately
> >>>>>>>> get an exception), then you face weird contradictions, like
> >>>>>>>> get(key) == null, but containsKey(key) can be either true or false,
> >>>>>>>> and
> >>>>>>>> you may or may not see it while iterating. Which would violate the
> >>>>>>>> expectations of many Java programs. You can just take a look at
> >>>>>>>> the AK codebase, and you'll find many occurrences where we
> >>>>>>>> assume a null mapping means the map doesn't contain the key.
> >>>>>>>>
> >>>>>>>> Of course, the practical question, which Ivan brought up, is also
> >>>>>>>> a good one. What would you actually do with these null values
> >>>>>>>> in the map, besides get an NPE ?
> >>>>>>>>
> >>>>>>>> I'm not sure why I didn't think of this before, but an alternative
> >>>>>>>> to this debate is to go back to adding Consumer into the API, but
> >>>>>>>> with the same method name, withChain(Consumer<KStream<K,V>>).
> >>>>>>>>
> >>>>>>>> Now that we have only static methods, it's less confusing 
> >>>>>>>> because it's
> >>>>>>>> not possible to choose _both_ the Function<KStream,KStream> _and_
> >>>>>>>> the Consumer<KStream> in the same branch (which was one of the
> >>>>>>>> main drawbacks of the original plan wrt withChain and
> >>>>>>>> withJavaConsumer as builder methods.
> >>>>>>>>
> >>>>>>>> I.e., I was previously concerned about:
> >>>>>>>> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2) 
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> It's just a little confusing trying to reason about how this is all
> >>>>>>>> going to wire up.
> >>>>>>>>
> >>>>>>>> But with only the static methods, users have only three methods,
> >>>>>>>> and they are all clear:
> >>>>>>>> 1. Branched.as(name): just names the branch, the branch-predicated
> >>>>>>>> stream is the value of the map
> >>>>>>>> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the
> >>>>>>>> branch, applies the transformation after the branch predicate, and
> >>>>>>>> the transformed stream is the value of the map (null is not 
> >>>>>>>> permitted)
> >>>>>>>> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch,
> >>>>>>>> passes the predicated stream in to the Consumer function and omits
> >>>>>>>> the branch from the map
> >>>>>>>>
> >>>>>>>> Sorry for digging this option up again, but it's starting to 
> >>>>>>>> look more
> >>>>>>>> attractive after we dropped the builder functions, and I think 
> >>>>>>>> it also
> >>>>>>>> resolves the "null" concern.
> >>>>>>>>
> >>>>>>>> Thoughts?
> >>>>>>>> -John
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
> >>>>>>>>> Hi Matthias!
> >>>>>>>>>
> >>>>>>>>> Thanks for your reply!
> >>>>>>>>>
> >>>>>>>>> (1)
> >>>>>>>>>
> >>>>>>>>>     > Do you imply that there won't be a `name -> null` entry 
> >>>>>>>>> in the
> >>>>>>>>> `Map`
> >>>>>>>>> for the branch? If yes, I am wondering why?
> >>>>>>>>>
> >>>>>>>>> Ah, I re-read the discussion and found out that omitting null 
> >>>>>>>>> entries
> >>>>>>>>> was John's idea, not yours :-)
> >>>>>>>>>
> >>>>>>>>> Anyway. As you remember, I was against the 'swallowing' of
> >>>>>>>>> branches by
> >>>>>>>>> Consumer. I had a feeling that there are cases when I would like
> >>>>>>>>> both to
> >>>>>>>>> consume a branch and post-process it in a Map handler.
> >>>>>>>>>
> >>>>>>>>> But then the idea of getting rid of `withJavaConsumer` came up. 
> >>>>>>>>> And
> >>>>>>>>> then
> >>>>>>>>> we realized that we should handle nulls somehow.
> >>>>>>>>>
> >>>>>>>>> So it looked like taking the best from both approaches: if we 
> >>>>>>>>> want to
> >>>>>>>>> omit a branch in the resulting map, we just return null, thus
> >>>>>>>>> emulating
> >>>>>>>>> a 'swallowing consumer'. If we want to include a branch, we return
> >>>>>>>>> it.
> >>>>>>>>> It provides the full control for a user, and it spares the user 
> >>>>>>>>> from
> >>>>>>>>> null-checking in their code.
> >>>>>>>>>
> >>>>>>>>>     > My reasoning is, that it's very clear that `name -> null`
> >>>>>>>>> would be
> >>>>>>>>> there, because the user code did execute `return null`.
> >>>>>>>>>
> >>>>>>>>> My reasoning is following. The way such lambdas are written, it's
> >>>>>>>>> hard
> >>>>>>>>> to return null unintentionally. And if a user returns null
> >>>>>>>>> intentionally, what `name->null` in the resulting Map can be good
> >>>>>>>>> for?
> >>>>>>>>> It's either NPE (which is clearly not intended) or something that
> >>>>>>>>> should
> >>>>>>>>> be null-checked and skipped. Ok, let's do it!
> >>>>>>>>>
> >>>>>>>>> But, I'm ready to be convinced to drop this rule. It doesn't seem
> >>>>>>>>> to me
> >>>>>>>>> to be a principal thing.
> >>>>>>>>>
> >>>>>>>>> (2)
> >>>>>>>>>
> >>>>>>>>>     > I think it would be good if the KIP could explains the 
> >>>>>>>>> intended
> >>>>>>>>> default naming schema. This is important because the naming schema
> >>>>>>>>> must
> >>>>>>>>> be part of the API contract; otherwise, users cannot rely on the
> >>>>>>>>> naming
> >>>>>>>>> when trying to use the returned `Map`.
> >>>>>>>>>
> >>>>>>>>> Well, in fact, I intentionally didn't include any naming schema in
> >>>>>>>>> the
> >>>>>>>>> KIP. And it's done exactly for the purpose that a user won't be
> >>>>>>>>> able to
> >>>>>>>>> rely on default naming. If they want to use certain branches 
> >>>>>>>>> from the
> >>>>>>>>> Map, they will devise their own naming strategy that will never
> >>>>>>>>> change
> >>>>>>>>> and will never let them down.
> >>>>>>>>>
> >>>>>>>>> If they don't mind what's in a Map, if they are looking up the Map
> >>>>>>>>> just
> >>>>>>>>> for debugging, or if they want to treat a Map just like a
> >>>>>>>>> Collection of
> >>>>>>>>> values (a possible use case!), it doesn't matter how the branches
> >>>>>>>>> are named.
> >>>>>>>>>
> >>>>>>>>> This way we are giving ourselves the full freedom to change the
> >>>>>>>>> naming
> >>>>>>>>> schema afterwards without violating the API contract.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>>
> >>>>>>>>> Ivan
> >>>>>>>>>
> >>>>>>>>> 29.05.2020 4:58, Matthias J. Sax пишет:
> >>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>
> >>>>>>>>>> Using covariant generics is a good idea! I am also fine with only
> >>>>>>>>>> using
> >>>>>>>>>> static method in `Branched` for now, as it's only two 
> >>>>>>>>>> parameters and
> >>>>>>>>>> thus not too many overloads.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> (1) What is unclear to me is, what you exaclty mean by:
> >>>>>>>>>>
> >>>>>>>>>>> If a function returns `null`, its result is omitted.
> >>>>>>>>>>
> >>>>>>>>>> Do you imply that there won't be a `name -> null` entry in the
> >>>>>>>>>> `Map` for
> >>>>>>>>>> the branch? If yes, I am wondering why?
> >>>>>>>>>>
> >>>>>>>>>> My previous argument was to omit an entry only for the
> >>>>>>>>>> `withJavaConsumer()` case, because the return type is "void" (and
> >>>>>>>>>> thus
> >>>>>>>>>> it's unclear what should be added and it would introduce
> >>>>>>>>>> inconsistencies
> >>>>>>>>>> in the `Map`).
> >>>>>>>>>>
> >>>>>>>>>> However, for a `Function` that returns a `KStream`, we could
> >>>>>>>>>> actually
> >>>>>>>>>> just add a `name -> null` entry if `null` is returned. My 
> >>>>>>>>>> reasoning
> >>>>>>>>>> is,
> >>>>>>>>>> that it's very clear that `name -> null` would be there, 
> >>>>>>>>>> because the
> >>>>>>>>>> user code did execute `return null`. Adding a `name -> null`
> >>>>>>>>>> entry for
> >>>>>>>>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
> >>>>>>>>>> exceptional case for handling `null`).
> >>>>>>>>>>
> >>>>>>>>>> Thoughts?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> (2) For `Branched.as()` the KIP says:
> >>>>>>>>>>
> >>>>>>>>>>> sets the name of the branch (auto-generated by default, when 
> >>>>>>>>>>> split
> >>>>>>>>>>> operation is named, then the names are index-suffixed)
> >>>>>>>>>>
> >>>>>>>>>> I think it would be good if the KIP could explains the intended
> >>>>>>>>>> default
> >>>>>>>>>> naming schema. This is important because the naming schema 
> >>>>>>>>>> must be
> >>>>>>>>>> part
> >>>>>>>>>> of the API contract; otherwise, users cannot rely on the 
> >>>>>>>>>> naming when
> >>>>>>>>>> trying to use the returned `Map`.
> >>>>>>>>>>
> >>>>>>>>>> Similalry, it should be explained how names are generated if
> >>>>>>>>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations
> >>>>>>>>>> how the
> >>>>>>>>>> API can be mixed and matched and it's unclear atm how it would
> >>>>>>>>>> work in
> >>>>>>>>>> detail.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thanks!
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 5/28/20 7:35 AM, John Roesler wrote:
> >>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for the updates. I agree, it seems like all the concerns
> >>>>>>>>>>> that have been raised in the discussion so far have been
> >>>>>>>>>>> addressed. And it's been a while since anyone raised a new
> >>>>>>>>>>> concern. At this point, it seems like a good time to start
> >>>>>>>>>>> the VOTE thread.
> >>>>>>>>>>>
> >>>>>>>>>>> Sometimes, the vote thread will trigger new people to look
> >>>>>>>>>>> into the KIP, and they may raise new concerns, but it's not a
> >>>>>>>>>>> problem. We'll just address those lingering concerns if there
> >>>>>>>>>>> are any, until you have all the votes you need.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks again for the contribution!
> >>>>>>>>>>>
> >>>>>>>>>>> -John
> >>>>>>>>>>>
> >>>>>>>>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
> >>>>>>>>>>>> Alright, I have updated the KIP with what we have discussed:
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1. Per Mathhias's suggestion, if a chain function returns null,
> >>>>>>>>>>>> the
> >>>>>>>>>>>> respective result if omitted in the resulting Map.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2. `with[Java]Consumer` method dropped.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3. `Branched` class has only three static methods with all the
> >>>>>>>>>>>> possible
> >>>>>>>>>>>> combinations of parameters.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 4. Chain function is defined 'fully covariant', let's see if we
> >>>>>>>>>>>> can
> >>>>>>>>>>>> implement it this way :-))
> >>>>>>>>>>>>
> >>>>>>>>>>>> + code example updates and minor edits.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Since this is my first KIP, I'm not sure what should I do 
> >>>>>>>>>>>> next. I
> >>>>>>>>>>>> feel
> >>>>>>>>>>>> that we talked over all the details and the consensus is 
> >>>>>>>>>>>> reached.
> >>>>>>>>>>>> Is it
> >>>>>>>>>>>> OK to call for VOTE now or is it better to wait for more 
> >>>>>>>>>>>> feedback?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Ivan
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 28.05.2020 3:26, John Roesler пишет:
> >>>>>>>>>>>>> Sounds good to me, Ivan!
> >>>>>>>>>>>>> -John
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>> John,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ---------------------
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>       > I'm sorry things have been dragging out a little, 
> >>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>> have the
> >>>>>>>>>>>>>> sense we're very close to the end of this discussion, 
> >>>>>>>>>>>>>> which is
> >>>>>>>>>>>>>> exciting.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> We are certainly moving forward!  And I'm not in a hurry at
> >>>>>>>>>>>>>> all. As I
> >>>>>>>>>>>>>> told you before in my projects I'm using Spring Kafka's
> >>>>>>>>>>>>>> KafkaStreamBrancher -- the implementation of the first,
> >>>>>>>>>>>>>> rejected version
> >>>>>>>>>>>>>> of this KIP. It's inferior to what we are discussing here, 
> >>>>>>>>>>>>>> but
> >>>>>>>>>>>>>> it does
> >>>>>>>>>>>>>> its work. So it's worth to design this KIP really, really 
> >>>>>>>>>>>>>> well.
> >>>>>>>>>>>>>> And by
> >>>>>>>>>>>>>> the way, from this discussion I'm learning the good API
> >>>>>>>>>>>>>> designing
> >>>>>>>>>>>>>> process. For me it has a value per se :-))
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ----------------------
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>       >>   > I'd wonder whether we need the non-static 
> >>>>>>>>>>>>>> builders
> >>>>>>>>>>>>>> (like withChain).
> >>>>>>>>>>>>>>       >>   > Do they provide any benefit over just using the
> >>>>>>>>>>>>>> right static
> >>>>>>>>>>>>>> factory?
> >>>>>>>>>>>>>>       >
> >>>>>>>>>>>>>>       > I don't have a strong feeling, either. It seems 
> >>>>>>>>>>>>>> nice to
> >>>>>>>>>>>>>> offer a better
> >>>>>>>>>>>>>>       > type inference experience than what we get with
> >>>>>>>>>>>>>> Materialized, by
> >>>>>>>>>>>>>>       > offering the static method that takes both name and
> >>>>>>>>>>>>>> chain.
> >>>>>>>>>>>>>>       > Given that, there doesn't seem to be a good reason to
> >>>>>>>>>>>>>> also offer the
> >>>>>>>>>>>>>>       > non-static builder-style methods, so I guess I'd 
> >>>>>>>>>>>>>> prefer
> >>>>>>>>>>>>>> to drop them.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I agree again! From a recent discussion on Twitter
> >>>>>>>>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) 
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>> found out
> >>>>>>>>>>>>>> an interesting fact about type inference rules in Java. Funny
> >>>>>>>>>>>>>> thing is
> >>>>>>>>>>>>>> that although we need to explicitly set types in a chain like
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> foo.branch(..., 
> >>>>>>>>>>>>>> Branched.<...,...>named("foo").withChain(...));
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> (otherwise it won't compile), the composition of static 
> >>>>>>>>>>>>>> method
> >>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>> works just fine, all the types are being calculated 
> >>>>>>>>>>>>>> correctly:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), 
> >>>>>>>>>>>>>> ...));
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> As I was told, for type inference there is difference between
> >>>>>>>>>>>>>> qualifiers
> >>>>>>>>>>>>>> and arguments, 'you go up if you are argument but stop if you
> >>>>>>>>>>>>>> are
> >>>>>>>>>>>>>> qualifier'. And it also seems that we should not bet on any
> >>>>>>>>>>>>>> future
> >>>>>>>>>>>>>> improvements in Java type inference here.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> So,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 1) I think we that in this KIP we should provide three static
> >>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function,
> >>>>>>>>>>>>>> String)`, and
> >>>>>>>>>>>>>> drop any non-static ones.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 2) If anything else will be ever needed, we can easily add
> >>>>>>>>>>>>>> anything.
> >>>>>>>>>>>>>> Maybe this can be done in a process of refinement of all the
> >>>>>>>>>>>>>> parameter
> >>>>>>>>>>>>>> classes.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ----------------------------
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>       > we may as well hope for the best, and propose the 
> >>>>>>>>>>>>>> "fully
> >>>>>>>>>>>>>>       > covariant" definition for now.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Understood and agreed! I will edit the KIP.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ----------------------------
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>       >> Good question, I already thought about it and 
> >>>>>>>>>>>>>> rejected
> >>>>>>>>>>>>>> the idea....
> >>>>>>>>>>>>>>       > (I cut off your quote; the rest is in the chain 
> >>>>>>>>>>>>>> below)
> >>>>>>>>>>>>>>       >  "Worst" case scenario: someone
> >>>>>>>>>>>>>>       > else will wish the return type is something 
> >>>>>>>>>>>>>> different,
> >>>>>>>>>>>>>> and we'll go
> >>>>>>>>>>>>>>       > through a painless deprecation transition to 
> >>>>>>>>>>>>>> change it
> >>>>>>>>>>>>>> later.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Of course, we cant' predict all the ways people are going to
> >>>>>>>>>>>>>> use it.
> >>>>>>>>>>>>>>       From my own humble experience with Kafka Streams, 
> >>>>>>>>>>>>>> the worst
> >>>>>>>>>>>>>> scenario is
> >>>>>>>>>>>>>> unlikely. Split is split, transform is transform, too much
> >>>>>>>>>>>>>> flexibility
> >>>>>>>>>>>>>> is often evil.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ------------------------
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> So it seems that we are close to the consensus. Two things to
> >>>>>>>>>>>>>> be altered
> >>>>>>>>>>>>>> in the current version of KIP:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> * list of Branched methods, drop non-static methods.
> >>>>>>>>>>>>>> * 'fully covariant' definition of `chained`.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Any ideas / questions / objections?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 27.05.2020 7:03, John Roesler пишет:
> >>>>>>>>>>>>>>> Thanks for the reply, Ivan,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I'm sorry things have been dragging out a little, but I have
> >>>>>>>>>>>>>>> the sense
> >>>>>>>>>>>>>>> we're very close to the end of this discussion, which is
> >>>>>>>>>>>>>>> exciting.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
> >>>>>>>>>>>>>>>> (like withChain).
> >>>>>>>>>>>>>>>>        > Do they provide any benefit over just using the 
> >>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>> static factory?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
> >>>>>>>>>>>>>>>> matter of taste.
> >>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
> >>>>>>>>>>>>>>>> builders can
> >>>>>>>>>>>>>>>> be omitted, I agree!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I don't have a strong feeling, either. It seems nice to 
> >>>>>>>>>>>>>>> offer
> >>>>>>>>>>>>>>> a better
> >>>>>>>>>>>>>>> type inference experience than what we get with
> >>>>>>>>>>>>>>> Materialized, by
> >>>>>>>>>>>>>>> offering the static method that takes both name and chain.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Given that, there doesn't seem to be a good reason to also
> >>>>>>>>>>>>>>> offer the
> >>>>>>>>>>>>>>> non-static builder-style methods, so I guess I'd prefer to
> >>>>>>>>>>>>>>> drop them.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I'll defer to Matthias, if he has a chance to consider 
> >>>>>>>>>>>>>>> whether
> >>>>>>>>>>>>>>> it's more
> >>>>>>>>>>>>>>> valuable to stick with the existing pattern or break the
> >>>>>>>>>>>>>>> pattern to offer
> >>>>>>>>>>>>>>> a better experience.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in 
> >>>>>>>>>>>>>>>> the KIP,
> >>>>>>>>>>>>>>>> which is the
> >>>>>>>>>>>>>>>>        > covariant typed function
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
> >>>>>>>>>>>>>>>> propose the
> >>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in 
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> KIP, 2)
> >>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
> >>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>> and unit testing?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been 
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>> we just
> >>>>>>>>>>>>>>> won't know what works until we really try it, and try it in
> >>>>>>>>>>>>>>> several different
> >>>>>>>>>>>>>>> ways. Still, we may as well hope for the best, and 
> >>>>>>>>>>>>>>> propose the
> >>>>>>>>>>>>>>> "fully
> >>>>>>>>>>>>>>> covariant" definition for now.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
> >>>>>>>>>>>>>>>> value
> >>>>>>>>>>>>>>>> types to be the
> >>>>>>>>>>>>>>>>        > same as the inputs?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
> >>>>>>>>>>>>>>>> idea....
> >>>>>>>>>>>>>>> (I cut off your quote; the rest is in the chain below)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> That's fair! It's your KIP, after all. I think I might have
> >>>>>>>>>>>>>>> made a different
> >>>>>>>>>>>>>>> call here, but I think this choice is fine. "Worst" case
> >>>>>>>>>>>>>>> scenario: someone
> >>>>>>>>>>>>>>> else will wish the return type is something different, and
> >>>>>>>>>>>>>>> we'll go
> >>>>>>>>>>>>>>> through a painless deprecation transition to change it 
> >>>>>>>>>>>>>>> later.
> >>>>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>>>> to the clean design of your API, this doesn't seem to bad.
> >>>>>>>>>>>>>>> And, of
> >>>>>>>>>>>>>>> course, you've actually been using similar functionality
> >>>>>>>>>>>>>>> already, so it
> >>>>>>>>>>>>>>> seems we should trust your intuition.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>> Hi John,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
> >>>>>>>>>>>>>>>> (like withChain).
> >>>>>>>>>>>>>>>> Do they provide any benefit over just using the right 
> >>>>>>>>>>>>>>>> static
> >>>>>>>>>>>>>>>> factory?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
> >>>>>>>>>>>>>>>> matter of taste.
> >>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
> >>>>>>>>>>>>>>>> builders can
> >>>>>>>>>>>>>>>> be omitted, I agree!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in 
> >>>>>>>>>>>>>>>> the KIP,
> >>>>>>>>>>>>>>>> which is the
> >>>>>>>>>>>>>>>> covariant typed function
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
> >>>>>>>>>>>>>>>> propose the
> >>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in 
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> KIP, 2)
> >>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
> >>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>> and unit testing?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
> >>>>>>>>>>>>>>>> value
> >>>>>>>>>>>>>>>> types to be the
> >>>>>>>>>>>>>>>> same as the inputs?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
> >>>>>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Look, if we want to keep `withChain`'s function optional,
> >>>>>>>>>>>>>>>> then we must
> >>>>>>>>>>>>>>>> keep the result key and value types the same. Because for
> >>>>>>>>>>>>>>>> now, the
> >>>>>>>>>>>>>>>> default value for the 'chain function' is 
> >>>>>>>>>>>>>>>> Function.identity().
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Of course, we can make the 'chain function' required. But I
> >>>>>>>>>>>>>>>> think this
> >>>>>>>>>>>>>>>> is not what `split` method is for. `split` is for
> >>>>>>>>>>>>>>>> splitting, not
> >>>>>>>>>>>>>>>> transforming, and `chainFunction` in most of the cases 
> >>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>> be either
> >>>>>>>>>>>>>>>> a consumer or the identity function.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
> >>>>>>>>>>>>>>>>> Thanks for the reply, Ivan.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past
> >>>>>>>>>>>>>>>>> that generic type inference problem is to offer a static
> >>>>>>>>>>>>>>>>> factory method that takes all the options (both name and
> >>>>>>>>>>>>>>>>> function).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I’m happy with adding that option, although if we have all
> >>>>>>>>>>>>>>>>> possible combinations available as static methods, then 
> >>>>>>>>>>>>>>>>> I’d
> >>>>>>>>>>>>>>>>> wonder whether we need the non-static builders (like
> >>>>>>>>>>>>>>>>> withChain). Do they provide any benefit over just using 
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> right static factory?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function 
> >>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>> bounds works algebraically, but I’m not sure whether the
> >>>>>>>>>>>>>>>>> limitations of Java type inference will actually let you
> >>>>>>>>>>>>>>>>> pass in all the different functions you would want to.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I recall Bruno and I wrestling with this when he added
> >>>>>>>>>>>>>>>>> Transform operators. In retrospect, this is a problem 
> >>>>>>>>>>>>>>>>> we can
> >>>>>>>>>>>>>>>>> iron out when we write tests in the PR. You might as well
> >>>>>>>>>>>>>>>>> propose the “ideal” API in the KIP, which is the covariant
> >>>>>>>>>>>>>>>>> typed function:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
> >>>>>>>>>>>>>>>>> KStream<? extends K, ? extends V>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 3. Actually, I have a new question about the types: Is it
> >>>>>>>>>>>>>>>>> necessary to restrict the result key and value types to be
> >>>>>>>>>>>>>>>>> the same as the inputs? I.e., shouldn’t the result 
> >>>>>>>>>>>>>>>>> types be
> >>>>>>>>>>>>>>>>> K1,V1?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>> John
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>> Hello John,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>> ---------------------------------------------
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>         > Perhaps it would be better to stick with "as"
> >>>>>>>>>>>>>>>>>> for now
> >>>>>>>>>>>>>>>>>>         > and just file a Jira to switch them all at 
> >>>>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>>>> time [for
> >>>>>>>>>>>>>>>>>> compatibility with Kotlin]
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin
> >>>>>>>>>>>>>>>>>> they have a
> >>>>>>>>>>>>>>>>>> standard workaround
> >>>>>>>>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin). 
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> So actually this should be a very low priority issue, 
> >>>>>>>>>>>>>>>>>> if an
> >>>>>>>>>>>>>>>>>> issue at
> >>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>         > I don't understand how your new proposed
> >>>>>>>>>>>>>>>>>>         > methods would work any differently than the 
> >>>>>>>>>>>>>>>>>> ones
> >>>>>>>>>>>>>>>>>> you already
> >>>>>>>>>>>>>>>>>>         > had proposed in the KIP. It seems like you'd 
> >>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>> have to provide
> >>>>>>>>>>>>>>>>>>         > the generic type parameters on the first static
> >>>>>>>>>>>>>>>>>> factory call. Can you
> >>>>>>>>>>>>>>>>>>         > explain how your new interface proposal differs
> >>>>>>>>>>>>>>>>>> from the existing KIP?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> In the KIP, I didn't clarify what methods should be 
> >>>>>>>>>>>>>>>>>> static.
> >>>>>>>>>>>>>>>>>> Now I
> >>>>>>>>>>>>>>>>>> propose the following methods:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> non-static: withChain(Function), withName(String).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> static: as(String), with(Function), with(Function, 
> >>>>>>>>>>>>>>>>>> String).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The overloaded `with` version that provides both Function
> >>>>>>>>>>>>>>>>>> and name can
> >>>>>>>>>>>>>>>>>> be used without causing type inference problem!!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>> ----------------------------
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>         > Regarding making the K,V types covariant 
> >>>>>>>>>>>>>>>>>> also, yes,
> >>>>>>>>>>>>>>>>>> that would indeed
> >>>>>>>>>>>>>>>>>>         > be nice, but I'm not sure it will actually 
> >>>>>>>>>>>>>>>>>> work.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> What I'm keeping in mind is the following example: 
> >>>>>>>>>>>>>>>>>> imagine
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> static KStream<String, Integer> func(KStream<String,
> >>>>>>>>>>>>>>>>>> Number> s) {
> >>>>>>>>>>>>>>>>>>                 return s.mapValues(n -> (Integer) n + 1);
> >>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> BranchedKStream<String, Number> b =
> >>>>>>>>>>>>>>>>>>             s.split().branch((k, v) -> isInteger(v),
> >>>>>>>>>>>>>>>>>>                        //Won't compile!!
> >>>>>>>>>>>>>>>>>>                        Branched.with(Me::func));
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The simple workaround here is to change `func`'s return
> >>>>>>>>>>>>>>>>>> type from
> >>>>>>>>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> [On the other hand, we already agreed to remove
> >>>>>>>>>>>>>>>>>> `withJavaConsumer` from
> >>>>>>>>>>>>>>>>>> `Branched`, so during code migration I will have to 
> >>>>>>>>>>>>>>>>>> modify
> >>>>>>>>>>>>>>>>>> my functions'
> >>>>>>>>>>>>>>>>>> return types anyway -- I mean, from `void` to 
> >>>>>>>>>>>>>>>>>> `KStream`!! ]
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>         >  the map you're returning is Map<K,V>, and of
> >>>>>>>>>>>>>>>>>> course a K is not the
> >>>>>>>>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I think what you actually meant here is that KStream<?
> >>>>>>>>>>>>>>>>>> extends K, ?
> >>>>>>>>>>>>>>>>>> extends V> is not fit as a value for Map<String, 
> >>>>>>>>>>>>>>>>>> KStream<K,
> >>>>>>>>>>>>>>>>>> V>>. This
> >>>>>>>>>>>>>>>>>> particularly is not a problem, since KStream<? extends 
> >>>>>>>>>>>>>>>>>> K, ?
> >>>>>>>>>>>>>>>>>> extends V>
> >>>>>>>>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be 
> >>>>>>>>>>>>>>>>>> put
> >>>>>>>>>>>>>>>>>> to the map.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard
> >>>>>>>>>>>>>>>>>> types. So maybe
> >>>>>>>>>>>>>>>>>> for now it's better to just admit that API is not
> >>>>>>>>>>>>>>>>>> absolutely perfect and
> >>>>>>>>>>>>>>>>>> accept it as is, that is
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
> >>>>>>>>>>>>>>>>>>> Hello Ivan,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that
> >>>>>>>>>>>>>>>>>>> "as" would
> >>>>>>>>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart 
> >>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>> convention
> >>>>>>>>>>>>>>>>>>> and just avoid methods named "as" in the future.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> The convention is that "as(String name)" is used for the
> >>>>>>>>>>>>>>>>>>> static factory
> >>>>>>>>>>>>>>>>>>> method, whereas "withName(String name)" is an instance
> >>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to
> >>>>>>>>>>>>>>>>>>> avoid "as"
> >>>>>>>>>>>>>>>>>>> for compatibility with Kotlin, I might suggest
> >>>>>>>>>>>>>>>>>>> "fromName(String name)",
> >>>>>>>>>>>>>>>>>>> although it's somewhat dubious, since all the other
> >>>>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick 
> >>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> "as" for now
> >>>>>>>>>>>>>>>>>>> and just file a Jira to switch them all at the same 
> >>>>>>>>>>>>>>>>>>> time.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Re. 3:
> >>>>>>>>>>>>>>>>>>> Regarding the type inference problem, yes, it's a 
> >>>>>>>>>>>>>>>>>>> blemish
> >>>>>>>>>>>>>>>>>>> on all of our
> >>>>>>>>>>>>>>>>>>> configuraion objects. The problem is that Java infers 
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>> based on the _first_ method in the chain. While it does
> >>>>>>>>>>>>>>>>>>> consider what
> >>>>>>>>>>>>>>>>>>> the recipient of the method result wants, it only
> >>>>>>>>>>>>>>>>>>> considers the _next_
> >>>>>>>>>>>>>>>>>>> recipient.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it 
> >>>>>>>>>>>>>>>>>>> to a
> >>>>>>>>>>>>>>>>>>> Branched<String,String> variable, java infers the type
> >>>>>>>>>>>>>>>>>>> correctly. But
> >>>>>>>>>>>>>>>>>>> when the "next recipient" is a chained method call, like
> >>>>>>>>>>>>>>>>>>> "withChain",
> >>>>>>>>>>>>>>>>>>> then the chained method doesn't bound the type (by
> >>>>>>>>>>>>>>>>>>> definition,
> >>>>>>>>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so 
> >>>>>>>>>>>>>>>>>>> Java
> >>>>>>>>>>>>>>>>>>> will take
> >>>>>>>>>>>>>>>>>>> the broadest possible inferece and bind the type to
> >>>>>>>>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be
> >>>>>>>>>>>>>>>>>>> revised anymore.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, 
> >>>>>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>>>> doesn't seem
> >>>>>>>>>>>>>>>>>>> that hard to recursively consider the entire context 
> >>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>> inferring the
> >>>>>>>>>>>>>>>>>>> generic type parameters, but this is what we have to 
> >>>>>>>>>>>>>>>>>>> work
> >>>>>>>>>>>>>>>>>>> with.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> To be honest, though, I don't understand how your new
> >>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>> methods would work any differently than the ones you
> >>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still 
> >>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>> the generic type parameters on the first static factory
> >>>>>>>>>>>>>>>>>>> call. Can you
> >>>>>>>>>>>>>>>>>>> explain how your new interface proposal differs from the
> >>>>>>>>>>>>>>>>>>> existing KIP?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Re. 4:
> >>>>>>>>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that
> >>>>>>>>>>>>>>>>>>> would indeed
> >>>>>>>>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You 
> >>>>>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>> want to give it a
> >>>>>>>>>>>>>>>>>>> try. In the past, we've run into soe truly strange
> >>>>>>>>>>>>>>>>>>> interactions between the
> >>>>>>>>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner
> >>>>>>>>>>>>>>>>>>> classes) in
> >>>>>>>>>>>>>>>>>>> combination with nested covariant types.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Another issue is that the value type of the map you're
> >>>>>>>>>>>>>>>>>>> returning is
> >>>>>>>>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? 
> >>>>>>>>>>>>>>>>>>> extends
> >>>>>>>>>>>>>>>>>>> K", so it
> >>>>>>>>>>>>>>>>>>> doesn't seem compatible.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks again,
> >>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that
> >>>>>>>>>>>>>>>>>>>> getting rid of
> >>>>>>>>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as
> >>>>>>>>>>>>>>>>>>>> possible, but not
> >>>>>>>>>>>>>>>>>>>> simpler'.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to
> >>>>>>>>>>>>>>>>>>>> implement examples
> >>>>>>>>>>>>>>>>>>>> from KIP.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 1. Having to return something from lambda is not a very
> >>>>>>>>>>>>>>>>>>>> big deal.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use
> >>>>>>>>>>>>>>>>>>>> method references
> >>>>>>>>>>>>>>>>>>>> for already written stream consumers, but then I 
> >>>>>>>>>>>>>>>>>>>> realized
> >>>>>>>>>>>>>>>>>>>> that I can
> >>>>>>>>>>>>>>>>>>>> just change my methods from returning void to returning
> >>>>>>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>>>>> parameter and use references to them. Not very
> >>>>>>>>>>>>>>>>>>>> convenient, but passable.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no
> >>>>>>>>>>>>>>>>>>>> consumer 2) when
> >>>>>>>>>>>>>>>>>>>> function returns null, we don't insert it into the
> >>>>>>>>>>>>>>>>>>>> resulting map.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but
> >>>>>>>>>>>>>>>>>>>> workable solution as
> >>>>>>>>>>>>>>>>>>>> a first approximation. And later we can always add to
> >>>>>>>>>>>>>>>>>>>> `Branched`
> >>>>>>>>>>>>>>>>>>>> anything we want.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes
> >>>>>>>>>>>>>>>>>>>> like Branched
> >>>>>>>>>>>>>>>>>>>> should be built? First of all, it seems that `as` 
> >>>>>>>>>>>>>>>>>>>> now is
> >>>>>>>>>>>>>>>>>>>> more preferred
> >>>>>>>>>>>>>>>>>>>> than `withName` (although as you probably know it 
> >>>>>>>>>>>>>>>>>>>> clashes
> >>>>>>>>>>>>>>>>>>>> with Kotlin's
> >>>>>>>>>>>>>>>>>>>> `as` operator).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out 
> >>>>>>>>>>>>>>>>>>>> that my
> >>>>>>>>>>>>>>>>>>>> Java cannot
> >>>>>>>>>>>>>>>>>>>> infer types in the following construction:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
> >>>>>>>>>>>>>>>>>>>>             Branched.as("foo").withChain(s ->
> >>>>>>>>>>>>>>>>>>>> s.mapValues(...)))
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> so I have to write
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
> >>>>>>>>>>>>>>>>>>>>             Branched.<String, 
> >>>>>>>>>>>>>>>>>>>> String>as("foo").withChain(s
> >>>>>>>>>>>>>>>>>>>> -> s.mapValues(...)))
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of
> >>>>>>>>>>>>>>>>>>>> `Branched` methods
> >>>>>>>>>>>>>>>>>>>> that I came to (will you please validate it):
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
> >>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
> >>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain);
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
> >>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
> >>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> //non-static!
> >>>>>>>>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, 
> >>>>>>>>>>>>>>>>>>>> V>,
> >>>>>>>>>>>>>>>>>>>> ? extends
> >>>>>>>>>>>>>>>>>>>> KStream<K, V>> chain);
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 4. And one more. What do you think, do we need that
> >>>>>>>>>>>>>>>>>>>> flexibility:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, 
> >>>>>>>>>>>>>>>>>>>> V>>
> >>>>>>>>>>>>>>>>>>>> chain
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> vs.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? 
> >>>>>>>>>>>>>>>>>>>> extends
> >>>>>>>>>>>>>>>>>>>> KStream<?
> >>>>>>>>>>>>>>>>>>>> extends K, ? extends V>> chain
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> ??
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
> >>>>>>>>>>>>>>>>>>>>> Thanks for this thought, Matthias,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
> >>>>>>>>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be 
> >>>>>>>>>>>>>>>>>>>>> in the
> >>>>>>>>>>>>>>>>>>>>> map, because there’s only one thing that could be 
> >>>>>>>>>>>>>>>>>>>>> there,
> >>>>>>>>>>>>>>>>>>>>> which is whatever is returned from the chain function.
> >>>>>>>>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the 
> >>>>>>>>>>>>>>>>>>>>> extensible way
> >>>>>>>>>>>>>>>>>>>>> this KIP is designed, it would be trivially easy to 
> >>>>>>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>>> the “terminal” chain later. As you say, fewer concepts
> >>>>>>>>>>>>>>>>>>>>> leads to an API that is easier to learn.
> >>>>>>>>>>>>>>>>>>>>> 3. We get to side-step the naming of this method.
> >>>>>>>>>>>>>>>>>>>>> Although I didn’t complain about withJavaConsumer, it
> >>>>>>>>>>>>>>>>>>>>> was only because I couldn’t think of a better name.
> >>>>>>>>>>>>>>>>>>>>> Still, it’s somewhat unsatisfying to name a method 
> >>>>>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>> its argument type, since this provides no 
> >>>>>>>>>>>>>>>>>>>>> information at
> >>>>>>>>>>>>>>>>>>>>> all about what the method does. I was willing to 
> >>>>>>>>>>>>>>>>>>>>> accept
> >>>>>>>>>>>>>>>>>>>>> it because I didn’t have an alternative, but I 
> >>>>>>>>>>>>>>>>>>>>> would be
> >>>>>>>>>>>>>>>>>>>>> happy to skip this method for now to avoid the problem
> >>>>>>>>>>>>>>>>>>>>> until we have more inspiration.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> The only con I see is that it makes the code a little
> >>>>>>>>>>>>>>>>>>>>> less ergonomic to write when you don’t want to return
> >>>>>>>>>>>>>>>>>>>>> the result of the chain (such as when the chain is
> >>>>>>>>>>>>>>>>>>>>> terminal), since I’m your example, you have to 
> >>>>>>>>>>>>>>>>>>>>> declare a
> >>>>>>>>>>>>>>>>>>>>> block with a return statement at the end. It’s not
> >>>>>>>>>>>>>>>>>>>>> ideal, but it doesn’t seem too bad to me.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with 
> >>>>>>>>>>>>>>>>>>>>> allowing
> >>>>>>>>>>>>>>>>>>>>> a null result, which would just remove the branch from
> >>>>>>>>>>>>>>>>>>>>> the returned map. It seems nicer than forcing 
> >>>>>>>>>>>>>>>>>>>>> people to
> >>>>>>>>>>>>>>>>>>>>> pick a stream to return when their chain is 
> >>>>>>>>>>>>>>>>>>>>> terminal and
> >>>>>>>>>>>>>>>>>>>>> they don’t want to use the result later.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks again for sharing the idea,
> >>>>>>>>>>>>>>>>>>>>> John
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I guess the only open question is about
> >>>>>>>>>>>>>>>>>>>>>> `Branched.withJavaConsumer` and
> >>>>>>>>>>>>>>>>>>>>>> its relationship to the returned `Map`.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>           (1) split a stream and return the 
> >>>>>>>>>>>>>>>>>>>>>> substreams
> >>>>>>>>>>>>>>>>>>>>>> for futher processing
> >>>>>>>>>>>>>>>>>>>>>>           (2) split a stream and modify the 
> >>>>>>>>>>>>>>>>>>>>>> substreams
> >>>>>>>>>>>>>>>>>>>>>> with in-place method chaining
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            -> split a stream, modify the 
> >>>>>>>>>>>>>>>>>>>>>> substreams, and
> >>>>>>>>>>>>>>>>>>>>>> return the _modified_
> >>>>>>>>>>>>>>>>>>>>>> substreams for further processing
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
> >>>>>>>>>>>>>>>>>>>>>>> split the topology graph at any point. Technically,
> >>>>>>>>>>>>>>>>>>>>>>> it's OK to do both: feed the KStream to a
> >>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer AND save it in resulting Map. If one
> >>>>>>>>>>>>>>>>>>>>>>> doesn't need the stream in the Map, one simply does
> >>>>>>>>>>>>>>>>>>>>>>> not extract it from there
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> That is of course possible. However, it introduces 
> >>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>> "hidded" semantics:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>           - using `withChain` I get the modified
> >>>>>>>>>>>>>>>>>>>>>> sub-stream
> >>>>>>>>>>>>>>>>>>>>>>           - using `withJavaConsumer` I get the 
> >>>>>>>>>>>>>>>>>>>>>> unmodifed
> >>>>>>>>>>>>>>>>>>>>>> sub-stream
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> This seems to be quite subtle to me.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>          From my understanding the original idea of
> >>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` was to
> >>>>>>>>>>>>>>>>>>>>>> model a terminal operation, ie, it should be 
> >>>>>>>>>>>>>>>>>>>>>> similar to:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
> >>>>>>>>>>>>>>>>>>>>>>            s.to();
> >>>>>>>>>>>>>>>>>>>>>>            return null;
> >>>>>>>>>>>>>>>>>>>>>> })
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> However, I am not sure if we should even allow
> >>>>>>>>>>>>>>>>>>>>>> `withChain()` to return
> >>>>>>>>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this
> >>>>>>>>>>>>>>>>>>>>>> case to avoid a `key
> >>>>>>>>>>>>>>>>>>>>>> -> null` entry in the returned Map.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Following this train of through, and if we want to
> >>>>>>>>>>>>>>>>>>>>>> allow the "return
> >>>>>>>>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer`
> >>>>>>>>>>>>>>>>>>>>>> that does not add
> >>>>>>>>>>>>>>>>>>>>>> an entry to the Map.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Following your proposal, the semantics of
> >>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` could also
> >>>>>>>>>>>>>>>>>>>>>> be achieved with `withChain`:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
> >>>>>>>>>>>>>>>>>>>>>>            s.to();
> >>>>>>>>>>>>>>>>>>>>>>            return s;
> >>>>>>>>>>>>>>>>>>>>>> })
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely
> >>>>>>>>>>>>>>>>>>>>>> syntactic sugar,
> >>>>>>>>>>>>>>>>>>>>>> while for the first proposal it adds new 
> >>>>>>>>>>>>>>>>>>>>>> functionality
> >>>>>>>>>>>>>>>>>>>>>> (if `return null`
> >>>>>>>>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not 
> >>>>>>>>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>>>>>>> "hide a
> >>>>>>>>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need
> >>>>>>>>>>>>>>>>>>>>>> to allow `return
> >>>>>>>>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a
> >>>>>>>>>>>>>>>>>>>>>> sub-stream in the Map.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we
> >>>>>>>>>>>>>>>>>>>>>> follow your
> >>>>>>>>>>>>>>>>>>>>>> proposal, I am wondering if we need 
> >>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer`
> >>>>>>>>>>>>>>>>>>>>>> at all? Its
> >>>>>>>>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API
> >>>>>>>>>>>>>>>>>>>>>> is usually
> >>>>>>>>>>>>>>>>>>>>>> preferable as it's simpler to learn.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> -----------------------------------------
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> John,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox,
> >>>>>>>>>>>>>>>>>>>>>>> fortunately we have
> >>>>>>>>>>>>>>>>>>>>>>> all the emails on the web.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) 
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> existing ‘branch’
> >>>>>>>>>>>>>>>>>>>>>>> method?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration
> >>>>>>>>>>>>>>>>>>>>>>> Plan" section.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to
> >>>>>>>>>>>>>>>>>>>>>>>> branching directly
> >>>>>>>>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic
> >>>>>>>>>>>>>>>>>>>>>>> branching]
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic
> >>>>>>>>>>>>>>>>>>>>>>> Branching' section.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka
> >>>>>>>>>>>>>>>>>>>>>>>> Consumer... maybe `withSink`?]
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be 
> >>>>>>>>>>>>>>>>>>>>>>> confusing. I
> >>>>>>>>>>>>>>>>>>>>>>> renamed this
> >>>>>>>>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' 
> >>>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use 
> >>>>>>>>>>>>>>>>>>>>>>>> cases:
> >>>>>>>>>>>>>>>>>>>>>>>> EITHER using
> >>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> This is discussed below.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> ----------------------------------------------
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Mathhias,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` ->
> >>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`]
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Done.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' 
> >>>>>>>>>>>>>>>>>>>>>>>> version of
> >>>>>>>>>>>>>>>>>>>>>>>> the `branch`
> >>>>>>>>>>>>>>>>>>>>>>> method]
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Fixed.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that
> >>>>>>>>>>>>>>>>>>>>>>>> using both
> >>>>>>>>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
> >>>>>>>>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of
> >>>>>>>>>>>>>>>>>>>>>>> `withChain()` will
> >>>>>>>>>>>>>>>>>>>>>>> be given to the `Consumer`.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Yes, I agree!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the
> >>>>>>>>>>>>>>>>>>>>>>>> returned `Map` of
> >>>>>>>>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a
> >>>>>>>>>>>>>>>>>>>>>>> reasonable
> >>>>>>>>>>>>>>>>>>>>>>> implementation would be to not add the "branch" 
> >>>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>> result map if
> >>>>>>>>>>>>>>>>>>>>>>> `withConsumer` is used?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
> >>>>>>>>>>>>>>>>>>>>>>> split the topology
> >>>>>>>>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both:
> >>>>>>>>>>>>>>>>>>>>>>> feed the KStream to
> >>>>>>>>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If 
> >>>>>>>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>>>> doesn't need the
> >>>>>>>>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it 
> >>>>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>> there :-)
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the
> >>>>>>>>>>>>>>>>>>>>>>> returned map
> >>>>>>>>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs
> >>>>>>>>>>>>>>>>>>>>>>> explicitly set by the
> >>>>>>>>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids.
> >>>>>>>>>>>>>>>>>>>>>>> Dealing with this
> >>>>>>>>>>>>>>>>>>>>>>> map is the user's responsibility.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing
> >>>>>>>>>>>>>>>>>>>>>>> exclusions to this
> >>>>>>>>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by
> >>>>>>>>>>>>>>>>>>>>>>> provided
> >>>>>>>>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated.
> >>>>>>>>>>>>>>>>>>>>>>> What if a user
> >>>>>>>>>>>>>>>>>>>>>>> provides both the name of the branch and a
> >>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer? What do they
> >>>>>>>>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or
> >>>>>>>>>>>>>>>>>>>>>>> save it to the map?
> >>>>>>>>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this 
> >>>>>>>>>>>>>>>>>>>>>>> map, so
> >>>>>>>>>>>>>>>>>>>>>>> maybe just leave
> >>>>>>>>>>>>>>>>>>>>>>> it as it is?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> ----
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Looking forward for your feedback again!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Ivan.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I also have some minor comment:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` ->
> >>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg,
> >>>>>>>>>>>>>>>>>>>>>>>> CoGroupedKStream,
> >>>>>>>>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the 
> >>>>>>>>>>>>>>>>>>>>>>>> "legacy"
> >>>>>>>>>>>>>>>>>>>>>>>> `KGroupedStream`
> >>>>>>>>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename 
> >>>>>>>>>>>>>>>>>>>>>>>> without a
> >>>>>>>>>>>>>>>>>>>>>>>> breaking change...
> >>>>>>>>>>>>>>>>>>>>>>>> so we just keep them.)
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> (2) Quote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have
> >>>>>>>>>>>>>>>>>>>>>>>>> overloaded
> >>>>>>>>>>>>>>>>>>>>>>>>> parameterless alternatives.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a 
> >>>>>>>>>>>>>>>>>>>>>>>> `Predicate`
> >>>>>>>>>>>>>>>>>>>>>>>> and assume you
> >>>>>>>>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe
> >>>>>>>>>>>>>>>>>>>>>>>> rephrase it accordingly
> >>>>>>>>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public
> >>>>>>>>>>>>>>>>>>>>>>>> Interface" section) of
> >>>>>>>>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
> >>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces) 
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction
> >>>>>>>>>>>>>>>>>>>>>>>> of `withConsumer()`
> >>>>>>>>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. 
> >>>>>>>>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>>>>> related to John's
> >>>>>>>>>>>>>>>>>>>>>>>> 4th comment:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use
> >>>>>>>>>>>>>>>>>>>>>>>>> cases: EITHER using
> >>>>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and
> >>>>>>>>>>>>>>>>>>>>>>>> `withConsumer()` is the
> >>>>>>>>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the
> >>>>>>>>>>>>>>>>>>>>>>>> result of `withChain()`
> >>>>>>>>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is 
> >>>>>>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>>> with the `Consumer`
> >>>>>>>>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and
> >>>>>>>>>>>>>>>>>>>>>>>> `noDefaultBranch()`.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to 
> >>>>>>>>>>>>>>>>>>>>>>>> not add
> >>>>>>>>>>>>>>>>>>>>>>>> the "branch" to
> >>>>>>>>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As 
> >>>>>>>>>>>>>>>>>>>>>>>> long as
> >>>>>>>>>>>>>>>>>>>>>>>> we clearly document
> >>>>>>>>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’
> >>>>>>>>>>>>>>>>>>>>>>>>> confusing; I thought you
> >>>>>>>>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface
> >>>>>>>>>>>>>>>>>>>>>>>>> (which doesn’t make
> >>>>>>>>>>>>>>>>>>>>>>>>> sense, of course). I get that you were 
> >>>>>>>>>>>>>>>>>>>>>>>>> referring to
> >>>>>>>>>>>>>>>>>>>>>>>>> the java Consumer
> >>>>>>>>>>>>>>>>>>>>>>>>> interface, but we should still probably to to 
> >>>>>>>>>>>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>>>>>> the ambiguity.
> >>>>>>>>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about 
> >>>>>>>>>>>>>>>>>>>>>>>>> ‘withSink’?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be
> >>>>>>>>>>>>>>>>>>>>>>>> confused with a "sink
> >>>>>>>>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less 
> >>>>>>>>>>>>>>>>>>>>>>>> ambiguous?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I
> >>>>>>>>>>>>>>>>>>>>>>>>> think it’s close,
> >>>>>>>>>>>>>>>>>>>>>>>>> but I had a few last comments.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>> John
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev 
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> will someone please take a look at the 
> >>>>>>>>>>>>>>>>>>>>>>>>>> reworked KIP?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I believe that now it follows design 
> >>>>>>>>>>>>>>>>>>>>>>>>>> principles and
> >>>>>>>>>>>>>>>>>>>>>>>>>> takes into account
> >>>>>>>>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design 
> >>>>>>>>>>>>>>>>>>>>>>>>>>> principles" and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have completely
> >>>>>>>>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
> >>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> This version includes all the previous 
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>> results and follows
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> The exception is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branched)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one
> >>>>>>>>>>>>>>>>>>>>>>>>>>> parameter' rule, but I think
> >>>>>>>>>>>>>>>>>>>>>>>>>>> here it is justified.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and 
> >>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> need to provide one
> >>>>>>>>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we may use a single
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method
> >>>>>>>>>>>>>>>>>>>>>>>>>>> parameter for `branch`.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part 
> >>>>>>>>>>>>>>>>>>>>>>>>>>> of a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch, no
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.'
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is expected here
> >>>>>>>>>>>>>>>>>>>>>>>>>>> as it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single 
> >>>>>>>>>>>>>>>>>>>>>>>>>>> parameter
> >>>>>>>>>>>>>>>>>>>>>>>>>>> rule'.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> have you back!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> already have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable to me to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> build on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> design
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> principles" that we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> want to follow:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> participation in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> June
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> and it didn't leave me
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we invested
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> not feel entitled to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalized.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> reviewing Kafka
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching, Spring-Kafka's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> original idea for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I gave up pushing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem with the scope of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<>();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .branch(....)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .defaultBranch(result::set)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .onTopOf(someStream);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> result.get()...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this approach.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution in his post
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Optional
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> names seems to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> your code brittle.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> purposes. Or,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> finally,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> named
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> original scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> June 4th, 2019, who
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The idea seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> trust
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> people who are more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> me.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status".
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Feel free to resume
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at any point.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP, feel free to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do so.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about my proposal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to mix
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worlds?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the `Map` back
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chaining" pattern
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of course be done.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `String`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is possible.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to merge both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> patterns
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional or required in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your example.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by it?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The returned
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should we prefix
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`, but optional in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense, if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suffix using a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> counter
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lead to the problem of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added/removed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, how would the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not all branches are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen that a user
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misses to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to runtime issues.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allow a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default name but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in `branch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explicit to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a configuration and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hence
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a `Consumer`?)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent a user to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what does not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a compile time
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that neither
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig`. A config
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> independently of all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> others,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to blend the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two main
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded chaining"?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operations also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams, if we want
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to add
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I this this is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> taught us
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that, although
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience when you can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> source code. Since you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chain extra
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument),
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can avoid the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Branch, you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can name the branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get index-suffixed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> guess
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branching operator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split(Named.withName("mysplit"))
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //creates node
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                           .branch(..., 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ...,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "abranch") // creates node
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // creates node
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> syntax
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> itself, though.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plenty of room to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> debate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation, but in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> general,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operator(function,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config_object?) OR
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "function"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but some other config
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implements
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               operator(function, function,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> string)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specified
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> roll all these
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                interface BranchConfig 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extends
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation {
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withPredicate(...);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withName(...);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchConfig something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that pattern.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noisier,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but it also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> range of alternatives
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purely
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deal with adding
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> decide we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want the name to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from my point of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sax
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage of both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "name")`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessible
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>) 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pick to add a name
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or not. The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names are unique. The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use the Map.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best of both worlds.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start" operator in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just questioning the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity, I'm in favor
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the present
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> life to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about using method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be validating
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to flattening the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hand.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of which disrupt
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different reasons). In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is *the* first-order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tells us
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that function
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see that by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they continue
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> callback-heavy language.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let you convert
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> popular variation on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is an effort to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offer the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (because you get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domain,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and Java+KStreams is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can derive some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domains. I'm just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the same time,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to the problem.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> post-branch merge
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can clearly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sink
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operators. But is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Ivan, I’ll definitely
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forfeit my point on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch(predicate, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumer)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, I don’t see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the dynamic case.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 IMO the one trade 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> off to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider at this point is the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 question. I don’t 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally agree that “we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the same scope” 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> merging the branches back
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 seems like a perfectly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plausible use case that can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 when the branched 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in the same scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the reasons Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> listed, I think it is overall the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution - working 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the scope thing is easy
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > On May 2, 2019, at 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7:00
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Hello everyone, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thank you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all for joining the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Well, I don't 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea of named branches,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 LinkedHashMap (no 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other Map
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will do, because order of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 matters) or `branch`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method  taking name and Consumer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 advantages than 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > In my opinion, the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real positive outcome from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal is that all 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned branches are in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 But 1) we rarely 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope 2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 workaround for the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, described in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 'Inlining the complex
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic' is not a problem,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method references 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of lambdas. In real world
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 tend to split the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complex
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic to methods anyway,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 going to be clean.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > The drawbacks are 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> strong.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The cohesion between
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers is lost. We 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to define predicates in one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers in another. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opens the door for bugs:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forget to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> define a handler for a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a handler?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misspell a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and duplicate a name?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > What Michael propose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have been totally OK
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 writing the API in Lua,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ruby or Python. In those
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "dynamic naming" 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have looked most
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 beautiful. But in 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect all the problems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 identifiers to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eliminated in compile time.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Do we have to invent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> duck-typing for the Java API?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > And if we do, what
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage are we supposed to get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 all the branches in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same scope? Michael, maybe I'm
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 point?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > ---
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Earlier in this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion John Roesler also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without "start 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator, and later Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the case when we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a dynamic number of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current KIP is 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'clumsier'
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared to Michael's 'Map'
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 me address both 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comments
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 1) "Start branching"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator (I think that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for it indeed) is 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we need to do a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 see example below.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 2) No, dynamic 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in current KIP is not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Imagine a real-world
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenario when you need one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 value (say, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordType).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You can have something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*John:if we had 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to start
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with stream.branch(...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 have been much 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messier.*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > KBranchedStream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = stream.split();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*Not clumsy at 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all :-)*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > for (RecordType
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType : RecordType.values())
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >             
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched =
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched.branch((k, v) ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recordType,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType::processRecords);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 02.05.2019 14:40,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> I also agree with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's observation about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> current `branch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> However, I also 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to pass in a clumsy Map
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 thinking
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> was more aligned 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's proposal to just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `branch()` 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statement and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> It makes the code 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read, and also make the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `Predicates` 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (that is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essential) easier to grasp.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        ��        >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Map<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>    .branch(
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> An open question 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for which no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> specified. Atm,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()` would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `BranchedKStream`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> and the call to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()` that returns the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> (what is not the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> atm). Or is this actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> because users can 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ignore the branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `defaultBranch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> in the result 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` ?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> About "inlining": So
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> far, it seems to be a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> preference. I can 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments for both, but no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 argument" yet
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> that clearly make 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for one or the other.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> On 5/1/19 6:26 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Perhaps inlining 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong terminology. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that a lambda with 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the full
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream topology be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 it can be a method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reference as with Ivan’s original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The advantage of 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the predicate and its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 (Consumer) together in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch() is that they are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to each other.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Ultimately the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream code has to live
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch trees will be 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read regardless.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> On May 1, 2019, at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> I'm less 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enthusiastic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about inlining the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 downstream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> functionality.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Programs that have deep branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 quickly become
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> harder to read 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> single unit.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    >>>>> Also +1 on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues/goals as Michael
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that sets a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> great 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> framework for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Regarding the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution, my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> proposal in 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is what is in my PR which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 decisions) is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> roughly this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(Consumer<KStream<K, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Obviously some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering is necessary, since
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 construct
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't work 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, but this solution seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 provides as much
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> associativity 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> directly 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associates
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "conditional" with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The value it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> provides over 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> The KIP 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> less "dynamic" than the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the sense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that it is 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsier to add a dynamic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branches, but it is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> certainly 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems to me like the API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the "static"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> case anyway, and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should make it simple and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluently declare and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> access your 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in-line.  It also makes it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ignore a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch, and it is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to build an (almost)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solution on 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> top of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> I could also 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> middle ground where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> taken in, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes a name and not a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 like this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Map<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>    .branch(
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Pros for that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - accessing 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams in same scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - no double brace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization, hopefully
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 readable than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Cons
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - downstream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic cannot be specified
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes it harder
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to read top to 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (like existing API and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - you can 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forget to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "handle" one of the branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API and 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> (KBranchedStreams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could even work *both* ways
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that's overdoing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Overall I'm 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> curious
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how important it is to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branched 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 possible that it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't need 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled directly by the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 left up to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user.  I'm 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sort of in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the middle on it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:48 PM Sophie
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <sophie@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I'd like to 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +1 what
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael said about the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> method, I 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what he's outlined and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proceed by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> trying to 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> alleviate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these problems.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 important to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> able to cleanly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the individual
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name->stream), which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thought was the original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> That said, I 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we should so easily
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 double brace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> anti-pattern or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> force ours users into it if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> avoid...just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> my two cents.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Sophie
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> On Tue, Apr 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 12:59 PM Michael
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I’d like to 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a different way of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 To me,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> are three 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 1. If you 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way most people do, Java
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> warnings.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2. The way 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you use the stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 positionally coupled
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the ordering 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 3. It is 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> brittle to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend existing branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 additional code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> paths.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Using 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associative
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> constructs instead of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 constructs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be a stronger
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach. Consider a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Map<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K,V>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Predicate<?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> super K,? super
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Branches are 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> given
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names in a map, and as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> mapping of 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams. The ordering
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> maintained
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> because it’s a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sorted map. Insert order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> evaluation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> This solves 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1 because there are no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solves
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2 because 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer lean on ordering to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch you’re
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> interested 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves problem 3 because
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 another
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> conditional by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply attaching another
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 structure, rather
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> messing with 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing indices.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> One of the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks is that creating the map
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 historically
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> awkward in 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java. I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know it’s an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 voluminously, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> double brace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization would clean up the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> On Tue, Apr 30,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <john@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> update.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> FWIW, I 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias that the current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> operator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> confusing when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named the same way as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "Split"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like a good 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alternatively, we can do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> operator at 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and just do:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Tentatively, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that this branching
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> terminal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> way, we don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> create ambiguity about how
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, `branch`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> should return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream`, while
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `void`, to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> enforce 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes last, and that there
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 definition of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> default 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Potentially, we should log a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 there's no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> default,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> and 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> additionally
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log a warning (or throw an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 record
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> falls
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> though with no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> On Fri, Apr 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 26,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Thanks for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP and your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> this is 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name similar to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> that also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The intend 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid name duplication.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> _not_
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be an array.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> IMHO, this 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a little odd, because
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `branch()` does
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> take any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameters and has different
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> `branch()` 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note, that from the code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> hidden
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call is `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Because I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested to rename
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 I though
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> might be 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to also rename
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> naming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> overlap that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems to be confusing. The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> me:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Maybe 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better alternative to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 avoid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> naming 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overlap.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 'default' 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> however, a reserved word, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cannot
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a method with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such name :-)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Bummer. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Didn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider this. Maybe we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> short
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> name?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Can you 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> methods? 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be part of public API and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contained in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP. For 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> example,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's unclear atm, what the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> You did not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on the idea to add a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#get(int
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> index)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -> KStream`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method to get the individually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branched-KStreams. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> nice to 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feedback about it. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that users
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> would need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write custom utility code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access them.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> should 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discuss
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the pros and cons of both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "incomplete" to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me atm, if the API has no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched-KStreams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> directly.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> On 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 4/13/19 2:13
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I have 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP-418 according to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Matthias, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thanks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for your comment!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I can see 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point: this is to make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> that also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right? But
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 loss of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can have overloaded
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> affecting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> the existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code. Maybe the old
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this is a subject for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Totally 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with 'addBranch->branch'
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> however, a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> reserved 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> word,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so unfortunately we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with such
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> name
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> :-)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does take an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> is not 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Absolutely! I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that was just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 something.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Dear 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> colleagues,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> please 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> revise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new version of the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512) 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Any new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions/objections?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 11.04.2019
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Thanks for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> driving the discussion of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> everybody
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> agrees 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current branch() method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> optimal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> I had a 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> quick
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> look into the PR and I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> There
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> are some 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> minor
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> things we need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recommend the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> renaming:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> It's just a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion to get slightly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> In the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR, defaultBranch() does
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> but I think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that is not required?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Also, we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider KIP-307, that was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 accepted and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> currently
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implemented:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Ie, we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add overloads that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 parameter.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> For the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the created
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> scopes:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> could we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` with a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 index)` method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> returns the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding "branched"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 object?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Maybe,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> second 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of `addBranch()` should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function<KStream,KStream>` and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 whatever
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> `Function`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Finally, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would also suggest to update
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal. That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes it easier to review.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On 3/31/19
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> I'm a 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bit of a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice here as well, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes sense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> revise 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and continue the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we'll
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> buy-in 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committers that have actual
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> whether
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> could be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> adopted.  It would be great
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 think this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> idea 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure if that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> vote,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> generally some indication of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said, I'll continue the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 assuming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> forward 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution of "stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream",
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> deprecate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "stream.branch(...) returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> favor
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since having two mutually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> accomplish
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> same 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing, especially when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 similar
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> just 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be sure we're not making
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible/difficult
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> is 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> currently
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible/easy.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR - I think the general
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> just a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> little 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sloppy
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall in terms of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> particular,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> passing 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "predicates" and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> modified
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream but read from all 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KStreamLazyBranch is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> follow.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On Fri, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I read 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code carefully and now I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 convinced: your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> looks 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and should work. We just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> crucial
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers are invoked as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 And then
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> going 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> very nice.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> What 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shall we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do now? I should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 resume the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here, right?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Why 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> telling that your PR
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> point
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we go 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction'? To me it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this project I might
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> details.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 28.03.2019
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Maybe 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I’m
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing the point, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> supports
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The couponIssuer::set*
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> invoked
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they’re
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> added, 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 still
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> ought
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> able 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 depend on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> streams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> having been set.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> The 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean to point out is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solutions.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> worked 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [Also, great
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear additional
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 excited
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> hear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts!]
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> On 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 28,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Hi 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> The 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> postpone the wiring of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> glance,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> the 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> newly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched streams are not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is, if we wanted to merge
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> again
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> don't see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a way 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> You 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> took the words right out
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> going
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> write in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details about this issue.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consider
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the example from Bill's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> identify
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> customers who
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have bought coffee and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> electronics
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> store 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to give
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them coupons.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> This 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code I usually write under
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 circumstances
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'brancher'
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> class
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer{
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> private
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> private
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<...> coupons(){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the real world the code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 complex, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> creation of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> separate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer class is fully
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 order to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> separate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> classes'
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> responsibilities.*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*Alas,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this won't work if we're
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 everything
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> later,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation!!!*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Does 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?  In order to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 initialize the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> is 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> [BTW 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just found out that your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 essentially
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> KIP I was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to write here. I have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 based on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> experience,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> so I will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join the discussion on KIP-401
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 28.03.2019
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tried to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make a very rough proof of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512), 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeeded
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at removing both cons.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Compatibility: I was incorrect
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> compatibility
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aren't any direct ones.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that Java
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> smart
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returning one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> thing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring a terminal method:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> build
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up the branches in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 shares
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorSupplier that will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branching.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> pretty
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in its current form, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    demonstrates
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> To be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear, I don't think that pull
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> final
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> even a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point if we go in this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 wanted to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will say
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though, that I'm not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in favor of this, which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> suggested
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> was a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 streams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope as each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, if we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> wanted
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together again I don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that.  The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> has 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same issue, though - all this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> either
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 table.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> On 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Wed,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> OK, let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me summarize what we have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems that it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> needs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two potential ways to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 1. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> origianlly proposed)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PROS: 1)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Fully backwards
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             ��   >>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> sense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary ingredients are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to create a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 instance
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluency
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of other KStream methods.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 2. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul proposes)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> PROS:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Generally follows the way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defined.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to define two terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 miss the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> of 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal methods should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 methods
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can throw an exception in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your point when you are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Still,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that this cannot
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 implemented the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> easy
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we all should think further.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on two of your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could specify a terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default branch,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) OK,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apparently this should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> besides
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 want to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> silently
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drop
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the messages that didn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 predicate. 2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Throwing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> like a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> bad
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream processing paradigm,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 emit a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to a dedicated stream.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> `default`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> You
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean a runtime exception,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> compiled
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> run?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Well,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd prefer an API that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compile if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> used
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method chain
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object? There is a huge
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 between
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> instantly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tests,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it costs more for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compilation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point about the terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 required.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bad thing?  If the user
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defaultBranch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other terminal method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��      
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it creates an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> default
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 which allows
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need for a terminal operation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> well
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> "build
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> step"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users to do other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 stream.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> do -
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to process off the original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> times,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a variable so you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operations
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> On
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> Hello
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afraid this won't work
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 always need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> In my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, onTopOf
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 so we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the original branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand your point that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> object
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 flow,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idiomatic.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it's a great idea to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 API, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> find
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contrasts the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> fluency
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream method calls.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> method on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it still reads top to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 cases
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          �� >>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is very
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> nice
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do things, but what if we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 how we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 terminated by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 incompatible with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch() would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> name,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a fairly small problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> Does
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this satisfy the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KIP?  It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> me,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allowing for clear in-line
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 while also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> allowing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KBranchedStreams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> On
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> Hi
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ks){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 String> ks){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleFirstCase)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleSecondCase)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(....)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> Hi
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have one question, the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> takes a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Consumer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 example in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we had
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    �� >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...).. 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey()..... 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> On
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bbejeck@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 418.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Please
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 feedback :)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/6164 
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Attachments:
> >>>>>>>>>>>>>>>>>>>>>> * signature.asc
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> > 
> 
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hello!

Just to remind: I am ready to make further improvements for this KIP and 
the voting is also open.

Regards,

Ivan

12.06.2020 21:07, Ivan Ponomarev пишет:
> Hi Matthias, what do you think? Shall I update the KIP with another 
> naming scheme?
> 
> Regards, Ivan
> 
> 05.06.2020 13:57, Ivan Ponomarev пишет:
>>  > Can you maybe explain your reasoning? Why not just start with "0" and
>>  > the default branch gets the next index as any other branch
>>
>> My reasoning was the following: when we add a new branch, all the 
>> following branches will be renumbered. If the default branch gets the 
>> maximum index, its handler should be modified each time a new branch 
>> is added. If the default branch always has "0" index, we'll have to 
>> modify less code when adding a new branch to the end of the list.
>>
>>  > that's a minor question.
>>
>> I agree that it's a minor question, although maybe for a different 
>> reason :-)
>>
>> As a programmer, I don't believe in default branch naming, I'm not going
>> to use it myself and I'm going to discourage others from using it.
>>
>> The code relying on default naming scheme is going to be brittle. For 
>> example, if we want to reorder branches, we will also have to 
>> synchronously change the code that extracts branches from the Map for 
>> handling, with lots of opportunities to make a mistake.
>>
>> Low cohesion between branch predicates and branch handlers is one of 
>> the problems of current branching API that this KIP was intended to 
>> fight.
>>
>> And this is why my first goal was to avoid explicit contract for 
>> default branch naming in KIP/JavaDoc.
>>
>> Note that I agree that some form of default branch naming is needed. I 
>> can imagine cases when I'm going to use the resulting Map without 
>> explicit knowledge of its keys (e. g. operating on map.values() only).
>>
>> So if for some reason you feel that other indexing scheme is more 
>> preferable, I'm absolutely open for any proposals, because I don't 
>> think that it is an important matter :-)
>>
>> Regards,
>>
>> Ivan
>>
>> 05.06.2020 3:18, Matthias J. Sax пишет:
>>> Thanks for updating the KIP!
>>>
>>> I am overall happy with it. The proposed default branch numbering (ie,
>>> start with "1" and use "0" for the default branch) is not super
>>> intuitive, but that's a minor question.
>>>
>>> Can you maybe explain your reasoning? Why not just start with "0" and
>>> the default branch gets the next index as any other branch (ie, this is
>>> how the currently returned array indexes the branches, too)?
>>>
>>>
>>> -Matthias
>>>
>>> On 6/1/20 5:29 AM, Ivan Ponomarev wrote:
>>>> Hi all,
>>>>
>>>> I updated the KIP with what we have discussed, see 'How the resulting
>>>> Map is formed' section. Four bullet points to define the rules for Map
>>>> keys, and four bullet points to define the rules for Map values.
>>>>
>>>> I also added the overloads to `Branched` in order to accept 
>>>> Consumers as
>>>> parameters.
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>>
>>>> 01.06.2020 8:56, Matthias J. Sax пишет:
>>>>> For naming:
>>>>>
>>>>> (1) _If_ people specify a name for split() _and_ branch(), we _must_
>>>>> have a pulic API contract. Otherwise it would be very bad user
>>>>> experience if we are allowed to change the names (eg, do from "we 
>>>>> don't
>>>>> insert a `-` to we insert a `-`). -- We also need to clarify what
>>>>> happens is only split() _or_ branch() has a name specified, ie, is 
>>>>> there
>>>>> a contract for this case or not?
>>>>>
>>>>> (2) If we have a contract for how split() and branch() names are put
>>>>> together, we can also apply it to auto-generated names (this seems 
>>>>> to be
>>>>> just consistent).
>>>>>
>>>>> (3) Do we want to have a contract for auto-generated names? This is
>>>>> debatable, but I personally don't see any harm in having a contract.
>>>>> Also, if people want to access the Map, they can easily do it without
>>>>> specifying names if there is a contract -- if there is no contract, we
>>>>> force people to specify names to use the Map, what seems to make 
>>>>> the API
>>>>> harder to use.
>>>>>
>>>>> But we can of course discuss furhter on the dev list.
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 5/30/20 3:53 PM, Ivan Ponomarev wrote:
>>>>>> Hi Mathhias,
>>>>>>
>>>>>> I received your letter just after I sent mine concerning nulls and
>>>>>> reintroducing Consumer overload. So, if you're fine with omitting 
>>>>>> nulls
>>>>>> and introducing a with(Consumer) overload, it's great!
>>>>>>
>>>>>> Concerning the naming algorithm. Well, actually what you propose 
>>>>>> is very
>>>>>> clear and this is what I was going to implement anyway. But are 
>>>>>> you sure
>>>>>> that we will never want to change this algorithm? Here I trust your
>>>>>> judgement, since you understand the KStreams API's evolution much 
>>>>>> more
>>>>>> than I do.
>>>>>>
>>>>>>
>>>>>> 31.05.2020 0:50, Matthias J. Sax пишет:
>>>>>>> Ivan, John,
>>>>>>>
>>>>>>> thanks for the details.
>>>>>>>
>>>>>>> I guess I can be convinced in either direction with regard to `null`
>>>>>>> handling. But I am less worried about it, because I assume that a 
>>>>>>> Java
>>>>>>> programmer understand the ambiguty of `get(key) == null` -- 
>>>>>>> that's why
>>>>>>> there is `containsKey()`. In the end it might be a minor detail 
>>>>>>> and as
>>>>>>> long as it's well documented all discussed approaches seem fine.
>>>>>>> Including the idea to re-introduce the `Consumer` overload.
>>>>>>>
>>>>>>> It's Ivan's KIP so I am fine with whatever he picks :)
>>>>>>>
>>>>>>>
>>>>>>> About the naming: Not sure why we don't want to make the naming a
>>>>>>> public
>>>>>>> contract? I guess for split() we could default to the empty 
>>>>>>> String, and
>>>>>>> for `branch()` we could just add a counter by default?
>>>>>>>
>>>>>>> With no passed names at all, we just generte branche names "1", "2",
>>>>>>> "3", etc... With only`split(Named.as("foo-"))` (but not branch
>>>>>>> names) we
>>>>>>> generate "foo-1", "foo-2", etc.
>>>>>>>
>>>>>>> If any branch has a dedicated name, the counting is preserved but 
>>>>>>> just
>>>>>>> not used int he name for this brach:
>>>>>>>
>>>>>>> split("foo-")
>>>>>>>      .brach()
>>>>>>>      .brach(Branched.as("bar")
>>>>>>>      .defaultBranch()
>>>>>>>
>>>>>>> would generate "foo-1", "foo-bar", "foo-3"?
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 5/29/20 1:20 PM, John Roesler wrote:
>>>>>>>> Hi all,
>>>>>>>>
>>>>>>>> Right, that was my suggestion; sorry for the confusion. I was 
>>>>>>>> thinking
>>>>>>>> that adding null as a value to maps is generally dubious, for which
>>>>>>>> reason a lot of Map implementations actually don't allow it at all.
>>>>>>>>
>>>>>>>> The reason is that it creates ambiguity, since a lot of code treats
>>>>>>>> "map.get(key) == null" as meaning the key is not in the map, but
>>>>>>>> if we place a null value in the map (presuming we don't immediately
>>>>>>>> get an exception), then you face weird contradictions, like
>>>>>>>> get(key) == null, but containsKey(key) can be either true or false,
>>>>>>>> and
>>>>>>>> you may or may not see it while iterating. Which would violate the
>>>>>>>> expectations of many Java programs. You can just take a look at
>>>>>>>> the AK codebase, and you'll find many occurrences where we
>>>>>>>> assume a null mapping means the map doesn't contain the key.
>>>>>>>>
>>>>>>>> Of course, the practical question, which Ivan brought up, is also
>>>>>>>> a good one. What would you actually do with these null values
>>>>>>>> in the map, besides get an NPE ?
>>>>>>>>
>>>>>>>> I'm not sure why I didn't think of this before, but an alternative
>>>>>>>> to this debate is to go back to adding Consumer into the API, but
>>>>>>>> with the same method name, withChain(Consumer<KStream<K,V>>).
>>>>>>>>
>>>>>>>> Now that we have only static methods, it's less confusing 
>>>>>>>> because it's
>>>>>>>> not possible to choose _both_ the Function<KStream,KStream> _and_
>>>>>>>> the Consumer<KStream> in the same branch (which was one of the
>>>>>>>> main drawbacks of the original plan wrt withChain and
>>>>>>>> withJavaConsumer as builder methods.
>>>>>>>>
>>>>>>>> I.e., I was previously concerned about:
>>>>>>>> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2) 
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> It's just a little confusing trying to reason about how this is all
>>>>>>>> going to wire up.
>>>>>>>>
>>>>>>>> But with only the static methods, users have only three methods,
>>>>>>>> and they are all clear:
>>>>>>>> 1. Branched.as(name): just names the branch, the branch-predicated
>>>>>>>> stream is the value of the map
>>>>>>>> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the
>>>>>>>> branch, applies the transformation after the branch predicate, and
>>>>>>>> the transformed stream is the value of the map (null is not 
>>>>>>>> permitted)
>>>>>>>> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch,
>>>>>>>> passes the predicated stream in to the Consumer function and omits
>>>>>>>> the branch from the map
>>>>>>>>
>>>>>>>> Sorry for digging this option up again, but it's starting to 
>>>>>>>> look more
>>>>>>>> attractive after we dropped the builder functions, and I think 
>>>>>>>> it also
>>>>>>>> resolves the "null" concern.
>>>>>>>>
>>>>>>>> Thoughts?
>>>>>>>> -John
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>>>>>>>>> Hi Matthias!
>>>>>>>>>
>>>>>>>>> Thanks for your reply!
>>>>>>>>>
>>>>>>>>> (1)
>>>>>>>>>
>>>>>>>>>     > Do you imply that there won't be a `name -> null` entry 
>>>>>>>>> in the
>>>>>>>>> `Map`
>>>>>>>>> for the branch? If yes, I am wondering why?
>>>>>>>>>
>>>>>>>>> Ah, I re-read the discussion and found out that omitting null 
>>>>>>>>> entries
>>>>>>>>> was John's idea, not yours :-)
>>>>>>>>>
>>>>>>>>> Anyway. As you remember, I was against the 'swallowing' of
>>>>>>>>> branches by
>>>>>>>>> Consumer. I had a feeling that there are cases when I would like
>>>>>>>>> both to
>>>>>>>>> consume a branch and post-process it in a Map handler.
>>>>>>>>>
>>>>>>>>> But then the idea of getting rid of `withJavaConsumer` came up. 
>>>>>>>>> And
>>>>>>>>> then
>>>>>>>>> we realized that we should handle nulls somehow.
>>>>>>>>>
>>>>>>>>> So it looked like taking the best from both approaches: if we 
>>>>>>>>> want to
>>>>>>>>> omit a branch in the resulting map, we just return null, thus
>>>>>>>>> emulating
>>>>>>>>> a 'swallowing consumer'. If we want to include a branch, we return
>>>>>>>>> it.
>>>>>>>>> It provides the full control for a user, and it spares the user 
>>>>>>>>> from
>>>>>>>>> null-checking in their code.
>>>>>>>>>
>>>>>>>>>     > My reasoning is, that it's very clear that `name -> null`
>>>>>>>>> would be
>>>>>>>>> there, because the user code did execute `return null`.
>>>>>>>>>
>>>>>>>>> My reasoning is following. The way such lambdas are written, it's
>>>>>>>>> hard
>>>>>>>>> to return null unintentionally. And if a user returns null
>>>>>>>>> intentionally, what `name->null` in the resulting Map can be good
>>>>>>>>> for?
>>>>>>>>> It's either NPE (which is clearly not intended) or something that
>>>>>>>>> should
>>>>>>>>> be null-checked and skipped. Ok, let's do it!
>>>>>>>>>
>>>>>>>>> But, I'm ready to be convinced to drop this rule. It doesn't seem
>>>>>>>>> to me
>>>>>>>>> to be a principal thing.
>>>>>>>>>
>>>>>>>>> (2)
>>>>>>>>>
>>>>>>>>>     > I think it would be good if the KIP could explains the 
>>>>>>>>> intended
>>>>>>>>> default naming schema. This is important because the naming schema
>>>>>>>>> must
>>>>>>>>> be part of the API contract; otherwise, users cannot rely on the
>>>>>>>>> naming
>>>>>>>>> when trying to use the returned `Map`.
>>>>>>>>>
>>>>>>>>> Well, in fact, I intentionally didn't include any naming schema in
>>>>>>>>> the
>>>>>>>>> KIP. And it's done exactly for the purpose that a user won't be
>>>>>>>>> able to
>>>>>>>>> rely on default naming. If they want to use certain branches 
>>>>>>>>> from the
>>>>>>>>> Map, they will devise their own naming strategy that will never
>>>>>>>>> change
>>>>>>>>> and will never let them down.
>>>>>>>>>
>>>>>>>>> If they don't mind what's in a Map, if they are looking up the Map
>>>>>>>>> just
>>>>>>>>> for debugging, or if they want to treat a Map just like a
>>>>>>>>> Collection of
>>>>>>>>> values (a possible use case!), it doesn't matter how the branches
>>>>>>>>> are named.
>>>>>>>>>
>>>>>>>>> This way we are giving ourselves the full freedom to change the
>>>>>>>>> naming
>>>>>>>>> schema afterwards without violating the API contract.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>
>>>>>>>>>> Using covariant generics is a good idea! I am also fine with only
>>>>>>>>>> using
>>>>>>>>>> static method in `Branched` for now, as it's only two 
>>>>>>>>>> parameters and
>>>>>>>>>> thus not too many overloads.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> (1) What is unclear to me is, what you exaclty mean by:
>>>>>>>>>>
>>>>>>>>>>> If a function returns `null`, its result is omitted.
>>>>>>>>>>
>>>>>>>>>> Do you imply that there won't be a `name -> null` entry in the
>>>>>>>>>> `Map` for
>>>>>>>>>> the branch? If yes, I am wondering why?
>>>>>>>>>>
>>>>>>>>>> My previous argument was to omit an entry only for the
>>>>>>>>>> `withJavaConsumer()` case, because the return type is "void" (and
>>>>>>>>>> thus
>>>>>>>>>> it's unclear what should be added and it would introduce
>>>>>>>>>> inconsistencies
>>>>>>>>>> in the `Map`).
>>>>>>>>>>
>>>>>>>>>> However, for a `Function` that returns a `KStream`, we could
>>>>>>>>>> actually
>>>>>>>>>> just add a `name -> null` entry if `null` is returned. My 
>>>>>>>>>> reasoning
>>>>>>>>>> is,
>>>>>>>>>> that it's very clear that `name -> null` would be there, 
>>>>>>>>>> because the
>>>>>>>>>> user code did execute `return null`. Adding a `name -> null`
>>>>>>>>>> entry for
>>>>>>>>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>>>>>>>>> exceptional case for handling `null`).
>>>>>>>>>>
>>>>>>>>>> Thoughts?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> (2) For `Branched.as()` the KIP says:
>>>>>>>>>>
>>>>>>>>>>> sets the name of the branch (auto-generated by default, when 
>>>>>>>>>>> split
>>>>>>>>>>> operation is named, then the names are index-suffixed)
>>>>>>>>>>
>>>>>>>>>> I think it would be good if the KIP could explains the intended
>>>>>>>>>> default
>>>>>>>>>> naming schema. This is important because the naming schema 
>>>>>>>>>> must be
>>>>>>>>>> part
>>>>>>>>>> of the API contract; otherwise, users cannot rely on the 
>>>>>>>>>> naming when
>>>>>>>>>> trying to use the returned `Map`.
>>>>>>>>>>
>>>>>>>>>> Similalry, it should be explained how names are generated if
>>>>>>>>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations
>>>>>>>>>> how the
>>>>>>>>>> API can be mixed and matched and it's unclear atm how it would
>>>>>>>>>> work in
>>>>>>>>>> detail.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks!
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>
>>>>>>>>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>>>>>>>>> that have been raised in the discussion so far have been
>>>>>>>>>>> addressed. And it's been a while since anyone raised a new
>>>>>>>>>>> concern. At this point, it seems like a good time to start
>>>>>>>>>>> the VOTE thread.
>>>>>>>>>>>
>>>>>>>>>>> Sometimes, the vote thread will trigger new people to look
>>>>>>>>>>> into the KIP, and they may raise new concerns, but it's not a
>>>>>>>>>>> problem. We'll just address those lingering concerns if there
>>>>>>>>>>> are any, until you have all the votes you need.
>>>>>>>>>>>
>>>>>>>>>>> Thanks again for the contribution!
>>>>>>>>>>>
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>>>>>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. Per Mathhias's suggestion, if a chain function returns null,
>>>>>>>>>>>> the
>>>>>>>>>>>> respective result if omitted in the resulting Map.
>>>>>>>>>>>>
>>>>>>>>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>>>>>>>>
>>>>>>>>>>>> 3. `Branched` class has only three static methods with all the
>>>>>>>>>>>> possible
>>>>>>>>>>>> combinations of parameters.
>>>>>>>>>>>>
>>>>>>>>>>>> 4. Chain function is defined 'fully covariant', let's see if we
>>>>>>>>>>>> can
>>>>>>>>>>>> implement it this way :-))
>>>>>>>>>>>>
>>>>>>>>>>>> + code example updates and minor edits.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Since this is my first KIP, I'm not sure what should I do 
>>>>>>>>>>>> next. I
>>>>>>>>>>>> feel
>>>>>>>>>>>> that we talked over all the details and the consensus is 
>>>>>>>>>>>> reached.
>>>>>>>>>>>> Is it
>>>>>>>>>>>> OK to call for VOTE now or is it better to wait for more 
>>>>>>>>>>>> feedback?
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Ivan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>>>>>>>>> Sounds good to me, Ivan!
>>>>>>>>>>>>> -John
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ---------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>       > I'm sorry things have been dragging out a little, 
>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>> have the
>>>>>>>>>>>>>> sense we're very close to the end of this discussion, 
>>>>>>>>>>>>>> which is
>>>>>>>>>>>>>> exciting.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> We are certainly moving forward!  And I'm not in a hurry at
>>>>>>>>>>>>>> all. As I
>>>>>>>>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>>>>>>>>> KafkaStreamBrancher -- the implementation of the first,
>>>>>>>>>>>>>> rejected version
>>>>>>>>>>>>>> of this KIP. It's inferior to what we are discussing here, 
>>>>>>>>>>>>>> but
>>>>>>>>>>>>>> it does
>>>>>>>>>>>>>> its work. So it's worth to design this KIP really, really 
>>>>>>>>>>>>>> well.
>>>>>>>>>>>>>> And by
>>>>>>>>>>>>>> the way, from this discussion I'm learning the good API
>>>>>>>>>>>>>> designing
>>>>>>>>>>>>>> process. For me it has a value per se :-))
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ----------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>       >>   > I'd wonder whether we need the non-static 
>>>>>>>>>>>>>> builders
>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>       >>   > Do they provide any benefit over just using the
>>>>>>>>>>>>>> right static
>>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>>       >
>>>>>>>>>>>>>>       > I don't have a strong feeling, either. It seems 
>>>>>>>>>>>>>> nice to
>>>>>>>>>>>>>> offer a better
>>>>>>>>>>>>>>       > type inference experience than what we get with
>>>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>>>       > offering the static method that takes both name and
>>>>>>>>>>>>>> chain.
>>>>>>>>>>>>>>       > Given that, there doesn't seem to be a good reason to
>>>>>>>>>>>>>> also offer the
>>>>>>>>>>>>>>       > non-static builder-style methods, so I guess I'd 
>>>>>>>>>>>>>> prefer
>>>>>>>>>>>>>> to drop them.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>>>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) 
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>> found out
>>>>>>>>>>>>>> an interesting fact about type inference rules in Java. Funny
>>>>>>>>>>>>>> thing is
>>>>>>>>>>>>>> that although we need to explicitly set types in a chain like
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> foo.branch(..., 
>>>>>>>>>>>>>> Branched.<...,...>named("foo").withChain(...));
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (otherwise it won't compile), the composition of static 
>>>>>>>>>>>>>> method
>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>> works just fine, all the types are being calculated 
>>>>>>>>>>>>>> correctly:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), 
>>>>>>>>>>>>>> ...));
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> As I was told, for type inference there is difference between
>>>>>>>>>>>>>> qualifiers
>>>>>>>>>>>>>> and arguments, 'you go up if you are argument but stop if you
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>> qualifier'. And it also seems that we should not bet on any
>>>>>>>>>>>>>> future
>>>>>>>>>>>>>> improvements in Java type inference here.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> So,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1) I think we that in this KIP we should provide three static
>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function,
>>>>>>>>>>>>>> String)`, and
>>>>>>>>>>>>>> drop any non-static ones.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2) If anything else will be ever needed, we can easily add
>>>>>>>>>>>>>> anything.
>>>>>>>>>>>>>> Maybe this can be done in a process of refinement of all the
>>>>>>>>>>>>>> parameter
>>>>>>>>>>>>>> classes.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>       > we may as well hope for the best, and propose the 
>>>>>>>>>>>>>> "fully
>>>>>>>>>>>>>>       > covariant" definition for now.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>       >> Good question, I already thought about it and 
>>>>>>>>>>>>>> rejected
>>>>>>>>>>>>>> the idea....
>>>>>>>>>>>>>>       > (I cut off your quote; the rest is in the chain 
>>>>>>>>>>>>>> below)
>>>>>>>>>>>>>>       >  "Worst" case scenario: someone
>>>>>>>>>>>>>>       > else will wish the return type is something 
>>>>>>>>>>>>>> different,
>>>>>>>>>>>>>> and we'll go
>>>>>>>>>>>>>>       > through a painless deprecation transition to 
>>>>>>>>>>>>>> change it
>>>>>>>>>>>>>> later.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Of course, we cant' predict all the ways people are going to
>>>>>>>>>>>>>> use it.
>>>>>>>>>>>>>>       From my own humble experience with Kafka Streams, 
>>>>>>>>>>>>>> the worst
>>>>>>>>>>>>>> scenario is
>>>>>>>>>>>>>> unlikely. Split is split, transform is transform, too much
>>>>>>>>>>>>>> flexibility
>>>>>>>>>>>>>> is often evil.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> So it seems that we are close to the consensus. Two things to
>>>>>>>>>>>>>> be altered
>>>>>>>>>>>>>> in the current version of KIP:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>>>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Any ideas / questions / objections?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'm sorry things have been dragging out a little, but I have
>>>>>>>>>>>>>>> the sense
>>>>>>>>>>>>>>> we're very close to the end of this discussion, which is
>>>>>>>>>>>>>>> exciting.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>>>        > Do they provide any benefit over just using the 
>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>> static factory?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I don't have a strong feeling, either. It seems nice to 
>>>>>>>>>>>>>>> offer
>>>>>>>>>>>>>>> a better
>>>>>>>>>>>>>>> type inference experience than what we get with
>>>>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Given that, there doesn't seem to be a good reason to also
>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>> non-static builder-style methods, so I guess I'd prefer to
>>>>>>>>>>>>>>> drop them.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'll defer to Matthias, if he has a chance to consider 
>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>> it's more
>>>>>>>>>>>>>>> valuable to stick with the existing pattern or break the
>>>>>>>>>>>>>>> pattern to offer
>>>>>>>>>>>>>>> a better experience.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in 
>>>>>>>>>>>>>>>> the KIP,
>>>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>>>>        > covariant typed function
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
>>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been 
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> we just
>>>>>>>>>>>>>>> won't know what works until we really try it, and try it in
>>>>>>>>>>>>>>> several different
>>>>>>>>>>>>>>> ways. Still, we may as well hope for the best, and 
>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>> "fully
>>>>>>>>>>>>>>> covariant" definition for now.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>>>>        > same as the inputs?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>>>> idea....
>>>>>>>>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> That's fair! It's your KIP, after all. I think I might have
>>>>>>>>>>>>>>> made a different
>>>>>>>>>>>>>>> call here, but I think this choice is fine. "Worst" case
>>>>>>>>>>>>>>> scenario: someone
>>>>>>>>>>>>>>> else will wish the return type is something different, and
>>>>>>>>>>>>>>> we'll go
>>>>>>>>>>>>>>> through a painless deprecation transition to change it 
>>>>>>>>>>>>>>> later.
>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>> to the clean design of your API, this doesn't seem to bad.
>>>>>>>>>>>>>>> And, of
>>>>>>>>>>>>>>> course, you've actually been using similar functionality
>>>>>>>>>>>>>>> already, so it
>>>>>>>>>>>>>>> seems we should trust your intuition.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>>> Do they provide any benefit over just using the right 
>>>>>>>>>>>>>>>> static
>>>>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in 
>>>>>>>>>>>>>>>> the KIP,
>>>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>>>> covariant typed function
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
>>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>>>> same as the inputs?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Look, if we want to keep `withChain`'s function optional,
>>>>>>>>>>>>>>>> then we must
>>>>>>>>>>>>>>>> keep the result key and value types the same. Because for
>>>>>>>>>>>>>>>> now, the
>>>>>>>>>>>>>>>> default value for the 'chain function' is 
>>>>>>>>>>>>>>>> Function.identity().
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Of course, we can make the 'chain function' required. But I
>>>>>>>>>>>>>>>> think this
>>>>>>>>>>>>>>>> is not what `split` method is for. `split` is for
>>>>>>>>>>>>>>>> splitting, not
>>>>>>>>>>>>>>>> transforming, and `chainFunction` in most of the cases 
>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>> be either
>>>>>>>>>>>>>>>> a consumer or the identity function.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past
>>>>>>>>>>>>>>>>> that generic type inference problem is to offer a static
>>>>>>>>>>>>>>>>> factory method that takes all the options (both name and
>>>>>>>>>>>>>>>>> function).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I’m happy with adding that option, although if we have all
>>>>>>>>>>>>>>>>> possible combinations available as static methods, then 
>>>>>>>>>>>>>>>>> I’d
>>>>>>>>>>>>>>>>> wonder whether we need the non-static builders (like
>>>>>>>>>>>>>>>>> withChain). Do they provide any benefit over just using 
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> right static factory?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function 
>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>> bounds works algebraically, but I’m not sure whether the
>>>>>>>>>>>>>>>>> limitations of Java type inference will actually let you
>>>>>>>>>>>>>>>>> pass in all the different functions you would want to.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I recall Bruno and I wrestling with this when he added
>>>>>>>>>>>>>>>>> Transform operators. In retrospect, this is a problem 
>>>>>>>>>>>>>>>>> we can
>>>>>>>>>>>>>>>>> iron out when we write tests in the PR. You might as well
>>>>>>>>>>>>>>>>> propose the “ideal” API in the KIP, which is the covariant
>>>>>>>>>>>>>>>>> typed function:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>>>>> KStream<? extends K, ? extends V>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. Actually, I have a new question about the types: Is it
>>>>>>>>>>>>>>>>> necessary to restrict the result key and value types to be
>>>>>>>>>>>>>>>>> the same as the inputs? I.e., shouldn’t the result 
>>>>>>>>>>>>>>>>> types be
>>>>>>>>>>>>>>>>> K1,V1?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>> Hello John,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         > Perhaps it would be better to stick with "as"
>>>>>>>>>>>>>>>>>> for now
>>>>>>>>>>>>>>>>>>         > and just file a Jira to switch them all at 
>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>> time [for
>>>>>>>>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin
>>>>>>>>>>>>>>>>>> they have a
>>>>>>>>>>>>>>>>>> standard workaround
>>>>>>>>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin). 
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> So actually this should be a very low priority issue, 
>>>>>>>>>>>>>>>>>> if an
>>>>>>>>>>>>>>>>>> issue at
>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         > I don't understand how your new proposed
>>>>>>>>>>>>>>>>>>         > methods would work any differently than the 
>>>>>>>>>>>>>>>>>> ones
>>>>>>>>>>>>>>>>>> you already
>>>>>>>>>>>>>>>>>>         > had proposed in the KIP. It seems like you'd 
>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>> have to provide
>>>>>>>>>>>>>>>>>>         > the generic type parameters on the first static
>>>>>>>>>>>>>>>>>> factory call. Can you
>>>>>>>>>>>>>>>>>>         > explain how your new interface proposal differs
>>>>>>>>>>>>>>>>>> from the existing KIP?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> In the KIP, I didn't clarify what methods should be 
>>>>>>>>>>>>>>>>>> static.
>>>>>>>>>>>>>>>>>> Now I
>>>>>>>>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> static: as(String), with(Function), with(Function, 
>>>>>>>>>>>>>>>>>> String).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The overloaded `with` version that provides both Function
>>>>>>>>>>>>>>>>>> and name can
>>>>>>>>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         > Regarding making the K,V types covariant 
>>>>>>>>>>>>>>>>>> also, yes,
>>>>>>>>>>>>>>>>>> that would indeed
>>>>>>>>>>>>>>>>>>         > be nice, but I'm not sure it will actually 
>>>>>>>>>>>>>>>>>> work.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> What I'm keeping in mind is the following example: 
>>>>>>>>>>>>>>>>>> imagine
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> static KStream<String, Integer> func(KStream<String,
>>>>>>>>>>>>>>>>>> Number> s) {
>>>>>>>>>>>>>>>>>>                 return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>>>>>>>>             s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>>>>>>>>                        //Won't compile!!
>>>>>>>>>>>>>>>>>>                        Branched.with(Me::func));
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The simple workaround here is to change `func`'s return
>>>>>>>>>>>>>>>>>> type from
>>>>>>>>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> [On the other hand, we already agreed to remove
>>>>>>>>>>>>>>>>>> `withJavaConsumer` from
>>>>>>>>>>>>>>>>>> `Branched`, so during code migration I will have to 
>>>>>>>>>>>>>>>>>> modify
>>>>>>>>>>>>>>>>>> my functions'
>>>>>>>>>>>>>>>>>> return types anyway -- I mean, from `void` to 
>>>>>>>>>>>>>>>>>> `KStream`!! ]
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         >  the map you're returning is Map<K,V>, and of
>>>>>>>>>>>>>>>>>> course a K is not the
>>>>>>>>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I think what you actually meant here is that KStream<?
>>>>>>>>>>>>>>>>>> extends K, ?
>>>>>>>>>>>>>>>>>> extends V> is not fit as a value for Map<String, 
>>>>>>>>>>>>>>>>>> KStream<K,
>>>>>>>>>>>>>>>>>> V>>. This
>>>>>>>>>>>>>>>>>> particularly is not a problem, since KStream<? extends 
>>>>>>>>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>>> extends V>
>>>>>>>>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be 
>>>>>>>>>>>>>>>>>> put
>>>>>>>>>>>>>>>>>> to the map.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard
>>>>>>>>>>>>>>>>>> types. So maybe
>>>>>>>>>>>>>>>>>> for now it's better to just admit that API is not
>>>>>>>>>>>>>>>>>> absolutely perfect and
>>>>>>>>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that
>>>>>>>>>>>>>>>>>>> "as" would
>>>>>>>>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart 
>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>> convention
>>>>>>>>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The convention is that "as(String name)" is used for the
>>>>>>>>>>>>>>>>>>> static factory
>>>>>>>>>>>>>>>>>>> method, whereas "withName(String name)" is an instance
>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to
>>>>>>>>>>>>>>>>>>> avoid "as"
>>>>>>>>>>>>>>>>>>> for compatibility with Kotlin, I might suggest
>>>>>>>>>>>>>>>>>>> "fromName(String name)",
>>>>>>>>>>>>>>>>>>> although it's somewhat dubious, since all the other
>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick 
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> "as" for now
>>>>>>>>>>>>>>>>>>> and just file a Jira to switch them all at the same 
>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>>>>>>>>> Regarding the type inference problem, yes, it's a 
>>>>>>>>>>>>>>>>>>> blemish
>>>>>>>>>>>>>>>>>>> on all of our
>>>>>>>>>>>>>>>>>>> configuraion objects. The problem is that Java infers 
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>> based on the _first_ method in the chain. While it does
>>>>>>>>>>>>>>>>>>> consider what
>>>>>>>>>>>>>>>>>>> the recipient of the method result wants, it only
>>>>>>>>>>>>>>>>>>> considers the _next_
>>>>>>>>>>>>>>>>>>> recipient.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it 
>>>>>>>>>>>>>>>>>>> to a
>>>>>>>>>>>>>>>>>>> Branched<String,String> variable, java infers the type
>>>>>>>>>>>>>>>>>>> correctly. But
>>>>>>>>>>>>>>>>>>> when the "next recipient" is a chained method call, like
>>>>>>>>>>>>>>>>>>> "withChain",
>>>>>>>>>>>>>>>>>>> then the chained method doesn't bound the type (by
>>>>>>>>>>>>>>>>>>> definition,
>>>>>>>>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so 
>>>>>>>>>>>>>>>>>>> Java
>>>>>>>>>>>>>>>>>>> will take
>>>>>>>>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be
>>>>>>>>>>>>>>>>>>> revised anymore.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, 
>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>> doesn't seem
>>>>>>>>>>>>>>>>>>> that hard to recursively consider the entire context 
>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>> inferring the
>>>>>>>>>>>>>>>>>>> generic type parameters, but this is what we have to 
>>>>>>>>>>>>>>>>>>> work
>>>>>>>>>>>>>>>>>>> with.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> To be honest, though, I don't understand how your new
>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>> methods would work any differently than the ones you
>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still 
>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>> the generic type parameters on the first static factory
>>>>>>>>>>>>>>>>>>> call. Can you
>>>>>>>>>>>>>>>>>>> explain how your new interface proposal differs from the
>>>>>>>>>>>>>>>>>>> existing KIP?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that
>>>>>>>>>>>>>>>>>>> would indeed
>>>>>>>>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You 
>>>>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>> want to give it a
>>>>>>>>>>>>>>>>>>> try. In the past, we've run into soe truly strange
>>>>>>>>>>>>>>>>>>> interactions between the
>>>>>>>>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner
>>>>>>>>>>>>>>>>>>> classes) in
>>>>>>>>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Another issue is that the value type of the map you're
>>>>>>>>>>>>>>>>>>> returning is
>>>>>>>>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? 
>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>> K", so it
>>>>>>>>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that
>>>>>>>>>>>>>>>>>>>> getting rid of
>>>>>>>>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as
>>>>>>>>>>>>>>>>>>>> possible, but not
>>>>>>>>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to
>>>>>>>>>>>>>>>>>>>> implement examples
>>>>>>>>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1. Having to return something from lambda is not a very
>>>>>>>>>>>>>>>>>>>> big deal.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use
>>>>>>>>>>>>>>>>>>>> method references
>>>>>>>>>>>>>>>>>>>> for already written stream consumers, but then I 
>>>>>>>>>>>>>>>>>>>> realized
>>>>>>>>>>>>>>>>>>>> that I can
>>>>>>>>>>>>>>>>>>>> just change my methods from returning void to returning
>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>> parameter and use references to them. Not very
>>>>>>>>>>>>>>>>>>>> convenient, but passable.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no
>>>>>>>>>>>>>>>>>>>> consumer 2) when
>>>>>>>>>>>>>>>>>>>> function returns null, we don't insert it into the
>>>>>>>>>>>>>>>>>>>> resulting map.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but
>>>>>>>>>>>>>>>>>>>> workable solution as
>>>>>>>>>>>>>>>>>>>> a first approximation. And later we can always add to
>>>>>>>>>>>>>>>>>>>> `Branched`
>>>>>>>>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes
>>>>>>>>>>>>>>>>>>>> like Branched
>>>>>>>>>>>>>>>>>>>> should be built? First of all, it seems that `as` 
>>>>>>>>>>>>>>>>>>>> now is
>>>>>>>>>>>>>>>>>>>> more preferred
>>>>>>>>>>>>>>>>>>>> than `withName` (although as you probably know it 
>>>>>>>>>>>>>>>>>>>> clashes
>>>>>>>>>>>>>>>>>>>> with Kotlin's
>>>>>>>>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out 
>>>>>>>>>>>>>>>>>>>> that my
>>>>>>>>>>>>>>>>>>>> Java cannot
>>>>>>>>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>>>             Branched.as("foo").withChain(s ->
>>>>>>>>>>>>>>>>>>>> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>>>             Branched.<String, 
>>>>>>>>>>>>>>>>>>>> String>as("foo").withChain(s
>>>>>>>>>>>>>>>>>>>> -> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of
>>>>>>>>>>>>>>>>>>>> `Branched` methods
>>>>>>>>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, 
>>>>>>>>>>>>>>>>>>>> V>,
>>>>>>>>>>>>>>>>>>>> ? extends
>>>>>>>>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 4. And one more. What do you think, do we need that
>>>>>>>>>>>>>>>>>>>> flexibility:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, 
>>>>>>>>>>>>>>>>>>>> V>>
>>>>>>>>>>>>>>>>>>>> chain
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? 
>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>> KStream<?
>>>>>>>>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ??
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be 
>>>>>>>>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>>>>>>>>> map, because there’s only one thing that could be 
>>>>>>>>>>>>>>>>>>>>> there,
>>>>>>>>>>>>>>>>>>>>> which is whatever is returned from the chain function.
>>>>>>>>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the 
>>>>>>>>>>>>>>>>>>>>> extensible way
>>>>>>>>>>>>>>>>>>>>> this KIP is designed, it would be trivially easy to 
>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>> the “terminal” chain later. As you say, fewer concepts
>>>>>>>>>>>>>>>>>>>>> leads to an API that is easier to learn.
>>>>>>>>>>>>>>>>>>>>> 3. We get to side-step the naming of this method.
>>>>>>>>>>>>>>>>>>>>> Although I didn’t complain about withJavaConsumer, it
>>>>>>>>>>>>>>>>>>>>> was only because I couldn’t think of a better name.
>>>>>>>>>>>>>>>>>>>>> Still, it’s somewhat unsatisfying to name a method 
>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>> its argument type, since this provides no 
>>>>>>>>>>>>>>>>>>>>> information at
>>>>>>>>>>>>>>>>>>>>> all about what the method does. I was willing to 
>>>>>>>>>>>>>>>>>>>>> accept
>>>>>>>>>>>>>>>>>>>>> it because I didn’t have an alternative, but I 
>>>>>>>>>>>>>>>>>>>>> would be
>>>>>>>>>>>>>>>>>>>>> happy to skip this method for now to avoid the problem
>>>>>>>>>>>>>>>>>>>>> until we have more inspiration.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The only con I see is that it makes the code a little
>>>>>>>>>>>>>>>>>>>>> less ergonomic to write when you don’t want to return
>>>>>>>>>>>>>>>>>>>>> the result of the chain (such as when the chain is
>>>>>>>>>>>>>>>>>>>>> terminal), since I’m your example, you have to 
>>>>>>>>>>>>>>>>>>>>> declare a
>>>>>>>>>>>>>>>>>>>>> block with a return statement at the end. It’s not
>>>>>>>>>>>>>>>>>>>>> ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with 
>>>>>>>>>>>>>>>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>> a null result, which would just remove the branch from
>>>>>>>>>>>>>>>>>>>>> the returned map. It seems nicer than forcing 
>>>>>>>>>>>>>>>>>>>>> people to
>>>>>>>>>>>>>>>>>>>>> pick a stream to return when their chain is 
>>>>>>>>>>>>>>>>>>>>> terminal and
>>>>>>>>>>>>>>>>>>>>> they don’t want to use the result later.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I guess the only open question is about
>>>>>>>>>>>>>>>>>>>>>> `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>           (1) split a stream and return the 
>>>>>>>>>>>>>>>>>>>>>> substreams
>>>>>>>>>>>>>>>>>>>>>> for futher processing
>>>>>>>>>>>>>>>>>>>>>>           (2) split a stream and modify the 
>>>>>>>>>>>>>>>>>>>>>> substreams
>>>>>>>>>>>>>>>>>>>>>> with in-place method chaining
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            -> split a stream, modify the 
>>>>>>>>>>>>>>>>>>>>>> substreams, and
>>>>>>>>>>>>>>>>>>>>>> return the _modified_
>>>>>>>>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>>>> split the topology graph at any point. Technically,
>>>>>>>>>>>>>>>>>>>>>>> it's OK to do both: feed the KStream to a
>>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>>>>> doesn't need the stream in the Map, one simply does
>>>>>>>>>>>>>>>>>>>>>>> not extract it from there
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> That is of course possible. However, it introduces 
>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>> "hidded" semantics:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>           - using `withChain` I get the modified
>>>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>>>           - using `withJavaConsumer` I get the 
>>>>>>>>>>>>>>>>>>>>>> unmodifed
>>>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>          From my understanding the original idea of
>>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` was to
>>>>>>>>>>>>>>>>>>>>>> model a terminal operation, ie, it should be 
>>>>>>>>>>>>>>>>>>>>>> similar to:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>>>            return null;
>>>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> However, I am not sure if we should even allow
>>>>>>>>>>>>>>>>>>>>>> `withChain()` to return
>>>>>>>>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this
>>>>>>>>>>>>>>>>>>>>>> case to avoid a `key
>>>>>>>>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Following this train of through, and if we want to
>>>>>>>>>>>>>>>>>>>>>> allow the "return
>>>>>>>>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>>>> that does not add
>>>>>>>>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Following your proposal, the semantics of
>>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` could also
>>>>>>>>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>>>            return s;
>>>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely
>>>>>>>>>>>>>>>>>>>>>> syntactic sugar,
>>>>>>>>>>>>>>>>>>>>>> while for the first proposal it adds new 
>>>>>>>>>>>>>>>>>>>>>> functionality
>>>>>>>>>>>>>>>>>>>>>> (if `return null`
>>>>>>>>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not 
>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>> "hide a
>>>>>>>>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need
>>>>>>>>>>>>>>>>>>>>>> to allow `return
>>>>>>>>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a
>>>>>>>>>>>>>>>>>>>>>> sub-stream in the Map.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we
>>>>>>>>>>>>>>>>>>>>>> follow your
>>>>>>>>>>>>>>>>>>>>>> proposal, I am wondering if we need 
>>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>>>> at all? Its
>>>>>>>>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API
>>>>>>>>>>>>>>>>>>>>>> is usually
>>>>>>>>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox,
>>>>>>>>>>>>>>>>>>>>>>> fortunately we have
>>>>>>>>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) 
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> existing ‘branch’
>>>>>>>>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration
>>>>>>>>>>>>>>>>>>>>>>> Plan" section.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to
>>>>>>>>>>>>>>>>>>>>>>>> branching directly
>>>>>>>>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic
>>>>>>>>>>>>>>>>>>>>>>> branching]
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic
>>>>>>>>>>>>>>>>>>>>>>> Branching' section.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka
>>>>>>>>>>>>>>>>>>>>>>>> Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be 
>>>>>>>>>>>>>>>>>>>>>>> confusing. I
>>>>>>>>>>>>>>>>>>>>>>> renamed this
>>>>>>>>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' 
>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use 
>>>>>>>>>>>>>>>>>>>>>>>> cases:
>>>>>>>>>>>>>>>>>>>>>>>> EITHER using
>>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`]
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' 
>>>>>>>>>>>>>>>>>>>>>>>> version of
>>>>>>>>>>>>>>>>>>>>>>>> the `branch`
>>>>>>>>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that
>>>>>>>>>>>>>>>>>>>>>>>> using both
>>>>>>>>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of
>>>>>>>>>>>>>>>>>>>>>>> `withChain()` will
>>>>>>>>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the
>>>>>>>>>>>>>>>>>>>>>>>> returned `Map` of
>>>>>>>>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a
>>>>>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>>>>>> implementation would be to not add the "branch" 
>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>> result map if
>>>>>>>>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>>>> split the topology
>>>>>>>>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both:
>>>>>>>>>>>>>>>>>>>>>>> feed the KStream to
>>>>>>>>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If 
>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>> doesn't need the
>>>>>>>>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it 
>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>> there :-)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the
>>>>>>>>>>>>>>>>>>>>>>> returned map
>>>>>>>>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs
>>>>>>>>>>>>>>>>>>>>>>> explicitly set by the
>>>>>>>>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids.
>>>>>>>>>>>>>>>>>>>>>>> Dealing with this
>>>>>>>>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing
>>>>>>>>>>>>>>>>>>>>>>> exclusions to this
>>>>>>>>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by
>>>>>>>>>>>>>>>>>>>>>>> provided
>>>>>>>>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated.
>>>>>>>>>>>>>>>>>>>>>>> What if a user
>>>>>>>>>>>>>>>>>>>>>>> provides both the name of the branch and a
>>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer? What do they
>>>>>>>>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or
>>>>>>>>>>>>>>>>>>>>>>> save it to the map?
>>>>>>>>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this 
>>>>>>>>>>>>>>>>>>>>>>> map, so
>>>>>>>>>>>>>>>>>>>>>>> maybe just leave
>>>>>>>>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg,
>>>>>>>>>>>>>>>>>>>>>>>> CoGroupedKStream,
>>>>>>>>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the 
>>>>>>>>>>>>>>>>>>>>>>>> "legacy"
>>>>>>>>>>>>>>>>>>>>>>>> `KGroupedStream`
>>>>>>>>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename 
>>>>>>>>>>>>>>>>>>>>>>>> without a
>>>>>>>>>>>>>>>>>>>>>>>> breaking change...
>>>>>>>>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have
>>>>>>>>>>>>>>>>>>>>>>>>> overloaded
>>>>>>>>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a 
>>>>>>>>>>>>>>>>>>>>>>>> `Predicate`
>>>>>>>>>>>>>>>>>>>>>>>> and assume you
>>>>>>>>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe
>>>>>>>>>>>>>>>>>>>>>>>> rephrase it accordingly
>>>>>>>>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public
>>>>>>>>>>>>>>>>>>>>>>>> Interface" section) of
>>>>>>>>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces) 
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction
>>>>>>>>>>>>>>>>>>>>>>>> of `withConsumer()`
>>>>>>>>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. 
>>>>>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>> related to John's
>>>>>>>>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use
>>>>>>>>>>>>>>>>>>>>>>>>> cases: EITHER using
>>>>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and
>>>>>>>>>>>>>>>>>>>>>>>> `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the
>>>>>>>>>>>>>>>>>>>>>>>> result of `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is 
>>>>>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>> with the `Consumer`
>>>>>>>>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and
>>>>>>>>>>>>>>>>>>>>>>>> `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to 
>>>>>>>>>>>>>>>>>>>>>>>> not add
>>>>>>>>>>>>>>>>>>>>>>>> the "branch" to
>>>>>>>>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As 
>>>>>>>>>>>>>>>>>>>>>>>> long as
>>>>>>>>>>>>>>>>>>>>>>>> we clearly document
>>>>>>>>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’
>>>>>>>>>>>>>>>>>>>>>>>>> confusing; I thought you
>>>>>>>>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface
>>>>>>>>>>>>>>>>>>>>>>>>> (which doesn’t make
>>>>>>>>>>>>>>>>>>>>>>>>> sense, of course). I get that you were 
>>>>>>>>>>>>>>>>>>>>>>>>> referring to
>>>>>>>>>>>>>>>>>>>>>>>>> the java Consumer
>>>>>>>>>>>>>>>>>>>>>>>>> interface, but we should still probably to to 
>>>>>>>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>>>> the ambiguity.
>>>>>>>>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about 
>>>>>>>>>>>>>>>>>>>>>>>>> ‘withSink’?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be
>>>>>>>>>>>>>>>>>>>>>>>> confused with a "sink
>>>>>>>>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less 
>>>>>>>>>>>>>>>>>>>>>>>> ambiguous?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I
>>>>>>>>>>>>>>>>>>>>>>>>> think it’s close,
>>>>>>>>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev 
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> will someone please take a look at the 
>>>>>>>>>>>>>>>>>>>>>>>>>> reworked KIP?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that now it follows design 
>>>>>>>>>>>>>>>>>>>>>>>>>> principles and
>>>>>>>>>>>>>>>>>>>>>>>>>> takes into account
>>>>>>>>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design 
>>>>>>>>>>>>>>>>>>>>>>>>>>> principles" and
>>>>>>>>>>>>>>>>>>>>>>>>>>> have completely
>>>>>>>>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> This version includes all the previous 
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>> results and follows
>>>>>>>>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V>
>>>>>>>>>>>>>>>>>>>>>>>>>>> branched)
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one
>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and 
>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>> need to provide one
>>>>>>>>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations
>>>>>>>>>>>>>>>>>>>>>>>>>>> we may use a single
>>>>>>>>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method
>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part 
>>>>>>>>>>>>>>>>>>>>>>>>>>> of a
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch, no
>>>>>>>>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.'
>>>>>>>>>>>>>>>>>>>>>>>>>>> is expected here
>>>>>>>>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single 
>>>>>>>>>>>>>>>>>>>>>>>>>>> parameter
>>>>>>>>>>>>>>>>>>>>>>>>>>> rule'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> have you back!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> already have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable to me to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>>>>>> principles" that we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> participation in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we invested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not feel entitled to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalized.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> writing and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original idea for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I gave up pushing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem with the scope of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .branch(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution in his post
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names seems to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your code brittle.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purposes. Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The idea seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trust
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> people who are more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status".
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about my proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worlds?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `String`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to merge both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your example.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added/removed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allow a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default name but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explicit to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that neither
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> independently of all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> taught us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that, although
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Branch, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> guess
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split(Named.withName("mysplit"))
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                           .branch(..., 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ...,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> syntax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operator(function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               operator(function, function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> string)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                interface BranchConfig 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noisier,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but it also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> decide we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tells us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domain,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sink
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Ivan, I’ll definitely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch(predicate, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumer)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 IMO the one trade 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> off to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 question. I don’t 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the same scope” 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 seems like a perfectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 when the branched 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the reasons Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution - working 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > On May 2, 2019, at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7:00
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Hello everyone, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thank you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Well, I don't 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 LinkedHashMap (no 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 matters) or `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 advantages than 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > In my opinion, the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal is that all 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 But 1) we rarely 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 workaround for the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 'Inlining the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method references 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 tend to split the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > The drawbacks are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> strong.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers is lost. We 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers in another. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misspell a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > What Michael propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 writing the API in Lua,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "dynamic naming" 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 beautiful. But in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 identifiers to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Do we have to invent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > And if we do, what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 all the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Earlier in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without "start 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the case when we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current KIP is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'clumsier'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 me address both 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 1) "Start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for it indeed) is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 2) No, dynamic 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Imagine a real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 value (say, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> RecordType).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*John:if we had 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 have been much 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > KBranchedStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*Not clumsy at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > for (RecordType
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >             
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 02.05.2019 14:40,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> I also agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> current `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> However, I also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> was more aligned 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `branch()` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statement and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> It makes the code 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `Predicates` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (that is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        ��        >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> An open question 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> specified. Atm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> and the call to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> (what is not the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> because users can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> in the result 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> About "inlining": So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> preference. I can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> that clearly make 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> On 5/1/19 6:26 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Perhaps inlining 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that a lambda with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the full
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 it can be a method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The advantage of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 (Consumer) together in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Ultimately the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch trees will be 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> On May 1, 2019, at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> I'm less 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enthusiastic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> functionality.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> harder to read 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    >>>>> Also +1 on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> great 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> framework for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Regarding the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> proposal in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(Consumer<KStream<K, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Obviously some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't work 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> associativity 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> directly 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> provides over 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> The KIP 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that it is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> certainly 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> case anyway, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> access your 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch, and it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solution on 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> top of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> I could also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> taken in, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Pros for that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - accessing 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - no double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - downstream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to read top to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - you can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> (KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Overall I'm 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> curious
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branched 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't need 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user.  I'm 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sort of in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <sophie@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I'd like to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> +1 what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> method, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> trying to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> alleviate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> able to cleanly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name->stream), which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> That said, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> anti-pattern or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> On Tue, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I’d like to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> are three 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 1. If you 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2. The way 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the ordering 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 3. It is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> brittle to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Using 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be a stronger
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> super K,? super
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Branches are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> given
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> mapping of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> because it’s a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> This solves 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2 because 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> interested 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> conditional by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> messing with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> One of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> awkward in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> On Tue, Apr 30,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <john@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> FWIW, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> confusing when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like a good 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> operator at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Tentatively, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> way, we don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> should return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> enforce 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> default 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> additionally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> though with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> On Fri, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 26,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> this is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The intend 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> IMHO, this 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> take any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> `branch()` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Because I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> might be 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> overlap that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Maybe 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> naming 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 'default' 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a method with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Bummer. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Can you 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> methods? 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP. For 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> You did not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -> KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branched-KStreams. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> nice to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> would need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> should 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "incomplete" to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched-KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 4/13/19 2:13
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I have 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Matthias, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I can see 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Totally 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> reserved 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> word,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> is not 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Absolutely! I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Dear 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> please 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> revise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Any new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 11.04.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> agrees 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> I had a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> quick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> are some 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> It's just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> In the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Also, we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Ie, we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> For the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> could we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> second 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> `Function`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Finally, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On 3/31/19
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> I'm a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bit of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> revise 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> buy-in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> could be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> idea 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> forward 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> deprecate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> same 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> just 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> little 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sloppy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> passing 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream but read from all 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On Fri, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I read 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> looks 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> going 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> What 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shall we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Why 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we go 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Maybe 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> added, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> able 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> The 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> worked 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> [Also, great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 28,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Hi 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> The 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a way 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> You 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> write in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consider
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> customers who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> store 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> This 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'brancher'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> class
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> classes'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*Alas,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Does 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> [BTW 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> KIP I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> so I will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tried to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512), 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeeded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> To be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> has 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Wed,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> OK, let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 1. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PROS: 1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 2. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> PROS:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Still,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) OK,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drop
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Well,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tests,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��      
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> Hello
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          �� >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> Does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    �� >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...).. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey()..... 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/6164 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 



Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi Matthias, what do you think? Shall I update the KIP with another 
naming scheme?

Regards, Ivan

05.06.2020 13:57, Ivan Ponomarev пишет:
>  > Can you maybe explain your reasoning? Why not just start with "0" and
>  > the default branch gets the next index as any other branch
> 
> My reasoning was the following: when we add a new branch, all the 
> following branches will be renumbered. If the default branch gets the 
> maximum index, its handler should be modified each time a new branch is 
> added. If the default branch always has "0" index, we'll have to modify 
> less code when adding a new branch to the end of the list.
> 
>  > that's a minor question.
> 
> I agree that it's a minor question, although maybe for a different 
> reason :-)
> 
> As a programmer, I don't believe in default branch naming, I'm not going
> to use it myself and I'm going to discourage others from using it.
> 
> The code relying on default naming scheme is going to be brittle. For 
> example, if we want to reorder branches, we will also have to 
> synchronously change the code that extracts branches from the Map for 
> handling, with lots of opportunities to make a mistake.
> 
> Low cohesion between branch predicates and branch handlers is one of the 
> problems of current branching API that this KIP was intended to fight.
> 
> And this is why my first goal was to avoid explicit contract for default 
> branch naming in KIP/JavaDoc.
> 
> Note that I agree that some form of default branch naming is needed. I 
> can imagine cases when I'm going to use the resulting Map without 
> explicit knowledge of its keys (e. g. operating on map.values() only).
> 
> So if for some reason you feel that other indexing scheme is more 
> preferable, I'm absolutely open for any proposals, because I don't think 
> that it is an important matter :-)
> 
> Regards,
> 
> Ivan
> 
> 05.06.2020 3:18, Matthias J. Sax пишет:
>> Thanks for updating the KIP!
>>
>> I am overall happy with it. The proposed default branch numbering (ie,
>> start with "1" and use "0" for the default branch) is not super
>> intuitive, but that's a minor question.
>>
>> Can you maybe explain your reasoning? Why not just start with "0" and
>> the default branch gets the next index as any other branch (ie, this is
>> how the currently returned array indexes the branches, too)?
>>
>>
>> -Matthias
>>
>> On 6/1/20 5:29 AM, Ivan Ponomarev wrote:
>>> Hi all,
>>>
>>> I updated the KIP with what we have discussed, see 'How the resulting
>>> Map is formed' section. Four bullet points to define the rules for Map
>>> keys, and four bullet points to define the rules for Map values.
>>>
>>> I also added the overloads to `Branched` in order to accept Consumers as
>>> parameters.
>>>
>>> Regards,
>>>
>>> Ivan
>>>
>>>
>>> 01.06.2020 8:56, Matthias J. Sax пишет:
>>>> For naming:
>>>>
>>>> (1) _If_ people specify a name for split() _and_ branch(), we _must_
>>>> have a pulic API contract. Otherwise it would be very bad user
>>>> experience if we are allowed to change the names (eg, do from "we don't
>>>> insert a `-` to we insert a `-`). -- We also need to clarify what
>>>> happens is only split() _or_ branch() has a name specified, ie, is 
>>>> there
>>>> a contract for this case or not?
>>>>
>>>> (2) If we have a contract for how split() and branch() names are put
>>>> together, we can also apply it to auto-generated names (this seems 
>>>> to be
>>>> just consistent).
>>>>
>>>> (3) Do we want to have a contract for auto-generated names? This is
>>>> debatable, but I personally don't see any harm in having a contract.
>>>> Also, if people want to access the Map, they can easily do it without
>>>> specifying names if there is a contract -- if there is no contract, we
>>>> force people to specify names to use the Map, what seems to make the 
>>>> API
>>>> harder to use.
>>>>
>>>> But we can of course discuss furhter on the dev list.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 5/30/20 3:53 PM, Ivan Ponomarev wrote:
>>>>> Hi Mathhias,
>>>>>
>>>>> I received your letter just after I sent mine concerning nulls and
>>>>> reintroducing Consumer overload. So, if you're fine with omitting 
>>>>> nulls
>>>>> and introducing a with(Consumer) overload, it's great!
>>>>>
>>>>> Concerning the naming algorithm. Well, actually what you propose is 
>>>>> very
>>>>> clear and this is what I was going to implement anyway. But are you 
>>>>> sure
>>>>> that we will never want to change this algorithm? Here I trust your
>>>>> judgement, since you understand the KStreams API's evolution much more
>>>>> than I do.
>>>>>
>>>>>
>>>>> 31.05.2020 0:50, Matthias J. Sax пишет:
>>>>>> Ivan, John,
>>>>>>
>>>>>> thanks for the details.
>>>>>>
>>>>>> I guess I can be convinced in either direction with regard to `null`
>>>>>> handling. But I am less worried about it, because I assume that a 
>>>>>> Java
>>>>>> programmer understand the ambiguty of `get(key) == null` -- that's 
>>>>>> why
>>>>>> there is `containsKey()`. In the end it might be a minor detail 
>>>>>> and as
>>>>>> long as it's well documented all discussed approaches seem fine.
>>>>>> Including the idea to re-introduce the `Consumer` overload.
>>>>>>
>>>>>> It's Ivan's KIP so I am fine with whatever he picks :)
>>>>>>
>>>>>>
>>>>>> About the naming: Not sure why we don't want to make the naming a
>>>>>> public
>>>>>> contract? I guess for split() we could default to the empty 
>>>>>> String, and
>>>>>> for `branch()` we could just add a counter by default?
>>>>>>
>>>>>> With no passed names at all, we just generte branche names "1", "2",
>>>>>> "3", etc... With only`split(Named.as("foo-"))` (but not branch
>>>>>> names) we
>>>>>> generate "foo-1", "foo-2", etc.
>>>>>>
>>>>>> If any branch has a dedicated name, the counting is preserved but 
>>>>>> just
>>>>>> not used int he name for this brach:
>>>>>>
>>>>>> split("foo-")
>>>>>>      .brach()
>>>>>>      .brach(Branched.as("bar")
>>>>>>      .defaultBranch()
>>>>>>
>>>>>> would generate "foo-1", "foo-bar", "foo-3"?
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 5/29/20 1:20 PM, John Roesler wrote:
>>>>>>> Hi all,
>>>>>>>
>>>>>>> Right, that was my suggestion; sorry for the confusion. I was 
>>>>>>> thinking
>>>>>>> that adding null as a value to maps is generally dubious, for which
>>>>>>> reason a lot of Map implementations actually don't allow it at all.
>>>>>>>
>>>>>>> The reason is that it creates ambiguity, since a lot of code treats
>>>>>>> "map.get(key) == null" as meaning the key is not in the map, but
>>>>>>> if we place a null value in the map (presuming we don't immediately
>>>>>>> get an exception), then you face weird contradictions, like
>>>>>>> get(key) == null, but containsKey(key) can be either true or false,
>>>>>>> and
>>>>>>> you may or may not see it while iterating. Which would violate the
>>>>>>> expectations of many Java programs. You can just take a look at
>>>>>>> the AK codebase, and you'll find many occurrences where we
>>>>>>> assume a null mapping means the map doesn't contain the key.
>>>>>>>
>>>>>>> Of course, the practical question, which Ivan brought up, is also
>>>>>>> a good one. What would you actually do with these null values
>>>>>>> in the map, besides get an NPE ?
>>>>>>>
>>>>>>> I'm not sure why I didn't think of this before, but an alternative
>>>>>>> to this debate is to go back to adding Consumer into the API, but
>>>>>>> with the same method name, withChain(Consumer<KStream<K,V>>).
>>>>>>>
>>>>>>> Now that we have only static methods, it's less confusing because 
>>>>>>> it's
>>>>>>> not possible to choose _both_ the Function<KStream,KStream> _and_
>>>>>>> the Consumer<KStream> in the same branch (which was one of the
>>>>>>> main drawbacks of the original plan wrt withChain and
>>>>>>> withJavaConsumer as builder methods.
>>>>>>>
>>>>>>> I.e., I was previously concerned about:
>>>>>>> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2) 
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> It's just a little confusing trying to reason about how this is all
>>>>>>> going to wire up.
>>>>>>>
>>>>>>> But with only the static methods, users have only three methods,
>>>>>>> and they are all clear:
>>>>>>> 1. Branched.as(name): just names the branch, the branch-predicated
>>>>>>> stream is the value of the map
>>>>>>> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the
>>>>>>> branch, applies the transformation after the branch predicate, and
>>>>>>> the transformed stream is the value of the map (null is not 
>>>>>>> permitted)
>>>>>>> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch,
>>>>>>> passes the predicated stream in to the Consumer function and omits
>>>>>>> the branch from the map
>>>>>>>
>>>>>>> Sorry for digging this option up again, but it's starting to look 
>>>>>>> more
>>>>>>> attractive after we dropped the builder functions, and I think it 
>>>>>>> also
>>>>>>> resolves the "null" concern.
>>>>>>>
>>>>>>> Thoughts?
>>>>>>> -John
>>>>>>>
>>>>>>>
>>>>>>> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>>>>>>>> Hi Matthias!
>>>>>>>>
>>>>>>>> Thanks for your reply!
>>>>>>>>
>>>>>>>> (1)
>>>>>>>>
>>>>>>>>     > Do you imply that there won't be a `name -> null` entry in 
>>>>>>>> the
>>>>>>>> `Map`
>>>>>>>> for the branch? If yes, I am wondering why?
>>>>>>>>
>>>>>>>> Ah, I re-read the discussion and found out that omitting null 
>>>>>>>> entries
>>>>>>>> was John's idea, not yours :-)
>>>>>>>>
>>>>>>>> Anyway. As you remember, I was against the 'swallowing' of
>>>>>>>> branches by
>>>>>>>> Consumer. I had a feeling that there are cases when I would like
>>>>>>>> both to
>>>>>>>> consume a branch and post-process it in a Map handler.
>>>>>>>>
>>>>>>>> But then the idea of getting rid of `withJavaConsumer` came up. And
>>>>>>>> then
>>>>>>>> we realized that we should handle nulls somehow.
>>>>>>>>
>>>>>>>> So it looked like taking the best from both approaches: if we 
>>>>>>>> want to
>>>>>>>> omit a branch in the resulting map, we just return null, thus
>>>>>>>> emulating
>>>>>>>> a 'swallowing consumer'. If we want to include a branch, we return
>>>>>>>> it.
>>>>>>>> It provides the full control for a user, and it spares the user 
>>>>>>>> from
>>>>>>>> null-checking in their code.
>>>>>>>>
>>>>>>>>     > My reasoning is, that it's very clear that `name -> null`
>>>>>>>> would be
>>>>>>>> there, because the user code did execute `return null`.
>>>>>>>>
>>>>>>>> My reasoning is following. The way such lambdas are written, it's
>>>>>>>> hard
>>>>>>>> to return null unintentionally. And if a user returns null
>>>>>>>> intentionally, what `name->null` in the resulting Map can be good
>>>>>>>> for?
>>>>>>>> It's either NPE (which is clearly not intended) or something that
>>>>>>>> should
>>>>>>>> be null-checked and skipped. Ok, let's do it!
>>>>>>>>
>>>>>>>> But, I'm ready to be convinced to drop this rule. It doesn't seem
>>>>>>>> to me
>>>>>>>> to be a principal thing.
>>>>>>>>
>>>>>>>> (2)
>>>>>>>>
>>>>>>>>     > I think it would be good if the KIP could explains the 
>>>>>>>> intended
>>>>>>>> default naming schema. This is important because the naming schema
>>>>>>>> must
>>>>>>>> be part of the API contract; otherwise, users cannot rely on the
>>>>>>>> naming
>>>>>>>> when trying to use the returned `Map`.
>>>>>>>>
>>>>>>>> Well, in fact, I intentionally didn't include any naming schema in
>>>>>>>> the
>>>>>>>> KIP. And it's done exactly for the purpose that a user won't be
>>>>>>>> able to
>>>>>>>> rely on default naming. If they want to use certain branches 
>>>>>>>> from the
>>>>>>>> Map, they will devise their own naming strategy that will never
>>>>>>>> change
>>>>>>>> and will never let them down.
>>>>>>>>
>>>>>>>> If they don't mind what's in a Map, if they are looking up the Map
>>>>>>>> just
>>>>>>>> for debugging, or if they want to treat a Map just like a
>>>>>>>> Collection of
>>>>>>>> values (a possible use case!), it doesn't matter how the branches
>>>>>>>> are named.
>>>>>>>>
>>>>>>>> This way we are giving ourselves the full freedom to change the
>>>>>>>> naming
>>>>>>>> schema afterwards without violating the API contract.
>>>>>>>>
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>
>>>>>>>>> Using covariant generics is a good idea! I am also fine with only
>>>>>>>>> using
>>>>>>>>> static method in `Branched` for now, as it's only two 
>>>>>>>>> parameters and
>>>>>>>>> thus not too many overloads.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> (1) What is unclear to me is, what you exaclty mean by:
>>>>>>>>>
>>>>>>>>>> If a function returns `null`, its result is omitted.
>>>>>>>>>
>>>>>>>>> Do you imply that there won't be a `name -> null` entry in the
>>>>>>>>> `Map` for
>>>>>>>>> the branch? If yes, I am wondering why?
>>>>>>>>>
>>>>>>>>> My previous argument was to omit an entry only for the
>>>>>>>>> `withJavaConsumer()` case, because the return type is "void" (and
>>>>>>>>> thus
>>>>>>>>> it's unclear what should be added and it would introduce
>>>>>>>>> inconsistencies
>>>>>>>>> in the `Map`).
>>>>>>>>>
>>>>>>>>> However, for a `Function` that returns a `KStream`, we could
>>>>>>>>> actually
>>>>>>>>> just add a `name -> null` entry if `null` is returned. My 
>>>>>>>>> reasoning
>>>>>>>>> is,
>>>>>>>>> that it's very clear that `name -> null` would be there, 
>>>>>>>>> because the
>>>>>>>>> user code did execute `return null`. Adding a `name -> null`
>>>>>>>>> entry for
>>>>>>>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>>>>>>>> exceptional case for handling `null`).
>>>>>>>>>
>>>>>>>>> Thoughts?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> (2) For `Branched.as()` the KIP says:
>>>>>>>>>
>>>>>>>>>> sets the name of the branch (auto-generated by default, when 
>>>>>>>>>> split
>>>>>>>>>> operation is named, then the names are index-suffixed)
>>>>>>>>>
>>>>>>>>> I think it would be good if the KIP could explains the intended
>>>>>>>>> default
>>>>>>>>> naming schema. This is important because the naming schema must be
>>>>>>>>> part
>>>>>>>>> of the API contract; otherwise, users cannot rely on the naming 
>>>>>>>>> when
>>>>>>>>> trying to use the returned `Map`.
>>>>>>>>>
>>>>>>>>> Similalry, it should be explained how names are generated if
>>>>>>>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations
>>>>>>>>> how the
>>>>>>>>> API can be mixed and matched and it's unclear atm how it would
>>>>>>>>> work in
>>>>>>>>> detail.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thanks!
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>>>>>>>> Hi Ivan,
>>>>>>>>>>
>>>>>>>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>>>>>>>> that have been raised in the discussion so far have been
>>>>>>>>>> addressed. And it's been a while since anyone raised a new
>>>>>>>>>> concern. At this point, it seems like a good time to start
>>>>>>>>>> the VOTE thread.
>>>>>>>>>>
>>>>>>>>>> Sometimes, the vote thread will trigger new people to look
>>>>>>>>>> into the KIP, and they may raise new concerns, but it's not a
>>>>>>>>>> problem. We'll just address those lingering concerns if there
>>>>>>>>>> are any, until you have all the votes you need.
>>>>>>>>>>
>>>>>>>>>> Thanks again for the contribution!
>>>>>>>>>>
>>>>>>>>>> -John
>>>>>>>>>>
>>>>>>>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>>>>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>>>>>>>
>>>>>>>>>>> 1. Per Mathhias's suggestion, if a chain function returns null,
>>>>>>>>>>> the
>>>>>>>>>>> respective result if omitted in the resulting Map.
>>>>>>>>>>>
>>>>>>>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>>>>>>>
>>>>>>>>>>> 3. `Branched` class has only three static methods with all the
>>>>>>>>>>> possible
>>>>>>>>>>> combinations of parameters.
>>>>>>>>>>>
>>>>>>>>>>> 4. Chain function is defined 'fully covariant', let's see if we
>>>>>>>>>>> can
>>>>>>>>>>> implement it this way :-))
>>>>>>>>>>>
>>>>>>>>>>> + code example updates and minor edits.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Since this is my first KIP, I'm not sure what should I do 
>>>>>>>>>>> next. I
>>>>>>>>>>> feel
>>>>>>>>>>> that we talked over all the details and the consensus is 
>>>>>>>>>>> reached.
>>>>>>>>>>> Is it
>>>>>>>>>>> OK to call for VOTE now or is it better to wait for more 
>>>>>>>>>>> feedback?
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>>>>>>>> Sounds good to me, Ivan!
>>>>>>>>>>>> -John
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>>>>>>>> John,
>>>>>>>>>>>>>
>>>>>>>>>>>>> ---------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>>       > I'm sorry things have been dragging out a little, 
>>>>>>>>>>>>> but I
>>>>>>>>>>>>> have the
>>>>>>>>>>>>> sense we're very close to the end of this discussion, which is
>>>>>>>>>>>>> exciting.
>>>>>>>>>>>>>
>>>>>>>>>>>>> We are certainly moving forward!  And I'm not in a hurry at
>>>>>>>>>>>>> all. As I
>>>>>>>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>>>>>>>> KafkaStreamBrancher -- the implementation of the first,
>>>>>>>>>>>>> rejected version
>>>>>>>>>>>>> of this KIP. It's inferior to what we are discussing here, but
>>>>>>>>>>>>> it does
>>>>>>>>>>>>> its work. So it's worth to design this KIP really, really 
>>>>>>>>>>>>> well.
>>>>>>>>>>>>> And by
>>>>>>>>>>>>> the way, from this discussion I'm learning the good API
>>>>>>>>>>>>> designing
>>>>>>>>>>>>> process. For me it has a value per se :-))
>>>>>>>>>>>>>
>>>>>>>>>>>>> ----------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>>       >>   > I'd wonder whether we need the non-static 
>>>>>>>>>>>>> builders
>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>       >>   > Do they provide any benefit over just using the
>>>>>>>>>>>>> right static
>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>       >
>>>>>>>>>>>>>       > I don't have a strong feeling, either. It seems 
>>>>>>>>>>>>> nice to
>>>>>>>>>>>>> offer a better
>>>>>>>>>>>>>       > type inference experience than what we get with
>>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>>       > offering the static method that takes both name and
>>>>>>>>>>>>> chain.
>>>>>>>>>>>>>       > Given that, there doesn't seem to be a good reason to
>>>>>>>>>>>>> also offer the
>>>>>>>>>>>>>       > non-static builder-style methods, so I guess I'd 
>>>>>>>>>>>>> prefer
>>>>>>>>>>>>> to drop them.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I
>>>>>>>>>>>>> found out
>>>>>>>>>>>>> an interesting fact about type inference rules in Java. Funny
>>>>>>>>>>>>> thing is
>>>>>>>>>>>>> that although we need to explicitly set types in a chain like
>>>>>>>>>>>>> this
>>>>>>>>>>>>>
>>>>>>>>>>>>> foo.branch(..., 
>>>>>>>>>>>>> Branched.<...,...>named("foo").withChain(...));
>>>>>>>>>>>>>
>>>>>>>>>>>>> (otherwise it won't compile), the composition of static method
>>>>>>>>>>>>> calls
>>>>>>>>>>>>> works just fine, all the types are being calculated correctly:
>>>>>>>>>>>>>
>>>>>>>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), 
>>>>>>>>>>>>> ...));
>>>>>>>>>>>>>
>>>>>>>>>>>>> As I was told, for type inference there is difference between
>>>>>>>>>>>>> qualifiers
>>>>>>>>>>>>> and arguments, 'you go up if you are argument but stop if you
>>>>>>>>>>>>> are
>>>>>>>>>>>>> qualifier'. And it also seems that we should not bet on any
>>>>>>>>>>>>> future
>>>>>>>>>>>>> improvements in Java type inference here.
>>>>>>>>>>>>>
>>>>>>>>>>>>> So,
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1) I think we that in this KIP we should provide three static
>>>>>>>>>>>>> methods
>>>>>>>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function,
>>>>>>>>>>>>> String)`, and
>>>>>>>>>>>>> drop any non-static ones.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2) If anything else will be ever needed, we can easily add
>>>>>>>>>>>>> anything.
>>>>>>>>>>>>> Maybe this can be done in a process of refinement of all the
>>>>>>>>>>>>> parameter
>>>>>>>>>>>>> classes.
>>>>>>>>>>>>>
>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>>       > we may as well hope for the best, and propose the 
>>>>>>>>>>>>> "fully
>>>>>>>>>>>>>       > covariant" definition for now.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>>       >> Good question, I already thought about it and 
>>>>>>>>>>>>> rejected
>>>>>>>>>>>>> the idea....
>>>>>>>>>>>>>       > (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>>>       >  "Worst" case scenario: someone
>>>>>>>>>>>>>       > else will wish the return type is something different,
>>>>>>>>>>>>> and we'll go
>>>>>>>>>>>>>       > through a painless deprecation transition to change it
>>>>>>>>>>>>> later.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Of course, we cant' predict all the ways people are going to
>>>>>>>>>>>>> use it.
>>>>>>>>>>>>>       From my own humble experience with Kafka Streams, the 
>>>>>>>>>>>>> worst
>>>>>>>>>>>>> scenario is
>>>>>>>>>>>>> unlikely. Split is split, transform is transform, too much
>>>>>>>>>>>>> flexibility
>>>>>>>>>>>>> is often evil.
>>>>>>>>>>>>>
>>>>>>>>>>>>> ------------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>> So it seems that we are close to the consensus. Two things to
>>>>>>>>>>>>> be altered
>>>>>>>>>>>>> in the current version of KIP:
>>>>>>>>>>>>>
>>>>>>>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Any ideas / questions / objections?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm sorry things have been dragging out a little, but I have
>>>>>>>>>>>>>> the sense
>>>>>>>>>>>>>> we're very close to the end of this discussion, which is
>>>>>>>>>>>>>> exciting.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>>        > Do they provide any benefit over just using the 
>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>> static factory?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I don't have a strong feeling, either. It seems nice to offer
>>>>>>>>>>>>>> a better
>>>>>>>>>>>>>> type inference experience than what we get with
>>>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Given that, there doesn't seem to be a good reason to also
>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>> non-static builder-style methods, so I guess I'd prefer to
>>>>>>>>>>>>>> drop them.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'll defer to Matthias, if he has a chance to consider 
>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>> it's more
>>>>>>>>>>>>>> valuable to stick with the existing pattern or break the
>>>>>>>>>>>>>> pattern to offer
>>>>>>>>>>>>>> a better experience.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in the 
>>>>>>>>>>>>>>> KIP,
>>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>>>        > covariant typed function
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been 
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>> we just
>>>>>>>>>>>>>> won't know what works until we really try it, and try it in
>>>>>>>>>>>>>> several different
>>>>>>>>>>>>>> ways. Still, we may as well hope for the best, and propose 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> "fully
>>>>>>>>>>>>>> covariant" definition for now.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>>>        > same as the inputs?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>>> idea....
>>>>>>>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That's fair! It's your KIP, after all. I think I might have
>>>>>>>>>>>>>> made a different
>>>>>>>>>>>>>> call here, but I think this choice is fine. "Worst" case
>>>>>>>>>>>>>> scenario: someone
>>>>>>>>>>>>>> else will wish the return type is something different, and
>>>>>>>>>>>>>> we'll go
>>>>>>>>>>>>>> through a painless deprecation transition to change it later.
>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>> to the clean design of your API, this doesn't seem to bad.
>>>>>>>>>>>>>> And, of
>>>>>>>>>>>>>> course, you've actually been using similar functionality
>>>>>>>>>>>>>> already, so it
>>>>>>>>>>>>>> seems we should trust your intuition.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>> Do they provide any benefit over just using the right static
>>>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in the 
>>>>>>>>>>>>>>> KIP,
>>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>>> covariant typed function
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you 
>>>>>>>>>>>>>>> propose the
>>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and 
>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>>> same as the inputs?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Look, if we want to keep `withChain`'s function optional,
>>>>>>>>>>>>>>> then we must
>>>>>>>>>>>>>>> keep the result key and value types the same. Because for
>>>>>>>>>>>>>>> now, the
>>>>>>>>>>>>>>> default value for the 'chain function' is 
>>>>>>>>>>>>>>> Function.identity().
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Of course, we can make the 'chain function' required. But I
>>>>>>>>>>>>>>> think this
>>>>>>>>>>>>>>> is not what `split` method is for. `split` is for
>>>>>>>>>>>>>>> splitting, not
>>>>>>>>>>>>>>> transforming, and `chainFunction` in most of the cases 
>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>> be either
>>>>>>>>>>>>>>> a consumer or the identity function.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past
>>>>>>>>>>>>>>>> that generic type inference problem is to offer a static
>>>>>>>>>>>>>>>> factory method that takes all the options (both name and
>>>>>>>>>>>>>>>> function).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I’m happy with adding that option, although if we have all
>>>>>>>>>>>>>>>> possible combinations available as static methods, then I’d
>>>>>>>>>>>>>>>> wonder whether we need the non-static builders (like
>>>>>>>>>>>>>>>> withChain). Do they provide any benefit over just using the
>>>>>>>>>>>>>>>> right static factory?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type
>>>>>>>>>>>>>>>> bounds works algebraically, but I’m not sure whether the
>>>>>>>>>>>>>>>> limitations of Java type inference will actually let you
>>>>>>>>>>>>>>>> pass in all the different functions you would want to.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I recall Bruno and I wrestling with this when he added
>>>>>>>>>>>>>>>> Transform operators. In retrospect, this is a problem we 
>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>> iron out when we write tests in the PR. You might as well
>>>>>>>>>>>>>>>> propose the “ideal” API in the KIP, which is the covariant
>>>>>>>>>>>>>>>> typed function:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>>>> KStream<? extends K, ? extends V>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 3. Actually, I have a new question about the types: Is it
>>>>>>>>>>>>>>>> necessary to restrict the result key and value types to be
>>>>>>>>>>>>>>>> the same as the inputs? I.e., shouldn’t the result types be
>>>>>>>>>>>>>>>> K1,V1?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>> Hello John,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         > Perhaps it would be better to stick with "as"
>>>>>>>>>>>>>>>>> for now
>>>>>>>>>>>>>>>>>         > and just file a Jira to switch them all at 
>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>> time [for
>>>>>>>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin
>>>>>>>>>>>>>>>>> they have a
>>>>>>>>>>>>>>>>> standard workaround
>>>>>>>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin). 
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> So actually this should be a very low priority issue, 
>>>>>>>>>>>>>>>>> if an
>>>>>>>>>>>>>>>>> issue at
>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         > I don't understand how your new proposed
>>>>>>>>>>>>>>>>>         > methods would work any differently than the ones
>>>>>>>>>>>>>>>>> you already
>>>>>>>>>>>>>>>>>         > had proposed in the KIP. It seems like you'd 
>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>> have to provide
>>>>>>>>>>>>>>>>>         > the generic type parameters on the first static
>>>>>>>>>>>>>>>>> factory call. Can you
>>>>>>>>>>>>>>>>>         > explain how your new interface proposal differs
>>>>>>>>>>>>>>>>> from the existing KIP?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> In the KIP, I didn't clarify what methods should be 
>>>>>>>>>>>>>>>>> static.
>>>>>>>>>>>>>>>>> Now I
>>>>>>>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> static: as(String), with(Function), with(Function, 
>>>>>>>>>>>>>>>>> String).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The overloaded `with` version that provides both Function
>>>>>>>>>>>>>>>>> and name can
>>>>>>>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         > Regarding making the K,V types covariant 
>>>>>>>>>>>>>>>>> also, yes,
>>>>>>>>>>>>>>>>> that would indeed
>>>>>>>>>>>>>>>>>         > be nice, but I'm not sure it will actually work.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> static KStream<String, Integer> func(KStream<String,
>>>>>>>>>>>>>>>>> Number> s) {
>>>>>>>>>>>>>>>>>                 return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>>>>>>>             s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>>>>>>>                        //Won't compile!!
>>>>>>>>>>>>>>>>>                        Branched.with(Me::func));
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The simple workaround here is to change `func`'s return
>>>>>>>>>>>>>>>>> type from
>>>>>>>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> [On the other hand, we already agreed to remove
>>>>>>>>>>>>>>>>> `withJavaConsumer` from
>>>>>>>>>>>>>>>>> `Branched`, so during code migration I will have to modify
>>>>>>>>>>>>>>>>> my functions'
>>>>>>>>>>>>>>>>> return types anyway -- I mean, from `void` to 
>>>>>>>>>>>>>>>>> `KStream`!! ]
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         >  the map you're returning is Map<K,V>, and of
>>>>>>>>>>>>>>>>> course a K is not the
>>>>>>>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I think what you actually meant here is that KStream<?
>>>>>>>>>>>>>>>>> extends K, ?
>>>>>>>>>>>>>>>>> extends V> is not fit as a value for Map<String, 
>>>>>>>>>>>>>>>>> KStream<K,
>>>>>>>>>>>>>>>>> V>>. This
>>>>>>>>>>>>>>>>> particularly is not a problem, since KStream<? extends 
>>>>>>>>>>>>>>>>> K, ?
>>>>>>>>>>>>>>>>> extends V>
>>>>>>>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put
>>>>>>>>>>>>>>>>> to the map.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard
>>>>>>>>>>>>>>>>> types. So maybe
>>>>>>>>>>>>>>>>> for now it's better to just admit that API is not
>>>>>>>>>>>>>>>>> absolutely perfect and
>>>>>>>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that
>>>>>>>>>>>>>>>>>> "as" would
>>>>>>>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from
>>>>>>>>>>>>>>>>>> convention
>>>>>>>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The convention is that "as(String name)" is used for the
>>>>>>>>>>>>>>>>>> static factory
>>>>>>>>>>>>>>>>>> method, whereas "withName(String name)" is an instance
>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to
>>>>>>>>>>>>>>>>>> avoid "as"
>>>>>>>>>>>>>>>>>> for compatibility with Kotlin, I might suggest
>>>>>>>>>>>>>>>>>> "fromName(String name)",
>>>>>>>>>>>>>>>>>> although it's somewhat dubious, since all the other
>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick 
>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> "as" for now
>>>>>>>>>>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>>>>>>>> Regarding the type inference problem, yes, it's a blemish
>>>>>>>>>>>>>>>>>> on all of our
>>>>>>>>>>>>>>>>>> configuraion objects. The problem is that Java infers the
>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>> based on the _first_ method in the chain. While it does
>>>>>>>>>>>>>>>>>> consider what
>>>>>>>>>>>>>>>>>> the recipient of the method result wants, it only
>>>>>>>>>>>>>>>>>> considers the _next_
>>>>>>>>>>>>>>>>>> recipient.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it 
>>>>>>>>>>>>>>>>>> to a
>>>>>>>>>>>>>>>>>> Branched<String,String> variable, java infers the type
>>>>>>>>>>>>>>>>>> correctly. But
>>>>>>>>>>>>>>>>>> when the "next recipient" is a chained method call, like
>>>>>>>>>>>>>>>>>> "withChain",
>>>>>>>>>>>>>>>>>> then the chained method doesn't bound the type (by
>>>>>>>>>>>>>>>>>> definition,
>>>>>>>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java
>>>>>>>>>>>>>>>>>> will take
>>>>>>>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be
>>>>>>>>>>>>>>>>>> revised anymore.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, since it
>>>>>>>>>>>>>>>>>> doesn't seem
>>>>>>>>>>>>>>>>>> that hard to recursively consider the entire context when
>>>>>>>>>>>>>>>>>> inferring the
>>>>>>>>>>>>>>>>>> generic type parameters, but this is what we have to work
>>>>>>>>>>>>>>>>>> with.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> To be honest, though, I don't understand how your new
>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>> methods would work any differently than the ones you
>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still 
>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>> the generic type parameters on the first static factory
>>>>>>>>>>>>>>>>>> call. Can you
>>>>>>>>>>>>>>>>>> explain how your new interface proposal differs from the
>>>>>>>>>>>>>>>>>> existing KIP?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that
>>>>>>>>>>>>>>>>>> would indeed
>>>>>>>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You 
>>>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>> want to give it a
>>>>>>>>>>>>>>>>>> try. In the past, we've run into soe truly strange
>>>>>>>>>>>>>>>>>> interactions between the
>>>>>>>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner
>>>>>>>>>>>>>>>>>> classes) in
>>>>>>>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Another issue is that the value type of the map you're
>>>>>>>>>>>>>>>>>> returning is
>>>>>>>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends
>>>>>>>>>>>>>>>>>> K", so it
>>>>>>>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that
>>>>>>>>>>>>>>>>>>> getting rid of
>>>>>>>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as
>>>>>>>>>>>>>>>>>>> possible, but not
>>>>>>>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to
>>>>>>>>>>>>>>>>>>> implement examples
>>>>>>>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 1. Having to return something from lambda is not a very
>>>>>>>>>>>>>>>>>>> big deal.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use
>>>>>>>>>>>>>>>>>>> method references
>>>>>>>>>>>>>>>>>>> for already written stream consumers, but then I 
>>>>>>>>>>>>>>>>>>> realized
>>>>>>>>>>>>>>>>>>> that I can
>>>>>>>>>>>>>>>>>>> just change my methods from returning void to returning
>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>> parameter and use references to them. Not very
>>>>>>>>>>>>>>>>>>> convenient, but passable.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no
>>>>>>>>>>>>>>>>>>> consumer 2) when
>>>>>>>>>>>>>>>>>>> function returns null, we don't insert it into the
>>>>>>>>>>>>>>>>>>> resulting map.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but
>>>>>>>>>>>>>>>>>>> workable solution as
>>>>>>>>>>>>>>>>>>> a first approximation. And later we can always add to
>>>>>>>>>>>>>>>>>>> `Branched`
>>>>>>>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes
>>>>>>>>>>>>>>>>>>> like Branched
>>>>>>>>>>>>>>>>>>> should be built? First of all, it seems that `as` now is
>>>>>>>>>>>>>>>>>>> more preferred
>>>>>>>>>>>>>>>>>>> than `withName` (although as you probably know it 
>>>>>>>>>>>>>>>>>>> clashes
>>>>>>>>>>>>>>>>>>> with Kotlin's
>>>>>>>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out that my
>>>>>>>>>>>>>>>>>>> Java cannot
>>>>>>>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>>             Branched.as("foo").withChain(s ->
>>>>>>>>>>>>>>>>>>> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>>             Branched.<String, 
>>>>>>>>>>>>>>>>>>> String>as("foo").withChain(s
>>>>>>>>>>>>>>>>>>> -> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of
>>>>>>>>>>>>>>>>>>> `Branched` methods
>>>>>>>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>,
>>>>>>>>>>>>>>>>>>> ? extends
>>>>>>>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 4. And one more. What do you think, do we need that
>>>>>>>>>>>>>>>>>>> flexibility:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>>>>> chain
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? 
>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>> KStream<?
>>>>>>>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ??
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in 
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> map, because there’s only one thing that could be 
>>>>>>>>>>>>>>>>>>>> there,
>>>>>>>>>>>>>>>>>>>> which is whatever is returned from the chain function.
>>>>>>>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible 
>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>> this KIP is designed, it would be trivially easy to add
>>>>>>>>>>>>>>>>>>>> the “terminal” chain later. As you say, fewer concepts
>>>>>>>>>>>>>>>>>>>> leads to an API that is easier to learn.
>>>>>>>>>>>>>>>>>>>> 3. We get to side-step the naming of this method.
>>>>>>>>>>>>>>>>>>>> Although I didn’t complain about withJavaConsumer, it
>>>>>>>>>>>>>>>>>>>> was only because I couldn’t think of a better name.
>>>>>>>>>>>>>>>>>>>> Still, it’s somewhat unsatisfying to name a method 
>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>> its argument type, since this provides no 
>>>>>>>>>>>>>>>>>>>> information at
>>>>>>>>>>>>>>>>>>>> all about what the method does. I was willing to accept
>>>>>>>>>>>>>>>>>>>> it because I didn’t have an alternative, but I would be
>>>>>>>>>>>>>>>>>>>> happy to skip this method for now to avoid the problem
>>>>>>>>>>>>>>>>>>>> until we have more inspiration.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The only con I see is that it makes the code a little
>>>>>>>>>>>>>>>>>>>> less ergonomic to write when you don’t want to return
>>>>>>>>>>>>>>>>>>>> the result of the chain (such as when the chain is
>>>>>>>>>>>>>>>>>>>> terminal), since I’m your example, you have to 
>>>>>>>>>>>>>>>>>>>> declare a
>>>>>>>>>>>>>>>>>>>> block with a return statement at the end. It’s not
>>>>>>>>>>>>>>>>>>>> ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing
>>>>>>>>>>>>>>>>>>>> a null result, which would just remove the branch from
>>>>>>>>>>>>>>>>>>>> the returned map. It seems nicer than forcing people to
>>>>>>>>>>>>>>>>>>>> pick a stream to return when their chain is terminal 
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> they don’t want to use the result later.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I guess the only open question is about
>>>>>>>>>>>>>>>>>>>>> `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>           (1) split a stream and return the substreams
>>>>>>>>>>>>>>>>>>>>> for futher processing
>>>>>>>>>>>>>>>>>>>>>           (2) split a stream and modify the substreams
>>>>>>>>>>>>>>>>>>>>> with in-place method chaining
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>            -> split a stream, modify the 
>>>>>>>>>>>>>>>>>>>>> substreams, and
>>>>>>>>>>>>>>>>>>>>> return the _modified_
>>>>>>>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>>> split the topology graph at any point. Technically,
>>>>>>>>>>>>>>>>>>>>>> it's OK to do both: feed the KStream to a
>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>>>> doesn't need the stream in the Map, one simply does
>>>>>>>>>>>>>>>>>>>>>> not extract it from there
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> That is of course possible. However, it introduces 
>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>> "hidded" semantics:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>           - using `withChain` I get the modified
>>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>>           - using `withJavaConsumer` I get the 
>>>>>>>>>>>>>>>>>>>>> unmodifed
>>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>          From my understanding the original idea of
>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` was to
>>>>>>>>>>>>>>>>>>>>> model a terminal operation, ie, it should be 
>>>>>>>>>>>>>>>>>>>>> similar to:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>>            return null;
>>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> However, I am not sure if we should even allow
>>>>>>>>>>>>>>>>>>>>> `withChain()` to return
>>>>>>>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this
>>>>>>>>>>>>>>>>>>>>> case to avoid a `key
>>>>>>>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Following this train of through, and if we want to
>>>>>>>>>>>>>>>>>>>>> allow the "return
>>>>>>>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>>> that does not add
>>>>>>>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Following your proposal, the semantics of
>>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` could also
>>>>>>>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>>            return s;
>>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely
>>>>>>>>>>>>>>>>>>>>> syntactic sugar,
>>>>>>>>>>>>>>>>>>>>> while for the first proposal it adds new functionality
>>>>>>>>>>>>>>>>>>>>> (if `return null`
>>>>>>>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to
>>>>>>>>>>>>>>>>>>>>> "hide a
>>>>>>>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need
>>>>>>>>>>>>>>>>>>>>> to allow `return
>>>>>>>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a
>>>>>>>>>>>>>>>>>>>>> sub-stream in the Map.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we
>>>>>>>>>>>>>>>>>>>>> follow your
>>>>>>>>>>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>>> at all? Its
>>>>>>>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API
>>>>>>>>>>>>>>>>>>>>> is usually
>>>>>>>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox,
>>>>>>>>>>>>>>>>>>>>>> fortunately we have
>>>>>>>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the
>>>>>>>>>>>>>>>>>>>>>>> existing ‘branch’
>>>>>>>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration
>>>>>>>>>>>>>>>>>>>>>> Plan" section.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to
>>>>>>>>>>>>>>>>>>>>>>> branching directly
>>>>>>>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic
>>>>>>>>>>>>>>>>>>>>>> branching]
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic
>>>>>>>>>>>>>>>>>>>>>> Branching' section.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka
>>>>>>>>>>>>>>>>>>>>>>> Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be 
>>>>>>>>>>>>>>>>>>>>>> confusing. I
>>>>>>>>>>>>>>>>>>>>>> renamed this
>>>>>>>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' 
>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use 
>>>>>>>>>>>>>>>>>>>>>>> cases:
>>>>>>>>>>>>>>>>>>>>>>> EITHER using
>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`]
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' 
>>>>>>>>>>>>>>>>>>>>>>> version of
>>>>>>>>>>>>>>>>>>>>>>> the `branch`
>>>>>>>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that
>>>>>>>>>>>>>>>>>>>>>>> using both
>>>>>>>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of
>>>>>>>>>>>>>>>>>>>>>> `withChain()` will
>>>>>>>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the
>>>>>>>>>>>>>>>>>>>>>>> returned `Map` of
>>>>>>>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a
>>>>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>>>>> implementation would be to not add the "branch" to 
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> result map if
>>>>>>>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>>> split the topology
>>>>>>>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both:
>>>>>>>>>>>>>>>>>>>>>> feed the KStream to
>>>>>>>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>>>> doesn't need the
>>>>>>>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it 
>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>> there :-)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the
>>>>>>>>>>>>>>>>>>>>>> returned map
>>>>>>>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs
>>>>>>>>>>>>>>>>>>>>>> explicitly set by the
>>>>>>>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids.
>>>>>>>>>>>>>>>>>>>>>> Dealing with this
>>>>>>>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing
>>>>>>>>>>>>>>>>>>>>>> exclusions to this
>>>>>>>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by
>>>>>>>>>>>>>>>>>>>>>> provided
>>>>>>>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated.
>>>>>>>>>>>>>>>>>>>>>> What if a user
>>>>>>>>>>>>>>>>>>>>>> provides both the name of the branch and a
>>>>>>>>>>>>>>>>>>>>>> [Java]Consumer? What do they
>>>>>>>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or
>>>>>>>>>>>>>>>>>>>>>> save it to the map?
>>>>>>>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this 
>>>>>>>>>>>>>>>>>>>>>> map, so
>>>>>>>>>>>>>>>>>>>>>> maybe just leave
>>>>>>>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg,
>>>>>>>>>>>>>>>>>>>>>>> CoGroupedKStream,
>>>>>>>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the 
>>>>>>>>>>>>>>>>>>>>>>> "legacy"
>>>>>>>>>>>>>>>>>>>>>>> `KGroupedStream`
>>>>>>>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a
>>>>>>>>>>>>>>>>>>>>>>> breaking change...
>>>>>>>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have
>>>>>>>>>>>>>>>>>>>>>>>> overloaded
>>>>>>>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a 
>>>>>>>>>>>>>>>>>>>>>>> `Predicate`
>>>>>>>>>>>>>>>>>>>>>>> and assume you
>>>>>>>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe
>>>>>>>>>>>>>>>>>>>>>>> rephrase it accordingly
>>>>>>>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public
>>>>>>>>>>>>>>>>>>>>>>> Interface" section) of
>>>>>>>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces) 
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction
>>>>>>>>>>>>>>>>>>>>>>> of `withConsumer()`
>>>>>>>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. 
>>>>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>> related to John's
>>>>>>>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use
>>>>>>>>>>>>>>>>>>>>>>>> cases: EITHER using
>>>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and
>>>>>>>>>>>>>>>>>>>>>>> `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the
>>>>>>>>>>>>>>>>>>>>>>> result of `withChain()`
>>>>>>>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really
>>>>>>>>>>>>>>>>>>>>>>> with the `Consumer`
>>>>>>>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and
>>>>>>>>>>>>>>>>>>>>>>> `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not 
>>>>>>>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>>>>> the "branch" to
>>>>>>>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as
>>>>>>>>>>>>>>>>>>>>>>> we clearly document
>>>>>>>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’
>>>>>>>>>>>>>>>>>>>>>>>> confusing; I thought you
>>>>>>>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface
>>>>>>>>>>>>>>>>>>>>>>>> (which doesn’t make
>>>>>>>>>>>>>>>>>>>>>>>> sense, of course). I get that you were referring to
>>>>>>>>>>>>>>>>>>>>>>>> the java Consumer
>>>>>>>>>>>>>>>>>>>>>>>> interface, but we should still probably to to avoid
>>>>>>>>>>>>>>>>>>>>>>>> the ambiguity.
>>>>>>>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about 
>>>>>>>>>>>>>>>>>>>>>>>> ‘withSink’?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be
>>>>>>>>>>>>>>>>>>>>>>> confused with a "sink
>>>>>>>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less 
>>>>>>>>>>>>>>>>>>>>>>> ambiguous?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I
>>>>>>>>>>>>>>>>>>>>>>>> think it’s close,
>>>>>>>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev 
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> will someone please take a look at the reworked 
>>>>>>>>>>>>>>>>>>>>>>>>> KIP?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I believe that now it follows design principles 
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> takes into account
>>>>>>>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" 
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> have completely
>>>>>>>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> This version includes all the previous discussion
>>>>>>>>>>>>>>>>>>>>>>>>>> results and follows
>>>>>>>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V>
>>>>>>>>>>>>>>>>>>>>>>>>>> branched)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one
>>>>>>>>>>>>>>>>>>>>>>>>>> parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and 
>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>> need to provide one
>>>>>>>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations
>>>>>>>>>>>>>>>>>>>>>>>>>> we may use a single
>>>>>>>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method
>>>>>>>>>>>>>>>>>>>>>>>>>> parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a
>>>>>>>>>>>>>>>>>>>>>>>>>> branch, no
>>>>>>>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.'
>>>>>>>>>>>>>>>>>>>>>>>>>> is expected here
>>>>>>>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single 
>>>>>>>>>>>>>>>>>>>>>>>>>> parameter
>>>>>>>>>>>>>>>>>>>>>>>>>> rule'.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to
>>>>>>>>>>>>>>>>>>>>>>>>>>> have you back!
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we
>>>>>>>>>>>>>>>>>>>>>>>>>>> already have a
>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable to me to
>>>>>>>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>>>>>>>>>>>> principles" that we
>>>>>>>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar 
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we invested
>>>>>>>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>> not feel entitled to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalized.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed
>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> original idea for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I gave up pushing
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem with the scope of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .branch(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>                .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright
>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution in his post
>>>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current
>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>> names seems to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making
>>>>>>>>>>>>>>>>>>>>>>>>>>>> your code brittle.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging
>>>>>>>>>>>>>>>>>>>>>>>>>>>> purposes. Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> named
>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on
>>>>>>>>>>>>>>>>>>>>>>>>>>>> June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The idea seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust
>>>>>>>>>>>>>>>>>>>>>>>>>>>> people who are more
>>>>>>>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>> me.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status".
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about my proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worlds?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to merge both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your example.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added/removed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default name but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> explicit to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that neither
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> independently of all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that, although
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> guess
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split(Named.withName("mysplit"))
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                           .branch(..., ...,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                           
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operator(function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               operator(function, function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> string)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                interface BranchConfig 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noisier,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but it also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sink
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Ivan, I’ll definitely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch(predicate, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumer)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 IMO the one trade off to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 question. I don’t 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the same scope” since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 seems like a perfectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 when the branched 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the reasons Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution - working 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > On May 2, 2019, at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7:00
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Hello everyone, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thank you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Well, I don't think 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 LinkedHashMap (no 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 matters) or `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 advantages than 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > In my opinion, the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal is that all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 But 1) we rarely need 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 workaround for the scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 'Inlining the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method references 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 tend to split the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > The drawbacks are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> strong.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers is lost. We 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers in another. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misspell a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > What Michael propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 writing the API in Lua,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "dynamic naming" 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 beautiful. But in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 identifiers to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Do we have to invent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > And if we do, what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 all the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Earlier in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without "start 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the case when we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current KIP is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'clumsier'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 me address both comments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 1) "Start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for it indeed) is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 2) No, dynamic 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Imagine a real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 value (say, RecordType).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*John:if we had to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 have been much 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > KBranchedStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*Not clumsy at all 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > for (RecordType
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >             branched =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 02.05.2019 14:40,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> I also agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> current `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> However, I also don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> was more aligned with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `branch()` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statement and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> It makes the code 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `Predicates` (that is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        ��        >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> An open question 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> specified. Atm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> and the call to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> (what is not the case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> because users can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> About "inlining": So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> preference. I can see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> that clearly make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> On 5/1/19 6:26 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Perhaps inlining 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that a lambda with 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the full
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 it can be a method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The advantage of putting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 (Consumer) together in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Ultimately the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch trees will be 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> On May 1, 2019, at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> I'm less 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enthusiastic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> functionality.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> harder to read as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 30, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    >>>>> Also +1 on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> great framework 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Regarding the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> proposal in the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Obviously some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't work 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> associativity 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> directly 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> provides over 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> The KIP 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that it is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> certainly 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> case anyway, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> access your 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch, and it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solution on top of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> I could also see a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> taken in, branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Pros for that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - accessing 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - no double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - downstream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to read top to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - you can 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API and SortedMap,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> (KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Overall I'm 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> curious
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branched 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't need to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user.  I'm sort 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 30, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <sophie@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I'd like to +1 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> method, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> trying to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> alleviate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> able to cleanly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> name->stream), 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> That said, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> anti-pattern or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> On Tue, Apr 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I’d like to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> are three 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 1. If you use it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2. The way in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the ordering 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 3. It is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> brittle to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Using 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> associative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be a stronger
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> super K,? super
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Branches are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> given
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> mapping of 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> because it’s a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> This solves 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2 because you no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> interested 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> conditional by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> messing with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> One of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> awkward in 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Java. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> On Tue, Apr 30,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <john@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> FWIW, I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> confusing when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like a good 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> operator at 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Tentatively, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> way, we don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> should return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> enforce that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> default branch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> and 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> additionally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> though with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> On Fri, Apr 26,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> this is to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The intend 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> IMHO, this 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> take any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> `branch()` 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Because I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> might be 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> overlap that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Maybe there 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> naming 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 'default' is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a method with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Bummer. Didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Can you add 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> methods? It 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP. For 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> You did not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -> KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> nice to get 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> would need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> should discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "incomplete" to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched-KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> On 4/13/19 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2:13
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I have 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Matthias, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I can see 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatibility?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Totally agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> reserved 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> word,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> is not 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Absolutely! I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Dear 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> please revise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512) 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Any new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 11.04.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> agrees 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> I had a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> quick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> are some 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> It's just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> In the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Also, we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Ie, we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> For the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> could we 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> second 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> `Function`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Finally, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> proposal. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On 3/31/19
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> I'm a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bit of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> revise 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> buy-in from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> could be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> idea 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> forward the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> deprecate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> same 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> just 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> is 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> little 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sloppy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> passing 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream but read from all the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> complicated to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On Fri, Mar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I read 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> looks 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> going 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> What 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shall we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Why are 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we go 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Maybe I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> added, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> able to 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> The 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> worked 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> [Also, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 28,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> The 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> the 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other.  
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a way 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> You just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> write in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Consider
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> customers who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> store 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> This 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 'brancher'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> class
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>   
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>   
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>       
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> /*In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> classes'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*Alas,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> without 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Does 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we need 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> [BTW 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> KIP I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> so I will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tried to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512), 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> succeeded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> To be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> them 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> has the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> On 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Wed,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> OK, 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> There are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 1. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PROS: 1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> all 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 2. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> PROS:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CONS: We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> I 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Still,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1) OK,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> drop
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Well,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> tests,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��      >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��      
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> Does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    ��            
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...).. 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey()..... 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/6164 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 



Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
 > Can you maybe explain your reasoning? Why not just start with "0" and
 > the default branch gets the next index as any other branch

My reasoning was the following: when we add a new branch, all the 
following branches will be renumbered. If the default branch gets the 
maximum index, its handler should be modified each time a new branch is 
added. If the default branch always has "0" index, we'll have to modify 
less code when adding a new branch to the end of the list.

 > that's a minor question.

I agree that it's a minor question, although maybe for a different 
reason :-)

As a programmer, I don't believe in default branch naming, I'm not going
to use it myself and I'm going to discourage others from using it.

The code relying on default naming scheme is going to be brittle. For 
example, if we want to reorder branches, we will also have to 
synchronously change the code that extracts branches from the Map for 
handling, with lots of opportunities to make a mistake.

Low cohesion between branch predicates and branch handlers is one of the 
problems of current branching API that this KIP was intended to fight.

And this is why my first goal was to avoid explicit contract for default 
branch naming in KIP/JavaDoc.

Note that I agree that some form of default branch naming is needed. I 
can imagine cases when I'm going to use the resulting Map without 
explicit knowledge of its keys (e. g. operating on map.values() only).

So if for some reason you feel that other indexing scheme is more 
preferable, I'm absolutely open for any proposals, because I don't think 
that it is an important matter :-)

Regards,

Ivan

05.06.2020 3:18, Matthias J. Sax пишет:
> Thanks for updating the KIP!
> 
> I am overall happy with it. The proposed default branch numbering (ie,
> start with "1" and use "0" for the default branch) is not super
> intuitive, but that's a minor question.
> 
> Can you maybe explain your reasoning? Why not just start with "0" and
> the default branch gets the next index as any other branch (ie, this is
> how the currently returned array indexes the branches, too)?
> 
> 
> -Matthias
> 
> On 6/1/20 5:29 AM, Ivan Ponomarev wrote:
>> Hi all,
>>
>> I updated the KIP with what we have discussed, see 'How the resulting
>> Map is formed' section. Four bullet points to define the rules for Map
>> keys, and four bullet points to define the rules for Map values.
>>
>> I also added the overloads to `Branched` in order to accept Consumers as
>> parameters.
>>
>> Regards,
>>
>> Ivan
>>
>>
>> 01.06.2020 8:56, Matthias J. Sax пишет:
>>> For naming:
>>>
>>> (1) _If_ people specify a name for split() _and_ branch(), we _must_
>>> have a pulic API contract. Otherwise it would be very bad user
>>> experience if we are allowed to change the names (eg, do from "we don't
>>> insert a `-` to we insert a `-`). -- We also need to clarify what
>>> happens is only split() _or_ branch() has a name specified, ie, is there
>>> a contract for this case or not?
>>>
>>> (2) If we have a contract for how split() and branch() names are put
>>> together, we can also apply it to auto-generated names (this seems to be
>>> just consistent).
>>>
>>> (3) Do we want to have a contract for auto-generated names? This is
>>> debatable, but I personally don't see any harm in having a contract.
>>> Also, if people want to access the Map, they can easily do it without
>>> specifying names if there is a contract -- if there is no contract, we
>>> force people to specify names to use the Map, what seems to make the API
>>> harder to use.
>>>
>>> But we can of course discuss furhter on the dev list.
>>>
>>>
>>> -Matthias
>>>
>>> On 5/30/20 3:53 PM, Ivan Ponomarev wrote:
>>>> Hi Mathhias,
>>>>
>>>> I received your letter just after I sent mine concerning nulls and
>>>> reintroducing Consumer overload. So, if you're fine with omitting nulls
>>>> and introducing a with(Consumer) overload, it's great!
>>>>
>>>> Concerning the naming algorithm. Well, actually what you propose is very
>>>> clear and this is what I was going to implement anyway. But are you sure
>>>> that we will never want to change this algorithm? Here I trust your
>>>> judgement, since you understand the KStreams API's evolution much more
>>>> than I do.
>>>>
>>>>
>>>> 31.05.2020 0:50, Matthias J. Sax пишет:
>>>>> Ivan, John,
>>>>>
>>>>> thanks for the details.
>>>>>
>>>>> I guess I can be convinced in either direction with regard to `null`
>>>>> handling. But I am less worried about it, because I assume that a Java
>>>>> programmer understand the ambiguty of `get(key) == null` -- that's why
>>>>> there is `containsKey()`. In the end it might be a minor detail and as
>>>>> long as it's well documented all discussed approaches seem fine.
>>>>> Including the idea to re-introduce the `Consumer` overload.
>>>>>
>>>>> It's Ivan's KIP so I am fine with whatever he picks :)
>>>>>
>>>>>
>>>>> About the naming: Not sure why we don't want to make the naming a
>>>>> public
>>>>> contract? I guess for split() we could default to the empty String, and
>>>>> for `branch()` we could just add a counter by default?
>>>>>
>>>>> With no passed names at all, we just generte branche names "1", "2",
>>>>> "3", etc... With only`split(Named.as("foo-"))` (but not branch
>>>>> names) we
>>>>> generate "foo-1", "foo-2", etc.
>>>>>
>>>>> If any branch has a dedicated name, the counting is preserved but just
>>>>> not used int he name for this brach:
>>>>>
>>>>> split("foo-")
>>>>>      .brach()
>>>>>      .brach(Branched.as("bar")
>>>>>      .defaultBranch()
>>>>>
>>>>> would generate "foo-1", "foo-bar", "foo-3"?
>>>>>
>>>>> Thoughts?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 5/29/20 1:20 PM, John Roesler wrote:
>>>>>> Hi all,
>>>>>>
>>>>>> Right, that was my suggestion; sorry for the confusion. I was thinking
>>>>>> that adding null as a value to maps is generally dubious, for which
>>>>>> reason a lot of Map implementations actually don't allow it at all.
>>>>>>
>>>>>> The reason is that it creates ambiguity, since a lot of code treats
>>>>>> "map.get(key) == null" as meaning the key is not in the map, but
>>>>>> if we place a null value in the map (presuming we don't immediately
>>>>>> get an exception), then you face weird contradictions, like
>>>>>> get(key) == null, but containsKey(key) can be either true or false,
>>>>>> and
>>>>>> you may or may not see it while iterating. Which would violate the
>>>>>> expectations of many Java programs. You can just take a look at
>>>>>> the AK codebase, and you'll find many occurrences where we
>>>>>> assume a null mapping means the map doesn't contain the key.
>>>>>>
>>>>>> Of course, the practical question, which Ivan brought up, is also
>>>>>> a good one. What would you actually do with these null values
>>>>>> in the map, besides get an NPE ?
>>>>>>
>>>>>> I'm not sure why I didn't think of this before, but an alternative
>>>>>> to this debate is to go back to adding Consumer into the API, but
>>>>>> with the same method name, withChain(Consumer<KStream<K,V>>).
>>>>>>
>>>>>> Now that we have only static methods, it's less confusing because it's
>>>>>> not possible to choose _both_ the Function<KStream,KStream> _and_
>>>>>> the Consumer<KStream> in the same branch (which was one of the
>>>>>> main drawbacks of the original plan wrt withChain and
>>>>>> withJavaConsumer as builder methods.
>>>>>>
>>>>>> I.e., I was previously concerned about:
>>>>>> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2)
>>>>>>
>>>>>>
>>>>>> It's just a little confusing trying to reason about how this is all
>>>>>> going to wire up.
>>>>>>
>>>>>> But with only the static methods, users have only three methods,
>>>>>> and they are all clear:
>>>>>> 1. Branched.as(name): just names the branch, the branch-predicated
>>>>>> stream is the value of the map
>>>>>> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the
>>>>>> branch, applies the transformation after the branch predicate, and
>>>>>> the transformed stream is the value of the map (null is not permitted)
>>>>>> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch,
>>>>>> passes the predicated stream in to the Consumer function and omits
>>>>>> the branch from the map
>>>>>>
>>>>>> Sorry for digging this option up again, but it's starting to look more
>>>>>> attractive after we dropped the builder functions, and I think it also
>>>>>> resolves the "null" concern.
>>>>>>
>>>>>> Thoughts?
>>>>>> -John
>>>>>>
>>>>>>
>>>>>> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>>>>>>> Hi Matthias!
>>>>>>>
>>>>>>> Thanks for your reply!
>>>>>>>
>>>>>>> (1)
>>>>>>>
>>>>>>>     > Do you imply that there won't be a `name -> null` entry in the
>>>>>>> `Map`
>>>>>>> for the branch? If yes, I am wondering why?
>>>>>>>
>>>>>>> Ah, I re-read the discussion and found out that omitting null entries
>>>>>>> was John's idea, not yours :-)
>>>>>>>
>>>>>>> Anyway. As you remember, I was against the 'swallowing' of
>>>>>>> branches by
>>>>>>> Consumer. I had a feeling that there are cases when I would like
>>>>>>> both to
>>>>>>> consume a branch and post-process it in a Map handler.
>>>>>>>
>>>>>>> But then the idea of getting rid of `withJavaConsumer` came up. And
>>>>>>> then
>>>>>>> we realized that we should handle nulls somehow.
>>>>>>>
>>>>>>> So it looked like taking the best from both approaches: if we want to
>>>>>>> omit a branch in the resulting map, we just return null, thus
>>>>>>> emulating
>>>>>>> a 'swallowing consumer'. If we want to include a branch, we return
>>>>>>> it.
>>>>>>> It provides the full control for a user, and it spares the user from
>>>>>>> null-checking in their code.
>>>>>>>
>>>>>>>     > My reasoning is, that it's very clear that `name -> null`
>>>>>>> would be
>>>>>>> there, because the user code did execute `return null`.
>>>>>>>
>>>>>>> My reasoning is following. The way such lambdas are written, it's
>>>>>>> hard
>>>>>>> to return null unintentionally. And if a user returns null
>>>>>>> intentionally, what `name->null` in the resulting Map can be good
>>>>>>> for?
>>>>>>> It's either NPE (which is clearly not intended) or something that
>>>>>>> should
>>>>>>> be null-checked and skipped. Ok, let's do it!
>>>>>>>
>>>>>>> But, I'm ready to be convinced to drop this rule. It doesn't seem
>>>>>>> to me
>>>>>>> to be a principal thing.
>>>>>>>
>>>>>>> (2)
>>>>>>>
>>>>>>>     > I think it would be good if the KIP could explains the intended
>>>>>>> default naming schema. This is important because the naming schema
>>>>>>> must
>>>>>>> be part of the API contract; otherwise, users cannot rely on the
>>>>>>> naming
>>>>>>> when trying to use the returned `Map`.
>>>>>>>
>>>>>>> Well, in fact, I intentionally didn't include any naming schema in
>>>>>>> the
>>>>>>> KIP. And it's done exactly for the purpose that a user won't be
>>>>>>> able to
>>>>>>> rely on default naming. If they want to use certain branches from the
>>>>>>> Map, they will devise their own naming strategy that will never
>>>>>>> change
>>>>>>> and will never let them down.
>>>>>>>
>>>>>>> If they don't mind what's in a Map, if they are looking up the Map
>>>>>>> just
>>>>>>> for debugging, or if they want to treat a Map just like a
>>>>>>> Collection of
>>>>>>> values (a possible use case!), it doesn't matter how the branches
>>>>>>> are named.
>>>>>>>
>>>>>>> This way we are giving ourselves the full freedom to change the
>>>>>>> naming
>>>>>>> schema afterwards without violating the API contract.
>>>>>>>
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan
>>>>>>>
>>>>>>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>>>>>>> Thanks for updating the KIP!
>>>>>>>>
>>>>>>>> Using covariant generics is a good idea! I am also fine with only
>>>>>>>> using
>>>>>>>> static method in `Branched` for now, as it's only two parameters and
>>>>>>>> thus not too many overloads.
>>>>>>>>
>>>>>>>>
>>>>>>>> (1) What is unclear to me is, what you exaclty mean by:
>>>>>>>>
>>>>>>>>> If a function returns `null`, its result is omitted.
>>>>>>>>
>>>>>>>> Do you imply that there won't be a `name -> null` entry in the
>>>>>>>> `Map` for
>>>>>>>> the branch? If yes, I am wondering why?
>>>>>>>>
>>>>>>>> My previous argument was to omit an entry only for the
>>>>>>>> `withJavaConsumer()` case, because the return type is "void" (and
>>>>>>>> thus
>>>>>>>> it's unclear what should be added and it would introduce
>>>>>>>> inconsistencies
>>>>>>>> in the `Map`).
>>>>>>>>
>>>>>>>> However, for a `Function` that returns a `KStream`, we could
>>>>>>>> actually
>>>>>>>> just add a `name -> null` entry if `null` is returned. My reasoning
>>>>>>>> is,
>>>>>>>> that it's very clear that `name -> null` would be there, because the
>>>>>>>> user code did execute `return null`. Adding a `name -> null`
>>>>>>>> entry for
>>>>>>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>>>>>>> exceptional case for handling `null`).
>>>>>>>>
>>>>>>>> Thoughts?
>>>>>>>>
>>>>>>>>
>>>>>>>> (2) For `Branched.as()` the KIP says:
>>>>>>>>
>>>>>>>>> sets the name of the branch (auto-generated by default, when split
>>>>>>>>> operation is named, then the names are index-suffixed)
>>>>>>>>
>>>>>>>> I think it would be good if the KIP could explains the intended
>>>>>>>> default
>>>>>>>> naming schema. This is important because the naming schema must be
>>>>>>>> part
>>>>>>>> of the API contract; otherwise, users cannot rely on the naming when
>>>>>>>> trying to use the returned `Map`.
>>>>>>>>
>>>>>>>> Similalry, it should be explained how names are generated if
>>>>>>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations
>>>>>>>> how the
>>>>>>>> API can be mixed and matched and it's unclear atm how it would
>>>>>>>> work in
>>>>>>>> detail.
>>>>>>>>
>>>>>>>>
>>>>>>>> Thanks!
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>>>>>>> Hi Ivan,
>>>>>>>>>
>>>>>>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>>>>>>> that have been raised in the discussion so far have been
>>>>>>>>> addressed. And it's been a while since anyone raised a new
>>>>>>>>> concern. At this point, it seems like a good time to start
>>>>>>>>> the VOTE thread.
>>>>>>>>>
>>>>>>>>> Sometimes, the vote thread will trigger new people to look
>>>>>>>>> into the KIP, and they may raise new concerns, but it's not a
>>>>>>>>> problem. We'll just address those lingering concerns if there
>>>>>>>>> are any, until you have all the votes you need.
>>>>>>>>>
>>>>>>>>> Thanks again for the contribution!
>>>>>>>>>
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>>>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>>>>>>
>>>>>>>>>> 1. Per Mathhias's suggestion, if a chain function returns null,
>>>>>>>>>> the
>>>>>>>>>> respective result if omitted in the resulting Map.
>>>>>>>>>>
>>>>>>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>>>>>>
>>>>>>>>>> 3. `Branched` class has only three static methods with all the
>>>>>>>>>> possible
>>>>>>>>>> combinations of parameters.
>>>>>>>>>>
>>>>>>>>>> 4. Chain function is defined 'fully covariant', let's see if we
>>>>>>>>>> can
>>>>>>>>>> implement it this way :-))
>>>>>>>>>>
>>>>>>>>>> + code example updates and minor edits.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Since this is my first KIP, I'm not sure what should I do next. I
>>>>>>>>>> feel
>>>>>>>>>> that we talked over all the details and the consensus is reached.
>>>>>>>>>> Is it
>>>>>>>>>> OK to call for VOTE now or is it better to wait for more feedback?
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>>
>>>>>>>>>> Ivan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>>>>>>> Sounds good to me, Ivan!
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>>>>>>> John,
>>>>>>>>>>>>
>>>>>>>>>>>> ---------------------
>>>>>>>>>>>>
>>>>>>>>>>>>       > I'm sorry things have been dragging out a little, but I
>>>>>>>>>>>> have the
>>>>>>>>>>>> sense we're very close to the end of this discussion, which is
>>>>>>>>>>>> exciting.
>>>>>>>>>>>>
>>>>>>>>>>>> We are certainly moving forward!  And I'm not in a hurry at
>>>>>>>>>>>> all. As I
>>>>>>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>>>>>>> KafkaStreamBrancher -- the implementation of the first,
>>>>>>>>>>>> rejected version
>>>>>>>>>>>> of this KIP. It's inferior to what we are discussing here, but
>>>>>>>>>>>> it does
>>>>>>>>>>>> its work. So it's worth to design this KIP really, really well.
>>>>>>>>>>>> And by
>>>>>>>>>>>> the way, from this discussion I'm learning the good API
>>>>>>>>>>>> designing
>>>>>>>>>>>> process. For me it has a value per se :-))
>>>>>>>>>>>>
>>>>>>>>>>>> ----------------------
>>>>>>>>>>>>
>>>>>>>>>>>>       >>   > I'd wonder whether we need the non-static builders
>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>       >>   > Do they provide any benefit over just using the
>>>>>>>>>>>> right static
>>>>>>>>>>>> factory?
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > I don't have a strong feeling, either. It seems nice to
>>>>>>>>>>>> offer a better
>>>>>>>>>>>>       > type inference experience than what we get with
>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>       > offering the static method that takes both name and
>>>>>>>>>>>> chain.
>>>>>>>>>>>>       > Given that, there doesn't seem to be a good reason to
>>>>>>>>>>>> also offer the
>>>>>>>>>>>>       > non-static builder-style methods, so I guess I'd prefer
>>>>>>>>>>>> to drop them.
>>>>>>>>>>>>
>>>>>>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I
>>>>>>>>>>>> found out
>>>>>>>>>>>> an interesting fact about type inference rules in Java. Funny
>>>>>>>>>>>> thing is
>>>>>>>>>>>> that although we need to explicitly set types in a chain like
>>>>>>>>>>>> this
>>>>>>>>>>>>
>>>>>>>>>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>>>>>>>>>>>
>>>>>>>>>>>> (otherwise it won't compile), the composition of static method
>>>>>>>>>>>> calls
>>>>>>>>>>>> works just fine, all the types are being calculated correctly:
>>>>>>>>>>>>
>>>>>>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>>>>>>>>>>>
>>>>>>>>>>>> As I was told, for type inference there is difference between
>>>>>>>>>>>> qualifiers
>>>>>>>>>>>> and arguments, 'you go up if you are argument but stop if you
>>>>>>>>>>>> are
>>>>>>>>>>>> qualifier'. And it also seems that we should not bet on any
>>>>>>>>>>>> future
>>>>>>>>>>>> improvements in Java type inference here.
>>>>>>>>>>>>
>>>>>>>>>>>> So,
>>>>>>>>>>>>
>>>>>>>>>>>> 1) I think we that in this KIP we should provide three static
>>>>>>>>>>>> methods
>>>>>>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function,
>>>>>>>>>>>> String)`, and
>>>>>>>>>>>> drop any non-static ones.
>>>>>>>>>>>>
>>>>>>>>>>>> 2) If anything else will be ever needed, we can easily add
>>>>>>>>>>>> anything.
>>>>>>>>>>>> Maybe this can be done in a process of refinement of all the
>>>>>>>>>>>> parameter
>>>>>>>>>>>> classes.
>>>>>>>>>>>>
>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>
>>>>>>>>>>>>       > we may as well hope for the best, and propose the "fully
>>>>>>>>>>>>       > covariant" definition for now.
>>>>>>>>>>>>
>>>>>>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>>>>>>
>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>
>>>>>>>>>>>>       >> Good question, I already thought about it and rejected
>>>>>>>>>>>> the idea....
>>>>>>>>>>>>       > (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>>       >  "Worst" case scenario: someone
>>>>>>>>>>>>       > else will wish the return type is something different,
>>>>>>>>>>>> and we'll go
>>>>>>>>>>>>       > through a painless deprecation transition to change it
>>>>>>>>>>>> later.
>>>>>>>>>>>>
>>>>>>>>>>>> Of course, we cant' predict all the ways people are going to
>>>>>>>>>>>> use it.
>>>>>>>>>>>>       From my own humble experience with Kafka Streams, the worst
>>>>>>>>>>>> scenario is
>>>>>>>>>>>> unlikely. Split is split, transform is transform, too much
>>>>>>>>>>>> flexibility
>>>>>>>>>>>> is often evil.
>>>>>>>>>>>>
>>>>>>>>>>>> ------------------------
>>>>>>>>>>>>
>>>>>>>>>>>> So it seems that we are close to the consensus. Two things to
>>>>>>>>>>>> be altered
>>>>>>>>>>>> in the current version of KIP:
>>>>>>>>>>>>
>>>>>>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>>>>>>
>>>>>>>>>>>> Any ideas / questions / objections?
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Ivan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'm sorry things have been dragging out a little, but I have
>>>>>>>>>>>>> the sense
>>>>>>>>>>>>> we're very close to the end of this discussion, which is
>>>>>>>>>>>>> exciting.
>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>>        > Do they provide any benefit over just using the right
>>>>>>>>>>>>>> static factory?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>
>>>>>>>>>>>>> I don't have a strong feeling, either. It seems nice to offer
>>>>>>>>>>>>> a better
>>>>>>>>>>>>> type inference experience than what we get with
>>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Given that, there doesn't seem to be a good reason to also
>>>>>>>>>>>>> offer the
>>>>>>>>>>>>> non-static builder-style methods, so I guess I'd prefer to
>>>>>>>>>>>>> drop them.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'll defer to Matthias, if he has a chance to consider whether
>>>>>>>>>>>>> it's more
>>>>>>>>>>>>> valuable to stick with the existing pattern or break the
>>>>>>>>>>>>> pattern to offer
>>>>>>>>>>>>> a better experience.
>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in the KIP,
>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>>        > covariant typed function
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been that
>>>>>>>>>>>>> we just
>>>>>>>>>>>>> won't know what works until we really try it, and try it in
>>>>>>>>>>>>> several different
>>>>>>>>>>>>> ways. Still, we may as well hope for the best, and propose the
>>>>>>>>>>>>> "fully
>>>>>>>>>>>>> covariant" definition for now.
>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and value
>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>>        > same as the inputs?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>> idea....
>>>>>>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>>>
>>>>>>>>>>>>> That's fair! It's your KIP, after all. I think I might have
>>>>>>>>>>>>> made a different
>>>>>>>>>>>>> call here, but I think this choice is fine. "Worst" case
>>>>>>>>>>>>> scenario: someone
>>>>>>>>>>>>> else will wish the return type is something different, and
>>>>>>>>>>>>> we'll go
>>>>>>>>>>>>> through a painless deprecation transition to change it later.
>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>> to the clean design of your API, this doesn't seem to bad.
>>>>>>>>>>>>> And, of
>>>>>>>>>>>>> course, you've actually been using similar functionality
>>>>>>>>>>>>> already, so it
>>>>>>>>>>>>> seems we should trust your intuition.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> -John
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>> Do they provide any benefit over just using the right static
>>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>>> builders can
>>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > You might as well propose the “ideal” API in the KIP,
>>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>> covariant typed function
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > Is it necessary to restrict the result key and value
>>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>> same as the inputs?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Look, if we want to keep `withChain`'s function optional,
>>>>>>>>>>>>>> then we must
>>>>>>>>>>>>>> keep the result key and value types the same. Because for
>>>>>>>>>>>>>> now, the
>>>>>>>>>>>>>> default value for the 'chain function' is Function.identity().
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Of course, we can make the 'chain function' required. But I
>>>>>>>>>>>>>> think this
>>>>>>>>>>>>>> is not what `split` method is for. `split` is for
>>>>>>>>>>>>>> splitting, not
>>>>>>>>>>>>>> transforming, and `chainFunction` in most of the cases should
>>>>>>>>>>>>>> be either
>>>>>>>>>>>>>> a consumer or the identity function.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past
>>>>>>>>>>>>>>> that generic type inference problem is to offer a static
>>>>>>>>>>>>>>> factory method that takes all the options (both name and
>>>>>>>>>>>>>>> function).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I’m happy with adding that option, although if we have all
>>>>>>>>>>>>>>> possible combinations available as static methods, then I’d
>>>>>>>>>>>>>>> wonder whether we need the non-static builders (like
>>>>>>>>>>>>>>> withChain). Do they provide any benefit over just using the
>>>>>>>>>>>>>>> right static factory?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type
>>>>>>>>>>>>>>> bounds works algebraically, but I’m not sure whether the
>>>>>>>>>>>>>>> limitations of Java type inference will actually let you
>>>>>>>>>>>>>>> pass in all the different functions you would want to.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I recall Bruno and I wrestling with this when he added
>>>>>>>>>>>>>>> Transform operators. In retrospect, this is a problem we can
>>>>>>>>>>>>>>> iron out when we write tests in the PR. You might as well
>>>>>>>>>>>>>>> propose the “ideal” API in the KIP, which is the covariant
>>>>>>>>>>>>>>> typed function:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>>> KStream<? extends K, ? extends V>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 3. Actually, I have a new question about the types: Is it
>>>>>>>>>>>>>>> necessary to restrict the result key and value types to be
>>>>>>>>>>>>>>> the same as the inputs? I.e., shouldn’t the result types be
>>>>>>>>>>>>>>> K1,V1?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>> Hello John,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         > Perhaps it would be better to stick with "as"
>>>>>>>>>>>>>>>> for now
>>>>>>>>>>>>>>>>         > and just file a Jira to switch them all at the same
>>>>>>>>>>>>>>>> time [for
>>>>>>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin
>>>>>>>>>>>>>>>> they have a
>>>>>>>>>>>>>>>> standard workaround
>>>>>>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> So actually this should be a very low priority issue, if an
>>>>>>>>>>>>>>>> issue at
>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         > I don't understand how your new proposed
>>>>>>>>>>>>>>>>         > methods would work any differently than the ones
>>>>>>>>>>>>>>>> you already
>>>>>>>>>>>>>>>>         > had proposed in the KIP. It seems like you'd still
>>>>>>>>>>>>>>>> have to provide
>>>>>>>>>>>>>>>>         > the generic type parameters on the first static
>>>>>>>>>>>>>>>> factory call. Can you
>>>>>>>>>>>>>>>>         > explain how your new interface proposal differs
>>>>>>>>>>>>>>>> from the existing KIP?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> In the KIP, I didn't clarify what methods should be static.
>>>>>>>>>>>>>>>> Now I
>>>>>>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The overloaded `with` version that provides both Function
>>>>>>>>>>>>>>>> and name can
>>>>>>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         > Regarding making the K,V types covariant also, yes,
>>>>>>>>>>>>>>>> that would indeed
>>>>>>>>>>>>>>>>         > be nice, but I'm not sure it will actually work.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> static KStream<String, Integer> func(KStream<String,
>>>>>>>>>>>>>>>> Number> s) {
>>>>>>>>>>>>>>>>                 return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>>>>>>             s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>>>>>>                        //Won't compile!!
>>>>>>>>>>>>>>>>                        Branched.with(Me::func));
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The simple workaround here is to change `func`'s return
>>>>>>>>>>>>>>>> type from
>>>>>>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [On the other hand, we already agreed to remove
>>>>>>>>>>>>>>>> `withJavaConsumer` from
>>>>>>>>>>>>>>>> `Branched`, so during code migration I will have to modify
>>>>>>>>>>>>>>>> my functions'
>>>>>>>>>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >  the map you're returning is Map<K,V>, and of
>>>>>>>>>>>>>>>> course a K is not the
>>>>>>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I think what you actually meant here is that KStream<?
>>>>>>>>>>>>>>>> extends K, ?
>>>>>>>>>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K,
>>>>>>>>>>>>>>>> V>>. This
>>>>>>>>>>>>>>>> particularly is not a problem, since KStream<? extends K, ?
>>>>>>>>>>>>>>>> extends V>
>>>>>>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put
>>>>>>>>>>>>>>>> to the map.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard
>>>>>>>>>>>>>>>> types. So maybe
>>>>>>>>>>>>>>>> for now it's better to just admit that API is not
>>>>>>>>>>>>>>>> absolutely perfect and
>>>>>>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that
>>>>>>>>>>>>>>>>> "as" would
>>>>>>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from
>>>>>>>>>>>>>>>>> convention
>>>>>>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The convention is that "as(String name)" is used for the
>>>>>>>>>>>>>>>>> static factory
>>>>>>>>>>>>>>>>> method, whereas "withName(String name)" is an instance
>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to
>>>>>>>>>>>>>>>>> avoid "as"
>>>>>>>>>>>>>>>>> for compatibility with Kotlin, I might suggest
>>>>>>>>>>>>>>>>> "fromName(String name)",
>>>>>>>>>>>>>>>>> although it's somewhat dubious, since all the other
>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick with
>>>>>>>>>>>>>>>>> "as" for now
>>>>>>>>>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>>>>>>> Regarding the type inference problem, yes, it's a blemish
>>>>>>>>>>>>>>>>> on all of our
>>>>>>>>>>>>>>>>> configuraion objects. The problem is that Java infers the
>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>> based on the _first_ method in the chain. While it does
>>>>>>>>>>>>>>>>> consider what
>>>>>>>>>>>>>>>>> the recipient of the method result wants, it only
>>>>>>>>>>>>>>>>> considers the _next_
>>>>>>>>>>>>>>>>> recipient.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>>>>>>>>>>>> Branched<String,String> variable, java infers the type
>>>>>>>>>>>>>>>>> correctly. But
>>>>>>>>>>>>>>>>> when the "next recipient" is a chained method call, like
>>>>>>>>>>>>>>>>> "withChain",
>>>>>>>>>>>>>>>>> then the chained method doesn't bound the type (by
>>>>>>>>>>>>>>>>> definition,
>>>>>>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java
>>>>>>>>>>>>>>>>> will take
>>>>>>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be
>>>>>>>>>>>>>>>>> revised anymore.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, since it
>>>>>>>>>>>>>>>>> doesn't seem
>>>>>>>>>>>>>>>>> that hard to recursively consider the entire context when
>>>>>>>>>>>>>>>>> inferring the
>>>>>>>>>>>>>>>>> generic type parameters, but this is what we have to work
>>>>>>>>>>>>>>>>> with.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> To be honest, though, I don't understand how your new
>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>> methods would work any differently than the ones you
>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still have to
>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>> the generic type parameters on the first static factory
>>>>>>>>>>>>>>>>> call. Can you
>>>>>>>>>>>>>>>>> explain how your new interface proposal differs from the
>>>>>>>>>>>>>>>>> existing KIP?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that
>>>>>>>>>>>>>>>>> would indeed
>>>>>>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You might
>>>>>>>>>>>>>>>>> want to give it a
>>>>>>>>>>>>>>>>> try. In the past, we've run into soe truly strange
>>>>>>>>>>>>>>>>> interactions between the
>>>>>>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner
>>>>>>>>>>>>>>>>> classes) in
>>>>>>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Another issue is that the value type of the map you're
>>>>>>>>>>>>>>>>> returning is
>>>>>>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends
>>>>>>>>>>>>>>>>> K", so it
>>>>>>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that
>>>>>>>>>>>>>>>>>> getting rid of
>>>>>>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as
>>>>>>>>>>>>>>>>>> possible, but not
>>>>>>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to
>>>>>>>>>>>>>>>>>> implement examples
>>>>>>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1. Having to return something from lambda is not a very
>>>>>>>>>>>>>>>>>> big deal.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use
>>>>>>>>>>>>>>>>>> method references
>>>>>>>>>>>>>>>>>> for already written stream consumers, but then I realized
>>>>>>>>>>>>>>>>>> that I can
>>>>>>>>>>>>>>>>>> just change my methods from returning void to returning
>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>> parameter and use references to them. Not very
>>>>>>>>>>>>>>>>>> convenient, but passable.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no
>>>>>>>>>>>>>>>>>> consumer 2) when
>>>>>>>>>>>>>>>>>> function returns null, we don't insert it into the
>>>>>>>>>>>>>>>>>> resulting map.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but
>>>>>>>>>>>>>>>>>> workable solution as
>>>>>>>>>>>>>>>>>> a first approximation. And later we can always add to
>>>>>>>>>>>>>>>>>> `Branched`
>>>>>>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes
>>>>>>>>>>>>>>>>>> like Branched
>>>>>>>>>>>>>>>>>> should be built? First of all, it seems that `as` now is
>>>>>>>>>>>>>>>>>> more preferred
>>>>>>>>>>>>>>>>>> than `withName` (although as you probably know it clashes
>>>>>>>>>>>>>>>>>> with Kotlin's
>>>>>>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out that my
>>>>>>>>>>>>>>>>>> Java cannot
>>>>>>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>             Branched.as("foo").withChain(s ->
>>>>>>>>>>>>>>>>>> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>>             Branched.<String, String>as("foo").withChain(s
>>>>>>>>>>>>>>>>>> -> s.mapValues(...)))
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of
>>>>>>>>>>>>>>>>>> `Branched` methods
>>>>>>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>,
>>>>>>>>>>>>>>>>>> ? extends
>>>>>>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 4. And one more. What do you think, do we need that
>>>>>>>>>>>>>>>>>> flexibility:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>>>> chain
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>>>>>> KStream<?
>>>>>>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ??
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the
>>>>>>>>>>>>>>>>>>> map, because there’s only one thing that could be there,
>>>>>>>>>>>>>>>>>>> which is whatever is returned from the chain function.
>>>>>>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way
>>>>>>>>>>>>>>>>>>> this KIP is designed, it would be trivially easy to add
>>>>>>>>>>>>>>>>>>> the “terminal” chain later. As you say, fewer concepts
>>>>>>>>>>>>>>>>>>> leads to an API that is easier to learn.
>>>>>>>>>>>>>>>>>>> 3. We get to side-step the naming of this method.
>>>>>>>>>>>>>>>>>>> Although I didn’t complain about withJavaConsumer, it
>>>>>>>>>>>>>>>>>>> was only because I couldn’t think of a better name.
>>>>>>>>>>>>>>>>>>> Still, it’s somewhat unsatisfying to name a method after
>>>>>>>>>>>>>>>>>>> its argument type, since this provides no information at
>>>>>>>>>>>>>>>>>>> all about what the method does. I was willing to accept
>>>>>>>>>>>>>>>>>>> it because I didn’t have an alternative, but I would be
>>>>>>>>>>>>>>>>>>> happy to skip this method for now to avoid the problem
>>>>>>>>>>>>>>>>>>> until we have more inspiration.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The only con I see is that it makes the code a little
>>>>>>>>>>>>>>>>>>> less ergonomic to write when you don’t want to return
>>>>>>>>>>>>>>>>>>> the result of the chain (such as when the chain is
>>>>>>>>>>>>>>>>>>> terminal), since I’m your example, you have to declare a
>>>>>>>>>>>>>>>>>>> block with a return statement at the end. It’s not
>>>>>>>>>>>>>>>>>>> ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing
>>>>>>>>>>>>>>>>>>> a null result, which would just remove the branch from
>>>>>>>>>>>>>>>>>>> the returned map. It seems nicer than forcing people to
>>>>>>>>>>>>>>>>>>> pick a stream to return when their chain is terminal and
>>>>>>>>>>>>>>>>>>> they don’t want to use the result later.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I guess the only open question is about
>>>>>>>>>>>>>>>>>>>> `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           (1) split a stream and return the substreams
>>>>>>>>>>>>>>>>>>>> for futher processing
>>>>>>>>>>>>>>>>>>>>           (2) split a stream and modify the substreams
>>>>>>>>>>>>>>>>>>>> with in-place method chaining
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>            -> split a stream, modify the substreams, and
>>>>>>>>>>>>>>>>>>>> return the _modified_
>>>>>>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>> split the topology graph at any point. Technically,
>>>>>>>>>>>>>>>>>>>>> it's OK to do both: feed the KStream to a
>>>>>>>>>>>>>>>>>>>>> [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>>> doesn't need the stream in the Map, one simply does
>>>>>>>>>>>>>>>>>>>>> not extract it from there
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> That is of course possible. However, it introduces some
>>>>>>>>>>>>>>>>>>>> "hidded" semantics:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           - using `withChain` I get the modified
>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>           - using `withJavaConsumer` I get the unmodifed
>>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>          From my understanding the original idea of
>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` was to
>>>>>>>>>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>            return null;
>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> However, I am not sure if we should even allow
>>>>>>>>>>>>>>>>>>>> `withChain()` to return
>>>>>>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this
>>>>>>>>>>>>>>>>>>>> case to avoid a `key
>>>>>>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Following this train of through, and if we want to
>>>>>>>>>>>>>>>>>>>> allow the "return
>>>>>>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>> that does not add
>>>>>>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Following your proposal, the semantics of
>>>>>>>>>>>>>>>>>>>> `withJavaConsumer` could also
>>>>>>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>>            s.to();
>>>>>>>>>>>>>>>>>>>>            return s;
>>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely
>>>>>>>>>>>>>>>>>>>> syntactic sugar,
>>>>>>>>>>>>>>>>>>>> while for the first proposal it adds new functionality
>>>>>>>>>>>>>>>>>>>> (if `return null`
>>>>>>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to
>>>>>>>>>>>>>>>>>>>> "hide a
>>>>>>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need
>>>>>>>>>>>>>>>>>>>> to allow `return
>>>>>>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a
>>>>>>>>>>>>>>>>>>>> sub-stream in the Map.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we
>>>>>>>>>>>>>>>>>>>> follow your
>>>>>>>>>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>>> at all? Its
>>>>>>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API
>>>>>>>>>>>>>>>>>>>> is usually
>>>>>>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox,
>>>>>>>>>>>>>>>>>>>>> fortunately we have
>>>>>>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the
>>>>>>>>>>>>>>>>>>>>>> existing ‘branch’
>>>>>>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration
>>>>>>>>>>>>>>>>>>>>> Plan" section.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to
>>>>>>>>>>>>>>>>>>>>>> branching directly
>>>>>>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic
>>>>>>>>>>>>>>>>>>>>> branching]
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic
>>>>>>>>>>>>>>>>>>>>> Branching' section.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka
>>>>>>>>>>>>>>>>>>>>>> Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I
>>>>>>>>>>>>>>>>>>>>> renamed this
>>>>>>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases:
>>>>>>>>>>>>>>>>>>>>>> EITHER using
>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`]
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of
>>>>>>>>>>>>>>>>>>>>>> the `branch`
>>>>>>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that
>>>>>>>>>>>>>>>>>>>>>> using both
>>>>>>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of
>>>>>>>>>>>>>>>>>>>>> `withChain()` will
>>>>>>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the
>>>>>>>>>>>>>>>>>>>>>> returned `Map` of
>>>>>>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a
>>>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>>>> implementation would be to not add the "branch" to the
>>>>>>>>>>>>>>>>>>>>> result map if
>>>>>>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>>> split the topology
>>>>>>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both:
>>>>>>>>>>>>>>>>>>>>> feed the KStream to
>>>>>>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>>> doesn't need the
>>>>>>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it from
>>>>>>>>>>>>>>>>>>>>> there :-)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the
>>>>>>>>>>>>>>>>>>>>> returned map
>>>>>>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs
>>>>>>>>>>>>>>>>>>>>> explicitly set by the
>>>>>>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids.
>>>>>>>>>>>>>>>>>>>>> Dealing with this
>>>>>>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing
>>>>>>>>>>>>>>>>>>>>> exclusions to this
>>>>>>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by
>>>>>>>>>>>>>>>>>>>>> provided
>>>>>>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated.
>>>>>>>>>>>>>>>>>>>>> What if a user
>>>>>>>>>>>>>>>>>>>>> provides both the name of the branch and a
>>>>>>>>>>>>>>>>>>>>> [Java]Consumer? What do they
>>>>>>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or
>>>>>>>>>>>>>>>>>>>>> save it to the map?
>>>>>>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so
>>>>>>>>>>>>>>>>>>>>> maybe just leave
>>>>>>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg,
>>>>>>>>>>>>>>>>>>>>>> CoGroupedKStream,
>>>>>>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy"
>>>>>>>>>>>>>>>>>>>>>> `KGroupedStream`
>>>>>>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a
>>>>>>>>>>>>>>>>>>>>>> breaking change...
>>>>>>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have
>>>>>>>>>>>>>>>>>>>>>>> overloaded
>>>>>>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate`
>>>>>>>>>>>>>>>>>>>>>> and assume you
>>>>>>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe
>>>>>>>>>>>>>>>>>>>>>> rephrase it accordingly
>>>>>>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public
>>>>>>>>>>>>>>>>>>>>>> Interface" section) of
>>>>>>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction
>>>>>>>>>>>>>>>>>>>>>> of `withConsumer()`
>>>>>>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This
>>>>>>>>>>>>>>>>>>>>>> related to John's
>>>>>>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use
>>>>>>>>>>>>>>>>>>>>>>> cases: EITHER using
>>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and
>>>>>>>>>>>>>>>>>>>>>> `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the
>>>>>>>>>>>>>>>>>>>>>> result of `withChain()`
>>>>>>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really
>>>>>>>>>>>>>>>>>>>>>> with the `Consumer`
>>>>>>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and
>>>>>>>>>>>>>>>>>>>>>> `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add
>>>>>>>>>>>>>>>>>>>>>> the "branch" to
>>>>>>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as
>>>>>>>>>>>>>>>>>>>>>> we clearly document
>>>>>>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’
>>>>>>>>>>>>>>>>>>>>>>> confusing; I thought you
>>>>>>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface
>>>>>>>>>>>>>>>>>>>>>>> (which doesn’t make
>>>>>>>>>>>>>>>>>>>>>>> sense, of course). I get that you were referring to
>>>>>>>>>>>>>>>>>>>>>>> the java Consumer
>>>>>>>>>>>>>>>>>>>>>>> interface, but we should still probably to to avoid
>>>>>>>>>>>>>>>>>>>>>>> the ambiguity.
>>>>>>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be
>>>>>>>>>>>>>>>>>>>>>> confused with a "sink
>>>>>>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I
>>>>>>>>>>>>>>>>>>>>>>> think it’s close,
>>>>>>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I believe that now it follows design principles and
>>>>>>>>>>>>>>>>>>>>>>>> takes into account
>>>>>>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and
>>>>>>>>>>>>>>>>>>>>>>>>> have completely
>>>>>>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> This version includes all the previous discussion
>>>>>>>>>>>>>>>>>>>>>>>>> results and follows
>>>>>>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V>
>>>>>>>>>>>>>>>>>>>>>>>>> branched)
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one
>>>>>>>>>>>>>>>>>>>>>>>>> parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't
>>>>>>>>>>>>>>>>>>>>>>>>> need to provide one
>>>>>>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations
>>>>>>>>>>>>>>>>>>>>>>>>> we may use a single
>>>>>>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method
>>>>>>>>>>>>>>>>>>>>>>>>> parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a
>>>>>>>>>>>>>>>>>>>>>>>>> branch, no
>>>>>>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.'
>>>>>>>>>>>>>>>>>>>>>>>>> is expected here
>>>>>>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter
>>>>>>>>>>>>>>>>>>>>>>>>> rule'.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to
>>>>>>>>>>>>>>>>>>>>>>>>>> have you back!
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we
>>>>>>>>>>>>>>>>>>>>>>>>>> already have a
>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems
>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable to me to
>>>>>>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design
>>>>>>>>>>>>>>>>>>>>>>>>>> principles" that we
>>>>>>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in
>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion in
>>>>>>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then
>>>>>>>>>>>>>>>>>>>>>>>>>>> and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this,
>>>>>>>>>>>>>>>>>>>>>>>>>>> because we invested
>>>>>>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>> not feel entitled to
>>>>>>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is
>>>>>>>>>>>>>>>>>>>>>>>>>>> finalized.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and
>>>>>>>>>>>>>>>>>>>>>>>>>>> reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed
>>>>>>>>>>>>>>>>>>>>>>>>>>> branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the
>>>>>>>>>>>>>>>>>>>>>>>>>>> original idea for
>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why
>>>>>>>>>>>>>>>>>>>>>>>>>>> I gave up pushing
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the
>>>>>>>>>>>>>>>>>>>>>>>>>>> problem with the scope of
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new
>>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>                .branch(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>                .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>>>>>>                .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with
>>>>>>>>>>>>>>>>>>>>>>>>>>> this approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright
>>>>>>>>>>>>>>>>>>>>>>>>>>> solution in his post
>>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current
>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional
>>>>>>>>>>>>>>>>>>>>>>>>>>> names seems to be
>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope,
>>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making
>>>>>>>>>>>>>>>>>>>>>>>>>>> your code brittle.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging
>>>>>>>>>>>>>>>>>>>>>>>>>>> purposes. Or,
>>>>>>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on
>>>>>>>>>>>>>>>>>>>>>>>>>>> June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on
>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The idea seems
>>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust
>>>>>>>>>>>>>>>>>>>>>>>>>>> people who are more
>>>>>>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than
>>>>>>>>>>>>>>>>>>>>>>>>>>> me.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status".
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about my proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worlds?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to merge both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your example.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default name but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that neither
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> independently of all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that, although
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split(Named.withName("mysplit"))
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                           .branch(..., ...,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                           .defaultBranch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operator(function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               operator(function, function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> string)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                interface BranchConfig extends
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but it also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Ivan, I’ll definitely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch(predicate, consumer)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 IMO the one trade off to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 question. I don’t know if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the same scope” since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 seems like a perfectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 when the branched streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for the reasons Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution - working around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > On May 2, 2019, at 7:00
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Hello everyone, thank you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Well, I don't think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 LinkedHashMap (no other Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 matters) or `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > In my opinion, the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal is that all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 But 1) we rarely need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 workaround for the scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 'Inlining the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method references instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 tend to split the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > The drawbacks are strong.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers is lost. We have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 handlers in another. This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we misspell a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > - what if we copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > What Michael propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 writing the API in Lua,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "dynamic naming" approach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 beautiful. But in Java we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 identifiers to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Do we have to invent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > And if we do, what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 all the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Earlier in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without "start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the case when we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current KIP is 'clumsier'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 me address both comments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 1) "Start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 for it indeed) is critical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 2) No, dynamic branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Imagine a real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 value (say, RecordType).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*John:if we had to start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > KBranchedStream branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > for (RecordType
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >             branched =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 > 02.05.2019 14:40,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> I also agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> current `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> However, I also don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> was more aligned with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `branch()` statement and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> It makes the code easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> `Predicates` (that is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        ��        >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> An open question is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> specified. Atm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> and the call to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> (what is not the case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> because users can just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> About "inlining": So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> preference. I can see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> that clearly make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> On 5/1/19 6:26 PM, Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Perhaps inlining is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that a lambda with the full
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 it can be a method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The advantage of putting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 (Consumer) together in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>> Ultimately the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch trees will be hard
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> On May 1, 2019, at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> I'm less enthusiastic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> functionality.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>> harder to read as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    >>>>> Also +1 on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> great framework for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Regarding the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> proposal in the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Obviously some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't work without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> associativity as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> directly associates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> provides over the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> The KIP solution is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that it is slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> certainly possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> case anyway, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> access your branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch, and it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solution on top of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> I could also see a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> taken in, branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Pros for that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - accessing branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - no double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - downstream branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to read top to bottom
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> - you can forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API and SortedMap,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> (KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Overall I'm curious
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branched KStream in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> doesn't need to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user.  I'm sort of in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <sophie@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I'd like to +1 what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> method, I agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> trying to alleviate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> able to cleanly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> name->stream), which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> That said, I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> anti-pattern or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I’d like to propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> are three problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 1. If you use it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2. The way in which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the ordering of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 3. It is brittle to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Using associative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be a stronger
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> super K,? super
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Branches are given
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> mapping of names to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> because it’s a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> This solves problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> 2 because you no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> interested in. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> conditional by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> messing with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> One of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> awkward in Java. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> On Tue, Apr 30,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 <john@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> FWIW, I agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> confusing when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like a good name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> operator at all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Tentatively, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> way, we don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> should return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> enforce that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> default branch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> and additionally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> though with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> On Fri, Apr 26,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The intend was to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> The current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> IMHO, this reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> take any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> `branch()` calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Because I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> might be better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> overlap that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Maybe there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 'default' is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a method with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Bummer. Didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Can you add the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> methods? It will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP. For example,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> You did not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -> KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> nice to get your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> would need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> should discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> "incomplete" to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched-KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> On 4/13/19 2:13
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I have updated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Matthias, thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> I can see your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> compatibility?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> deprecated, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Totally agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> reserved word,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Absolutely! I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> please revise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Any new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>> 11.04.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> agrees that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> I had a quick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> are some minor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> It's just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> In the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Also, we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Ie, we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> For the issue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> could we extend
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> second argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> `Function`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> Finally, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> proposal. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On 3/31/19
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> I'm a bit of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> revise the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> buy-in from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> could be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> idea overall.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> forward the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> deprecate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> deprecating,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> same thing is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> just need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> is currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Regarding my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> little sloppy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> passing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> complicated to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>> On Fri, Mar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I read your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> looks better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> going to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> What shall we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Why are you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a novice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Maybe I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> added, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> able to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> The issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> worked around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> [Also, great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> On Mar 28,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> The idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a way to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> You just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> write in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Consider
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> customers who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> store to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> This is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> 'brancher'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> class
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>   private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>   private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>       /*In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> a separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> classes'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> /*Alas,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> without the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Does this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> we need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> [BTW Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> KIP I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> so I will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I tried to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> succeeded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>    pretty
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��    demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> To be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> I will say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> deprecated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> has the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> On Wed,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> OK, let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> First, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> There are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 1. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> PROS: 1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> CONS: The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> 2. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> PROS:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> CONS: We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> called,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> I see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> Still,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> Let me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> 1) OK,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> drop
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> Well,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> tests,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��      >>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> Hello
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��      >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> Does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>    ��            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                 >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>
>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <mj...@apache.org>.
Thanks for updating the KIP!

I am overall happy with it. The proposed default branch numbering (ie,
start with "1" and use "0" for the default branch) is not super
intuitive, but that's a minor question.

Can you maybe explain your reasoning? Why not just start with "0" and
the default branch gets the next index as any other branch (ie, this is
how the currently returned array indexes the branches, too)?


-Matthias

On 6/1/20 5:29 AM, Ivan Ponomarev wrote:
> Hi all,
> 
> I updated the KIP with what we have discussed, see 'How the resulting
> Map is formed' section. Four bullet points to define the rules for Map
> keys, and four bullet points to define the rules for Map values.
> 
> I also added the overloads to `Branched` in order to accept Consumers as
> parameters.
> 
> Regards,
> 
> Ivan
> 
> 
> 01.06.2020 8:56, Matthias J. Sax пишет:
>> For naming:
>>
>> (1) _If_ people specify a name for split() _and_ branch(), we _must_
>> have a pulic API contract. Otherwise it would be very bad user
>> experience if we are allowed to change the names (eg, do from "we don't
>> insert a `-` to we insert a `-`). -- We also need to clarify what
>> happens is only split() _or_ branch() has a name specified, ie, is there
>> a contract for this case or not?
>>
>> (2) If we have a contract for how split() and branch() names are put
>> together, we can also apply it to auto-generated names (this seems to be
>> just consistent).
>>
>> (3) Do we want to have a contract for auto-generated names? This is
>> debatable, but I personally don't see any harm in having a contract.
>> Also, if people want to access the Map, they can easily do it without
>> specifying names if there is a contract -- if there is no contract, we
>> force people to specify names to use the Map, what seems to make the API
>> harder to use.
>>
>> But we can of course discuss furhter on the dev list.
>>
>>
>> -Matthias
>>
>> On 5/30/20 3:53 PM, Ivan Ponomarev wrote:
>>> Hi Mathhias,
>>>
>>> I received your letter just after I sent mine concerning nulls and
>>> reintroducing Consumer overload. So, if you're fine with omitting nulls
>>> and introducing a with(Consumer) overload, it's great!
>>>
>>> Concerning the naming algorithm. Well, actually what you propose is very
>>> clear and this is what I was going to implement anyway. But are you sure
>>> that we will never want to change this algorithm? Here I trust your
>>> judgement, since you understand the KStreams API's evolution much more
>>> than I do.
>>>
>>>
>>> 31.05.2020 0:50, Matthias J. Sax пишет:
>>>> Ivan, John,
>>>>
>>>> thanks for the details.
>>>>
>>>> I guess I can be convinced in either direction with regard to `null`
>>>> handling. But I am less worried about it, because I assume that a Java
>>>> programmer understand the ambiguty of `get(key) == null` -- that's why
>>>> there is `containsKey()`. In the end it might be a minor detail and as
>>>> long as it's well documented all discussed approaches seem fine.
>>>> Including the idea to re-introduce the `Consumer` overload.
>>>>
>>>> It's Ivan's KIP so I am fine with whatever he picks :)
>>>>
>>>>
>>>> About the naming: Not sure why we don't want to make the naming a
>>>> public
>>>> contract? I guess for split() we could default to the empty String, and
>>>> for `branch()` we could just add a counter by default?
>>>>
>>>> With no passed names at all, we just generte branche names "1", "2",
>>>> "3", etc... With only`split(Named.as("foo-"))` (but not branch
>>>> names) we
>>>> generate "foo-1", "foo-2", etc.
>>>>
>>>> If any branch has a dedicated name, the counting is preserved but just
>>>> not used int he name for this brach:
>>>>
>>>> split("foo-")
>>>>     .brach()
>>>>     .brach(Branched.as("bar")
>>>>     .defaultBranch()
>>>>
>>>> would generate "foo-1", "foo-bar", "foo-3"?
>>>>
>>>> Thoughts?
>>>>
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 5/29/20 1:20 PM, John Roesler wrote:
>>>>> Hi all,
>>>>>
>>>>> Right, that was my suggestion; sorry for the confusion. I was thinking
>>>>> that adding null as a value to maps is generally dubious, for which
>>>>> reason a lot of Map implementations actually don't allow it at all.
>>>>>
>>>>> The reason is that it creates ambiguity, since a lot of code treats
>>>>> "map.get(key) == null" as meaning the key is not in the map, but
>>>>> if we place a null value in the map (presuming we don't immediately
>>>>> get an exception), then you face weird contradictions, like
>>>>> get(key) == null, but containsKey(key) can be either true or false,
>>>>> and
>>>>> you may or may not see it while iterating. Which would violate the
>>>>> expectations of many Java programs. You can just take a look at
>>>>> the AK codebase, and you'll find many occurrences where we
>>>>> assume a null mapping means the map doesn't contain the key.
>>>>>
>>>>> Of course, the practical question, which Ivan brought up, is also
>>>>> a good one. What would you actually do with these null values
>>>>> in the map, besides get an NPE ?
>>>>>
>>>>> I'm not sure why I didn't think of this before, but an alternative
>>>>> to this debate is to go back to adding Consumer into the API, but
>>>>> with the same method name, withChain(Consumer<KStream<K,V>>).
>>>>>
>>>>> Now that we have only static methods, it's less confusing because it's
>>>>> not possible to choose _both_ the Function<KStream,KStream> _and_
>>>>> the Consumer<KStream> in the same branch (which was one of the
>>>>> main drawbacks of the original plan wrt withChain and
>>>>> withJavaConsumer as builder methods.
>>>>>
>>>>> I.e., I was previously concerned about:
>>>>> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2)
>>>>>
>>>>>
>>>>> It's just a little confusing trying to reason about how this is all
>>>>> going to wire up.
>>>>>
>>>>> But with only the static methods, users have only three methods,
>>>>> and they are all clear:
>>>>> 1. Branched.as(name): just names the branch, the branch-predicated
>>>>> stream is the value of the map
>>>>> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the
>>>>> branch, applies the transformation after the branch predicate, and
>>>>> the transformed stream is the value of the map (null is not permitted)
>>>>> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch,
>>>>> passes the predicated stream in to the Consumer function and omits
>>>>> the branch from the map
>>>>>
>>>>> Sorry for digging this option up again, but it's starting to look more
>>>>> attractive after we dropped the builder functions, and I think it also
>>>>> resolves the "null" concern.
>>>>>
>>>>> Thoughts?
>>>>> -John
>>>>>
>>>>>
>>>>> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>>>>>> Hi Matthias!
>>>>>>
>>>>>> Thanks for your reply!
>>>>>>
>>>>>> (1)
>>>>>>
>>>>>>    > Do you imply that there won't be a `name -> null` entry in the
>>>>>> `Map`
>>>>>> for the branch? If yes, I am wondering why?
>>>>>>
>>>>>> Ah, I re-read the discussion and found out that omitting null entries
>>>>>> was John's idea, not yours :-)
>>>>>>
>>>>>> Anyway. As you remember, I was against the 'swallowing' of
>>>>>> branches by
>>>>>> Consumer. I had a feeling that there are cases when I would like
>>>>>> both to
>>>>>> consume a branch and post-process it in a Map handler.
>>>>>>
>>>>>> But then the idea of getting rid of `withJavaConsumer` came up. And
>>>>>> then
>>>>>> we realized that we should handle nulls somehow.
>>>>>>
>>>>>> So it looked like taking the best from both approaches: if we want to
>>>>>> omit a branch in the resulting map, we just return null, thus
>>>>>> emulating
>>>>>> a 'swallowing consumer'. If we want to include a branch, we return
>>>>>> it.
>>>>>> It provides the full control for a user, and it spares the user from
>>>>>> null-checking in their code.
>>>>>>
>>>>>>    > My reasoning is, that it's very clear that `name -> null`
>>>>>> would be
>>>>>> there, because the user code did execute `return null`.
>>>>>>
>>>>>> My reasoning is following. The way such lambdas are written, it's
>>>>>> hard
>>>>>> to return null unintentionally. And if a user returns null
>>>>>> intentionally, what `name->null` in the resulting Map can be good
>>>>>> for?
>>>>>> It's either NPE (which is clearly not intended) or something that
>>>>>> should
>>>>>> be null-checked and skipped. Ok, let's do it!
>>>>>>
>>>>>> But, I'm ready to be convinced to drop this rule. It doesn't seem
>>>>>> to me
>>>>>> to be a principal thing.
>>>>>>
>>>>>> (2)
>>>>>>
>>>>>>    > I think it would be good if the KIP could explains the intended
>>>>>> default naming schema. This is important because the naming schema
>>>>>> must
>>>>>> be part of the API contract; otherwise, users cannot rely on the
>>>>>> naming
>>>>>> when trying to use the returned `Map`.
>>>>>>
>>>>>> Well, in fact, I intentionally didn't include any naming schema in
>>>>>> the
>>>>>> KIP. And it's done exactly for the purpose that a user won't be
>>>>>> able to
>>>>>> rely on default naming. If they want to use certain branches from the
>>>>>> Map, they will devise their own naming strategy that will never
>>>>>> change
>>>>>> and will never let them down.
>>>>>>
>>>>>> If they don't mind what's in a Map, if they are looking up the Map
>>>>>> just
>>>>>> for debugging, or if they want to treat a Map just like a
>>>>>> Collection of
>>>>>> values (a possible use case!), it doesn't matter how the branches
>>>>>> are named.
>>>>>>
>>>>>> This way we are giving ourselves the full freedom to change the
>>>>>> naming
>>>>>> schema afterwards without violating the API contract.
>>>>>>
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>>>>>> Thanks for updating the KIP!
>>>>>>>
>>>>>>> Using covariant generics is a good idea! I am also fine with only
>>>>>>> using
>>>>>>> static method in `Branched` for now, as it's only two parameters and
>>>>>>> thus not too many overloads.
>>>>>>>
>>>>>>>
>>>>>>> (1) What is unclear to me is, what you exaclty mean by:
>>>>>>>
>>>>>>>> If a function returns `null`, its result is omitted.
>>>>>>>
>>>>>>> Do you imply that there won't be a `name -> null` entry in the
>>>>>>> `Map` for
>>>>>>> the branch? If yes, I am wondering why?
>>>>>>>
>>>>>>> My previous argument was to omit an entry only for the
>>>>>>> `withJavaConsumer()` case, because the return type is "void" (and
>>>>>>> thus
>>>>>>> it's unclear what should be added and it would introduce
>>>>>>> inconsistencies
>>>>>>> in the `Map`).
>>>>>>>
>>>>>>> However, for a `Function` that returns a `KStream`, we could
>>>>>>> actually
>>>>>>> just add a `name -> null` entry if `null` is returned. My reasoning
>>>>>>> is,
>>>>>>> that it's very clear that `name -> null` would be there, because the
>>>>>>> user code did execute `return null`. Adding a `name -> null`
>>>>>>> entry for
>>>>>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>>>>>> exceptional case for handling `null`).
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>>
>>>>>>> (2) For `Branched.as()` the KIP says:
>>>>>>>
>>>>>>>> sets the name of the branch (auto-generated by default, when split
>>>>>>>> operation is named, then the names are index-suffixed)
>>>>>>>
>>>>>>> I think it would be good if the KIP could explains the intended
>>>>>>> default
>>>>>>> naming schema. This is important because the naming schema must be
>>>>>>> part
>>>>>>> of the API contract; otherwise, users cannot rely on the naming when
>>>>>>> trying to use the returned `Map`.
>>>>>>>
>>>>>>> Similalry, it should be explained how names are generated if
>>>>>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations
>>>>>>> how the
>>>>>>> API can be mixed and matched and it's unclear atm how it would
>>>>>>> work in
>>>>>>> detail.
>>>>>>>
>>>>>>>
>>>>>>> Thanks!
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>>>>>> Hi Ivan,
>>>>>>>>
>>>>>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>>>>>> that have been raised in the discussion so far have been
>>>>>>>> addressed. And it's been a while since anyone raised a new
>>>>>>>> concern. At this point, it seems like a good time to start
>>>>>>>> the VOTE thread.
>>>>>>>>
>>>>>>>> Sometimes, the vote thread will trigger new people to look
>>>>>>>> into the KIP, and they may raise new concerns, but it's not a
>>>>>>>> problem. We'll just address those lingering concerns if there
>>>>>>>> are any, until you have all the votes you need.
>>>>>>>>
>>>>>>>> Thanks again for the contribution!
>>>>>>>>
>>>>>>>> -John
>>>>>>>>
>>>>>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>>>>>
>>>>>>>>> 1. Per Mathhias's suggestion, if a chain function returns null,
>>>>>>>>> the
>>>>>>>>> respective result if omitted in the resulting Map.
>>>>>>>>>
>>>>>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>>>>>
>>>>>>>>> 3. `Branched` class has only three static methods with all the
>>>>>>>>> possible
>>>>>>>>> combinations of parameters.
>>>>>>>>>
>>>>>>>>> 4. Chain function is defined 'fully covariant', let's see if we
>>>>>>>>> can
>>>>>>>>> implement it this way :-))
>>>>>>>>>
>>>>>>>>> + code example updates and minor edits.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Since this is my first KIP, I'm not sure what should I do next. I
>>>>>>>>> feel
>>>>>>>>> that we talked over all the details and the consensus is reached.
>>>>>>>>> Is it
>>>>>>>>> OK to call for VOTE now or is it better to wait for more feedback?
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>>>>>> Sounds good to me, Ivan!
>>>>>>>>>> -John
>>>>>>>>>>
>>>>>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>>>>>> John,
>>>>>>>>>>>
>>>>>>>>>>> ---------------------
>>>>>>>>>>>
>>>>>>>>>>>      > I'm sorry things have been dragging out a little, but I
>>>>>>>>>>> have the
>>>>>>>>>>> sense we're very close to the end of this discussion, which is
>>>>>>>>>>> exciting.
>>>>>>>>>>>
>>>>>>>>>>> We are certainly moving forward!  And I'm not in a hurry at
>>>>>>>>>>> all. As I
>>>>>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>>>>>> KafkaStreamBrancher -- the implementation of the first,
>>>>>>>>>>> rejected version
>>>>>>>>>>> of this KIP. It's inferior to what we are discussing here, but
>>>>>>>>>>> it does
>>>>>>>>>>> its work. So it's worth to design this KIP really, really well.
>>>>>>>>>>> And by
>>>>>>>>>>> the way, from this discussion I'm learning the good API
>>>>>>>>>>> designing
>>>>>>>>>>> process. For me it has a value per se :-))
>>>>>>>>>>>
>>>>>>>>>>> ----------------------
>>>>>>>>>>>
>>>>>>>>>>>      >>   > I'd wonder whether we need the non-static builders
>>>>>>>>>>> (like withChain).
>>>>>>>>>>>      >>   > Do they provide any benefit over just using the
>>>>>>>>>>> right static
>>>>>>>>>>> factory?
>>>>>>>>>>>      >
>>>>>>>>>>>      > I don't have a strong feeling, either. It seems nice to
>>>>>>>>>>> offer a better
>>>>>>>>>>>      > type inference experience than what we get with
>>>>>>>>>>> Materialized, by
>>>>>>>>>>>      > offering the static method that takes both name and
>>>>>>>>>>> chain.
>>>>>>>>>>>      > Given that, there doesn't seem to be a good reason to
>>>>>>>>>>> also offer the
>>>>>>>>>>>      > non-static builder-style methods, so I guess I'd prefer
>>>>>>>>>>> to drop them.
>>>>>>>>>>>
>>>>>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I
>>>>>>>>>>> found out
>>>>>>>>>>> an interesting fact about type inference rules in Java. Funny
>>>>>>>>>>> thing is
>>>>>>>>>>> that although we need to explicitly set types in a chain like
>>>>>>>>>>> this
>>>>>>>>>>>
>>>>>>>>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>>>>>>>>>>
>>>>>>>>>>> (otherwise it won't compile), the composition of static method
>>>>>>>>>>> calls
>>>>>>>>>>> works just fine, all the types are being calculated correctly:
>>>>>>>>>>>
>>>>>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>>>>>>>>>>
>>>>>>>>>>> As I was told, for type inference there is difference between
>>>>>>>>>>> qualifiers
>>>>>>>>>>> and arguments, 'you go up if you are argument but stop if you
>>>>>>>>>>> are
>>>>>>>>>>> qualifier'. And it also seems that we should not bet on any
>>>>>>>>>>> future
>>>>>>>>>>> improvements in Java type inference here.
>>>>>>>>>>>
>>>>>>>>>>> So,
>>>>>>>>>>>
>>>>>>>>>>> 1) I think we that in this KIP we should provide three static
>>>>>>>>>>> methods
>>>>>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function,
>>>>>>>>>>> String)`, and
>>>>>>>>>>> drop any non-static ones.
>>>>>>>>>>>
>>>>>>>>>>> 2) If anything else will be ever needed, we can easily add
>>>>>>>>>>> anything.
>>>>>>>>>>> Maybe this can be done in a process of refinement of all the
>>>>>>>>>>> parameter
>>>>>>>>>>> classes.
>>>>>>>>>>>
>>>>>>>>>>> ----------------------------
>>>>>>>>>>>
>>>>>>>>>>>      > we may as well hope for the best, and propose the "fully
>>>>>>>>>>>      > covariant" definition for now.
>>>>>>>>>>>
>>>>>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>>>>>
>>>>>>>>>>> ----------------------------
>>>>>>>>>>>
>>>>>>>>>>>      >> Good question, I already thought about it and rejected
>>>>>>>>>>> the idea....
>>>>>>>>>>>      > (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>      >  "Worst" case scenario: someone
>>>>>>>>>>>      > else will wish the return type is something different,
>>>>>>>>>>> and we'll go
>>>>>>>>>>>      > through a painless deprecation transition to change it
>>>>>>>>>>> later.
>>>>>>>>>>>
>>>>>>>>>>> Of course, we cant' predict all the ways people are going to
>>>>>>>>>>> use it.
>>>>>>>>>>>      From my own humble experience with Kafka Streams, the worst
>>>>>>>>>>> scenario is
>>>>>>>>>>> unlikely. Split is split, transform is transform, too much
>>>>>>>>>>> flexibility
>>>>>>>>>>> is often evil.
>>>>>>>>>>>
>>>>>>>>>>> ------------------------
>>>>>>>>>>>
>>>>>>>>>>> So it seems that we are close to the consensus. Two things to
>>>>>>>>>>> be altered
>>>>>>>>>>> in the current version of KIP:
>>>>>>>>>>>
>>>>>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>>>>>
>>>>>>>>>>> Any ideas / questions / objections?
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>>>>>
>>>>>>>>>>>> I'm sorry things have been dragging out a little, but I have
>>>>>>>>>>>> the sense
>>>>>>>>>>>> we're very close to the end of this discussion, which is
>>>>>>>>>>>> exciting.
>>>>>>>>>>>>
>>>>>>>>>>>>>       > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>>       > Do they provide any benefit over just using the right
>>>>>>>>>>>>> static factory?
>>>>>>>>>>>>>
>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>> builders can
>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>
>>>>>>>>>>>> I don't have a strong feeling, either. It seems nice to offer
>>>>>>>>>>>> a better
>>>>>>>>>>>> type inference experience than what we get with
>>>>>>>>>>>> Materialized, by
>>>>>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>>>>>
>>>>>>>>>>>> Given that, there doesn't seem to be a good reason to also
>>>>>>>>>>>> offer the
>>>>>>>>>>>> non-static builder-style methods, so I guess I'd prefer to
>>>>>>>>>>>> drop them.
>>>>>>>>>>>>
>>>>>>>>>>>> I'll defer to Matthias, if he has a chance to consider whether
>>>>>>>>>>>> it's more
>>>>>>>>>>>> valuable to stick with the existing pattern or break the
>>>>>>>>>>>> pattern to offer
>>>>>>>>>>>> a better experience.
>>>>>>>>>>>>
>>>>>>>>>>>>>       > You might as well propose the “ideal” API in the KIP,
>>>>>>>>>>>>> which is the
>>>>>>>>>>>>>       > covariant typed function
>>>>>>>>>>>>>
>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>> implementation
>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>
>>>>>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been that
>>>>>>>>>>>> we just
>>>>>>>>>>>> won't know what works until we really try it, and try it in
>>>>>>>>>>>> several different
>>>>>>>>>>>> ways. Still, we may as well hope for the best, and propose the
>>>>>>>>>>>> "fully
>>>>>>>>>>>> covariant" definition for now.
>>>>>>>>>>>>
>>>>>>>>>>>>>       > Is it necessary to restrict the result key and value
>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>>       > same as the inputs?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>> idea....
>>>>>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>>
>>>>>>>>>>>> That's fair! It's your KIP, after all. I think I might have
>>>>>>>>>>>> made a different
>>>>>>>>>>>> call here, but I think this choice is fine. "Worst" case
>>>>>>>>>>>> scenario: someone
>>>>>>>>>>>> else will wish the return type is something different, and
>>>>>>>>>>>> we'll go
>>>>>>>>>>>> through a painless deprecation transition to change it later.
>>>>>>>>>>>> Thanks
>>>>>>>>>>>> to the clean design of your API, this doesn't seem to bad.
>>>>>>>>>>>> And, of
>>>>>>>>>>>> course, you've actually been using similar functionality
>>>>>>>>>>>> already, so it
>>>>>>>>>>>> seems we should trust your intuition.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> -John
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>
>>>>>>>>>>>>>       > I'd wonder whether we need the non-static builders
>>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>> Do they provide any benefit over just using the right static
>>>>>>>>>>>>> factory?
>>>>>>>>>>>>>
>>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>>> builders can
>>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>>
>>>>>>>>>>>>>       > You might as well propose the “ideal” API in the KIP,
>>>>>>>>>>>>> which is the
>>>>>>>>>>>>> covariant typed function
>>>>>>>>>>>>>
>>>>>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>>> implementation
>>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>>
>>>>>>>>>>>>>       > Is it necessary to restrict the result key and value
>>>>>>>>>>>>> types to be the
>>>>>>>>>>>>> same as the inputs?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>>> idea.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Look, if we want to keep `withChain`'s function optional,
>>>>>>>>>>>>> then we must
>>>>>>>>>>>>> keep the result key and value types the same. Because for
>>>>>>>>>>>>> now, the
>>>>>>>>>>>>> default value for the 'chain function' is Function.identity().
>>>>>>>>>>>>>
>>>>>>>>>>>>> Of course, we can make the 'chain function' required. But I
>>>>>>>>>>>>> think this
>>>>>>>>>>>>> is not what `split` method is for. `split` is for
>>>>>>>>>>>>> splitting, not
>>>>>>>>>>>>> transforming, and `chainFunction` in most of the cases should
>>>>>>>>>>>>> be either
>>>>>>>>>>>>> a consumer or the identity function.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past
>>>>>>>>>>>>>> that generic type inference problem is to offer a static
>>>>>>>>>>>>>> factory method that takes all the options (both name and
>>>>>>>>>>>>>> function).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I’m happy with adding that option, although if we have all
>>>>>>>>>>>>>> possible combinations available as static methods, then I’d
>>>>>>>>>>>>>> wonder whether we need the non-static builders (like
>>>>>>>>>>>>>> withChain). Do they provide any benefit over just using the
>>>>>>>>>>>>>> right static factory?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type
>>>>>>>>>>>>>> bounds works algebraically, but I’m not sure whether the
>>>>>>>>>>>>>> limitations of Java type inference will actually let you
>>>>>>>>>>>>>> pass in all the different functions you would want to.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I recall Bruno and I wrestling with this when he added
>>>>>>>>>>>>>> Transform operators. In retrospect, this is a problem we can
>>>>>>>>>>>>>> iron out when we write tests in the PR. You might as well
>>>>>>>>>>>>>> propose the “ideal” API in the KIP, which is the covariant
>>>>>>>>>>>>>> typed function:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>> KStream<? extends K, ? extends V>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3. Actually, I have a new question about the types: Is it
>>>>>>>>>>>>>> necessary to restrict the result key and value types to be
>>>>>>>>>>>>>> the same as the inputs? I.e., shouldn’t the result types be
>>>>>>>>>>>>>> K1,V1?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>> Hello John,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > Perhaps it would be better to stick with "as"
>>>>>>>>>>>>>>> for now
>>>>>>>>>>>>>>>        > and just file a Jira to switch them all at the same
>>>>>>>>>>>>>>> time [for
>>>>>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin
>>>>>>>>>>>>>>> they have a
>>>>>>>>>>>>>>> standard workaround
>>>>>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> So actually this should be a very low priority issue, if an
>>>>>>>>>>>>>>> issue at
>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > I don't understand how your new proposed
>>>>>>>>>>>>>>>        > methods would work any differently than the ones
>>>>>>>>>>>>>>> you already
>>>>>>>>>>>>>>>        > had proposed in the KIP. It seems like you'd still
>>>>>>>>>>>>>>> have to provide
>>>>>>>>>>>>>>>        > the generic type parameters on the first static
>>>>>>>>>>>>>>> factory call. Can you
>>>>>>>>>>>>>>>        > explain how your new interface proposal differs
>>>>>>>>>>>>>>> from the existing KIP?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> In the KIP, I didn't clarify what methods should be static.
>>>>>>>>>>>>>>> Now I
>>>>>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The overloaded `with` version that provides both Function
>>>>>>>>>>>>>>> and name can
>>>>>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        > Regarding making the K,V types covariant also, yes,
>>>>>>>>>>>>>>> that would indeed
>>>>>>>>>>>>>>>        > be nice, but I'm not sure it will actually work.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> static KStream<String, Integer> func(KStream<String,
>>>>>>>>>>>>>>> Number> s) {
>>>>>>>>>>>>>>>                return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>>>>>            s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>>>>>                       //Won't compile!!
>>>>>>>>>>>>>>>                       Branched.with(Me::func));
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The simple workaround here is to change `func`'s return
>>>>>>>>>>>>>>> type from
>>>>>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> [On the other hand, we already agreed to remove
>>>>>>>>>>>>>>> `withJavaConsumer` from
>>>>>>>>>>>>>>> `Branched`, so during code migration I will have to modify
>>>>>>>>>>>>>>> my functions'
>>>>>>>>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        >  the map you're returning is Map<K,V>, and of
>>>>>>>>>>>>>>> course a K is not the
>>>>>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think what you actually meant here is that KStream<?
>>>>>>>>>>>>>>> extends K, ?
>>>>>>>>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K,
>>>>>>>>>>>>>>> V>>. This
>>>>>>>>>>>>>>> particularly is not a problem, since KStream<? extends K, ?
>>>>>>>>>>>>>>> extends V>
>>>>>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put
>>>>>>>>>>>>>>> to the map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard
>>>>>>>>>>>>>>> types. So maybe
>>>>>>>>>>>>>>> for now it's better to just admit that API is not
>>>>>>>>>>>>>>> absolutely perfect and
>>>>>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that
>>>>>>>>>>>>>>>> "as" would
>>>>>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from
>>>>>>>>>>>>>>>> convention
>>>>>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The convention is that "as(String name)" is used for the
>>>>>>>>>>>>>>>> static factory
>>>>>>>>>>>>>>>> method, whereas "withName(String name)" is an instance
>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to
>>>>>>>>>>>>>>>> avoid "as"
>>>>>>>>>>>>>>>> for compatibility with Kotlin, I might suggest
>>>>>>>>>>>>>>>> "fromName(String name)",
>>>>>>>>>>>>>>>> although it's somewhat dubious, since all the other
>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick with
>>>>>>>>>>>>>>>> "as" for now
>>>>>>>>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>>>>>> Regarding the type inference problem, yes, it's a blemish
>>>>>>>>>>>>>>>> on all of our
>>>>>>>>>>>>>>>> configuraion objects. The problem is that Java infers the
>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>> based on the _first_ method in the chain. While it does
>>>>>>>>>>>>>>>> consider what
>>>>>>>>>>>>>>>> the recipient of the method result wants, it only
>>>>>>>>>>>>>>>> considers the _next_
>>>>>>>>>>>>>>>> recipient.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>>>>>>>>>>> Branched<String,String> variable, java infers the type
>>>>>>>>>>>>>>>> correctly. But
>>>>>>>>>>>>>>>> when the "next recipient" is a chained method call, like
>>>>>>>>>>>>>>>> "withChain",
>>>>>>>>>>>>>>>> then the chained method doesn't bound the type (by
>>>>>>>>>>>>>>>> definition,
>>>>>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java
>>>>>>>>>>>>>>>> will take
>>>>>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be
>>>>>>>>>>>>>>>> revised anymore.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, since it
>>>>>>>>>>>>>>>> doesn't seem
>>>>>>>>>>>>>>>> that hard to recursively consider the entire context when
>>>>>>>>>>>>>>>> inferring the
>>>>>>>>>>>>>>>> generic type parameters, but this is what we have to work
>>>>>>>>>>>>>>>> with.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> To be honest, though, I don't understand how your new
>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>> methods would work any differently than the ones you
>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still have to
>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>> the generic type parameters on the first static factory
>>>>>>>>>>>>>>>> call. Can you
>>>>>>>>>>>>>>>> explain how your new interface proposal differs from the
>>>>>>>>>>>>>>>> existing KIP?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that
>>>>>>>>>>>>>>>> would indeed
>>>>>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You might
>>>>>>>>>>>>>>>> want to give it a
>>>>>>>>>>>>>>>> try. In the past, we've run into soe truly strange
>>>>>>>>>>>>>>>> interactions between the
>>>>>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner
>>>>>>>>>>>>>>>> classes) in
>>>>>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Another issue is that the value type of the map you're
>>>>>>>>>>>>>>>> returning is
>>>>>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends
>>>>>>>>>>>>>>>> K", so it
>>>>>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that
>>>>>>>>>>>>>>>>> getting rid of
>>>>>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as
>>>>>>>>>>>>>>>>> possible, but not
>>>>>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to
>>>>>>>>>>>>>>>>> implement examples
>>>>>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1. Having to return something from lambda is not a very
>>>>>>>>>>>>>>>>> big deal.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use
>>>>>>>>>>>>>>>>> method references
>>>>>>>>>>>>>>>>> for already written stream consumers, but then I realized
>>>>>>>>>>>>>>>>> that I can
>>>>>>>>>>>>>>>>> just change my methods from returning void to returning
>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>> parameter and use references to them. Not very
>>>>>>>>>>>>>>>>> convenient, but passable.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no
>>>>>>>>>>>>>>>>> consumer 2) when
>>>>>>>>>>>>>>>>> function returns null, we don't insert it into the
>>>>>>>>>>>>>>>>> resulting map.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but
>>>>>>>>>>>>>>>>> workable solution as
>>>>>>>>>>>>>>>>> a first approximation. And later we can always add to
>>>>>>>>>>>>>>>>> `Branched`
>>>>>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes
>>>>>>>>>>>>>>>>> like Branched
>>>>>>>>>>>>>>>>> should be built? First of all, it seems that `as` now is
>>>>>>>>>>>>>>>>> more preferred
>>>>>>>>>>>>>>>>> than `withName` (although as you probably know it clashes
>>>>>>>>>>>>>>>>> with Kotlin's
>>>>>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out that my
>>>>>>>>>>>>>>>>> Java cannot
>>>>>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>            Branched.as("foo").withChain(s ->
>>>>>>>>>>>>>>>>> s.mapValues(...)))
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>>            Branched.<String, String>as("foo").withChain(s
>>>>>>>>>>>>>>>>> -> s.mapValues(...)))
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of
>>>>>>>>>>>>>>>>> `Branched` methods
>>>>>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>,
>>>>>>>>>>>>>>>>> ? extends
>>>>>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 4. And one more. What do you think, do we need that
>>>>>>>>>>>>>>>>> flexibility:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>>> chain
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>>>>> KStream<?
>>>>>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> ??
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the
>>>>>>>>>>>>>>>>>> map, because there’s only one thing that could be there,
>>>>>>>>>>>>>>>>>> which is whatever is returned from the chain function.
>>>>>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way
>>>>>>>>>>>>>>>>>> this KIP is designed, it would be trivially easy to add
>>>>>>>>>>>>>>>>>> the “terminal” chain later. As you say, fewer concepts
>>>>>>>>>>>>>>>>>> leads to an API that is easier to learn.
>>>>>>>>>>>>>>>>>> 3. We get to side-step the naming of this method.
>>>>>>>>>>>>>>>>>> Although I didn’t complain about withJavaConsumer, it
>>>>>>>>>>>>>>>>>> was only because I couldn’t think of a better name.
>>>>>>>>>>>>>>>>>> Still, it’s somewhat unsatisfying to name a method after
>>>>>>>>>>>>>>>>>> its argument type, since this provides no information at
>>>>>>>>>>>>>>>>>> all about what the method does. I was willing to accept
>>>>>>>>>>>>>>>>>> it because I didn’t have an alternative, but I would be
>>>>>>>>>>>>>>>>>> happy to skip this method for now to avoid the problem
>>>>>>>>>>>>>>>>>> until we have more inspiration.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The only con I see is that it makes the code a little
>>>>>>>>>>>>>>>>>> less ergonomic to write when you don’t want to return
>>>>>>>>>>>>>>>>>> the result of the chain (such as when the chain is
>>>>>>>>>>>>>>>>>> terminal), since I’m your example, you have to declare a
>>>>>>>>>>>>>>>>>> block with a return statement at the end. It’s not
>>>>>>>>>>>>>>>>>> ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing
>>>>>>>>>>>>>>>>>> a null result, which would just remove the branch from
>>>>>>>>>>>>>>>>>> the returned map. It seems nicer than forcing people to
>>>>>>>>>>>>>>>>>> pick a stream to return when their chain is terminal and
>>>>>>>>>>>>>>>>>> they don’t want to use the result later.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I guess the only open question is about
>>>>>>>>>>>>>>>>>>> `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>          (1) split a stream and return the substreams
>>>>>>>>>>>>>>>>>>> for futher processing
>>>>>>>>>>>>>>>>>>>          (2) split a stream and modify the substreams
>>>>>>>>>>>>>>>>>>> with in-place method chaining
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           -> split a stream, modify the substreams, and
>>>>>>>>>>>>>>>>>>> return the _modified_
>>>>>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>> split the topology graph at any point. Technically,
>>>>>>>>>>>>>>>>>>>> it's OK to do both: feed the KStream to a
>>>>>>>>>>>>>>>>>>>> [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>> doesn't need the stream in the Map, one simply does
>>>>>>>>>>>>>>>>>>>> not extract it from there
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> That is of course possible. However, it introduces some
>>>>>>>>>>>>>>>>>>> "hidded" semantics:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>          - using `withChain` I get the modified
>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>          - using `withJavaConsumer` I get the unmodifed
>>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         From my understanding the original idea of
>>>>>>>>>>>>>>>>>>> `withJavaConsumer` was to
>>>>>>>>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>           s.to();
>>>>>>>>>>>>>>>>>>>           return null;
>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> However, I am not sure if we should even allow
>>>>>>>>>>>>>>>>>>> `withChain()` to return
>>>>>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this
>>>>>>>>>>>>>>>>>>> case to avoid a `key
>>>>>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Following this train of through, and if we want to
>>>>>>>>>>>>>>>>>>> allow the "return
>>>>>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>> that does not add
>>>>>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Following your proposal, the semantics of
>>>>>>>>>>>>>>>>>>> `withJavaConsumer` could also
>>>>>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>>           s.to();
>>>>>>>>>>>>>>>>>>>           return s;
>>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely
>>>>>>>>>>>>>>>>>>> syntactic sugar,
>>>>>>>>>>>>>>>>>>> while for the first proposal it adds new functionality
>>>>>>>>>>>>>>>>>>> (if `return null`
>>>>>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to
>>>>>>>>>>>>>>>>>>> "hide a
>>>>>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need
>>>>>>>>>>>>>>>>>>> to allow `return
>>>>>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a
>>>>>>>>>>>>>>>>>>> sub-stream in the Map.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we
>>>>>>>>>>>>>>>>>>> follow your
>>>>>>>>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>>> at all? Its
>>>>>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API
>>>>>>>>>>>>>>>>>>> is usually
>>>>>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox,
>>>>>>>>>>>>>>>>>>>> fortunately we have
>>>>>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the
>>>>>>>>>>>>>>>>>>>>> existing ‘branch’
>>>>>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration
>>>>>>>>>>>>>>>>>>>> Plan" section.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to
>>>>>>>>>>>>>>>>>>>>> branching directly
>>>>>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic
>>>>>>>>>>>>>>>>>>>> branching]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic
>>>>>>>>>>>>>>>>>>>> Branching' section.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka
>>>>>>>>>>>>>>>>>>>>> Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I
>>>>>>>>>>>>>>>>>>>> renamed this
>>>>>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases:
>>>>>>>>>>>>>>>>>>>>> EITHER using
>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of
>>>>>>>>>>>>>>>>>>>>> the `branch`
>>>>>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that
>>>>>>>>>>>>>>>>>>>>> using both
>>>>>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of
>>>>>>>>>>>>>>>>>>>> `withChain()` will
>>>>>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the
>>>>>>>>>>>>>>>>>>>>> returned `Map` of
>>>>>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a
>>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>>> implementation would be to not add the "branch" to the
>>>>>>>>>>>>>>>>>>>> result map if
>>>>>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>>> split the topology
>>>>>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both:
>>>>>>>>>>>>>>>>>>>> feed the KStream to
>>>>>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>>> doesn't need the
>>>>>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it from
>>>>>>>>>>>>>>>>>>>> there :-)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the
>>>>>>>>>>>>>>>>>>>> returned map
>>>>>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs
>>>>>>>>>>>>>>>>>>>> explicitly set by the
>>>>>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids.
>>>>>>>>>>>>>>>>>>>> Dealing with this
>>>>>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing
>>>>>>>>>>>>>>>>>>>> exclusions to this
>>>>>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by
>>>>>>>>>>>>>>>>>>>> provided
>>>>>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated.
>>>>>>>>>>>>>>>>>>>> What if a user
>>>>>>>>>>>>>>>>>>>> provides both the name of the branch and a
>>>>>>>>>>>>>>>>>>>> [Java]Consumer? What do they
>>>>>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or
>>>>>>>>>>>>>>>>>>>> save it to the map?
>>>>>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so
>>>>>>>>>>>>>>>>>>>> maybe just leave
>>>>>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>>> `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg,
>>>>>>>>>>>>>>>>>>>>> CoGroupedKStream,
>>>>>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy"
>>>>>>>>>>>>>>>>>>>>> `KGroupedStream`
>>>>>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a
>>>>>>>>>>>>>>>>>>>>> breaking change...
>>>>>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have
>>>>>>>>>>>>>>>>>>>>>> overloaded
>>>>>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate`
>>>>>>>>>>>>>>>>>>>>> and assume you
>>>>>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe
>>>>>>>>>>>>>>>>>>>>> rephrase it accordingly
>>>>>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public
>>>>>>>>>>>>>>>>>>>>> Interface" section) of
>>>>>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction
>>>>>>>>>>>>>>>>>>>>> of `withConsumer()`
>>>>>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This
>>>>>>>>>>>>>>>>>>>>> related to John's
>>>>>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use
>>>>>>>>>>>>>>>>>>>>>> cases: EITHER using
>>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and
>>>>>>>>>>>>>>>>>>>>> `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the
>>>>>>>>>>>>>>>>>>>>> result of `withChain()`
>>>>>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really
>>>>>>>>>>>>>>>>>>>>> with the `Consumer`
>>>>>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and
>>>>>>>>>>>>>>>>>>>>> `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add
>>>>>>>>>>>>>>>>>>>>> the "branch" to
>>>>>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as
>>>>>>>>>>>>>>>>>>>>> we clearly document
>>>>>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’
>>>>>>>>>>>>>>>>>>>>>> confusing; I thought you
>>>>>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface
>>>>>>>>>>>>>>>>>>>>>> (which doesn’t make
>>>>>>>>>>>>>>>>>>>>>> sense, of course). I get that you were referring to
>>>>>>>>>>>>>>>>>>>>>> the java Consumer
>>>>>>>>>>>>>>>>>>>>>> interface, but we should still probably to to avoid
>>>>>>>>>>>>>>>>>>>>>> the ambiguity.
>>>>>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be
>>>>>>>>>>>>>>>>>>>>> confused with a "sink
>>>>>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I
>>>>>>>>>>>>>>>>>>>>>> think it’s close,
>>>>>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I believe that now it follows design principles and
>>>>>>>>>>>>>>>>>>>>>>> takes into account
>>>>>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and
>>>>>>>>>>>>>>>>>>>>>>>> have completely
>>>>>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> This version includes all the previous discussion
>>>>>>>>>>>>>>>>>>>>>>>> results and follows
>>>>>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V>
>>>>>>>>>>>>>>>>>>>>>>>> branched)
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one
>>>>>>>>>>>>>>>>>>>>>>>> parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't
>>>>>>>>>>>>>>>>>>>>>>>> need to provide one
>>>>>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations
>>>>>>>>>>>>>>>>>>>>>>>> we may use a single
>>>>>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method
>>>>>>>>>>>>>>>>>>>>>>>> parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a
>>>>>>>>>>>>>>>>>>>>>>>> branch, no
>>>>>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.'
>>>>>>>>>>>>>>>>>>>>>>>> is expected here
>>>>>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter
>>>>>>>>>>>>>>>>>>>>>>>> rule'.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to
>>>>>>>>>>>>>>>>>>>>>>>>> have you back!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we
>>>>>>>>>>>>>>>>>>>>>>>>> already have a
>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems
>>>>>>>>>>>>>>>>>>>>>>>>> reasonable to me to
>>>>>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design
>>>>>>>>>>>>>>>>>>>>>>>>> principles" that we
>>>>>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in
>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion in
>>>>>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then
>>>>>>>>>>>>>>>>>>>>>>>>>> and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this,
>>>>>>>>>>>>>>>>>>>>>>>>>> because we invested
>>>>>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm
>>>>>>>>>>>>>>>>>>>>>>>>>> not feel entitled to
>>>>>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is
>>>>>>>>>>>>>>>>>>>>>>>>>> finalized.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and
>>>>>>>>>>>>>>>>>>>>>>>>>> reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed
>>>>>>>>>>>>>>>>>>>>>>>>>> branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the
>>>>>>>>>>>>>>>>>>>>>>>>>> original idea for
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why
>>>>>>>>>>>>>>>>>>>>>>>>>> I gave up pushing
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the
>>>>>>>>>>>>>>>>>>>>>>>>>> problem with the scope of
>>>>>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new
>>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>               .branch(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>               .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>>>>>               .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with
>>>>>>>>>>>>>>>>>>>>>>>>>> this approach.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright
>>>>>>>>>>>>>>>>>>>>>>>>>> solution in his post
>>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current
>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>             -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional
>>>>>>>>>>>>>>>>>>>>>>>>>> names seems to be
>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches
>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope,
>>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making
>>>>>>>>>>>>>>>>>>>>>>>>>> your code brittle.
>>>>>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging
>>>>>>>>>>>>>>>>>>>>>>>>>> purposes. Or,
>>>>>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named
>>>>>>>>>>>>>>>>>>>>>>>>>> branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on
>>>>>>>>>>>>>>>>>>>>>>>>>> June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on
>>>>>>>>>>>>>>>>>>>>>>>>>> this. The idea seems
>>>>>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust
>>>>>>>>>>>>>>>>>>>>>>>>>> people who are more
>>>>>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than
>>>>>>>>>>>>>>>>>>>>>>>>>> me.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status".
>>>>>>>>>>>>>>>>>>>>>>>>>>> Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up
>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think
>>>>>>>>>>>>>>>>>>>>>>>>>>>> about my proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both
>>>>>>>>>>>>>>>>>>>>>>>>>>>> worlds?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to merge both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your example.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default name but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that neither
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> independently of all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that, although
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> out.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split(Named.withName("mysplit"))
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                          .branch(..., ...,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                          .defaultBranch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                operator(function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              operator(function, function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> string)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>             
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               interface BranchConfig extends
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but it also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                Ivan, I’ll definitely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch(predicate, consumer)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                IMO the one trade off to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                question. I don’t know if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                in the same scope” since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                seems like a perfectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                when the branched streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                for the reasons Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                solution - working around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > On May 2, 2019, at 7:00
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Hello everyone, thank you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Well, I don't think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                LinkedHashMap (no other Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                matters) or `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > In my opinion, the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                proposal is that all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                But 1) we rarely need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                workaround for the scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 'Inlining the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                method references instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                tend to split the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > The drawbacks are strong.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                handlers is lost. We have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                handlers in another. This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > - what if we forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > - what if we misspell a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > - what if we copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > What Michael propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                writing the API in Lua,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                "dynamic naming" approach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                beautiful. But in Java we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                identifiers to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Do we have to invent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > And if we do, what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                all the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Earlier in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                without "start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                the case when we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                current KIP is 'clumsier'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                me address both comments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 1) "Start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                for it indeed) is critical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 2) No, dynamic branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                Imagine a real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                value (say, RecordType).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > /*John:if we had to start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > KBranchedStream branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > for (RecordType
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >             branched =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 02.05.2019 14:40,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> I also agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> current `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> However, I also don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> was more aligned with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> `branch()` statement and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> It makes the code easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> `Predicates` (that is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       ��        >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> An open question is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> specified. Atm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> and the call to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> (what is not the case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> because users can just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> About "inlining": So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> preference. I can see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> that clearly make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>> On 5/1/19 6:26 PM, Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>> Perhaps inlining is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that a lambda with the full
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                it can be a method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                The advantage of putting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                (Consumer) together in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>> Ultimately the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch trees will be hard
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> On May 1, 2019, at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> I'm less enthusiastic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> functionality.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> harder to read as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>           ��    >>>>> Also +1 on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> great framework for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Regarding the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> proposal in the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Obviously some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> doesn't work without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> associativity as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> directly associates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> provides over the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> The KIP solution is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> that it is slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> certainly possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> case anyway, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> access your branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branch, and it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> solution on top of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> I could also see a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> taken in, branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Pros for that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - accessing branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - no double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - downstream branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> to read top to bottom
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - you can forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> API and SortedMap,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> (KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Overall I'm curious
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branched KStream in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> doesn't need to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> user.  I'm sort of in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                <sophie@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> I'd like to +1 what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> method, I agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> trying to alleviate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> able to cleanly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> name->stream), which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> That said, I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> anti-pattern or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> I’d like to propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> are three problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 1. If you use it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 2. The way in which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> the ordering of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 3. It is brittle to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Using associative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> be a stronger
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> super K,? super
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Branches are given
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> mapping of names to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> because it’s a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> This solves problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 2 because you no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> interested in. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> conditional by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> messing with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> One of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> awkward in Java. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> On Tue, Apr 30,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                <john@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Thanks for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> FWIW, I agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> confusing when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> like a good name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> operator at all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Tentatively, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> way, we don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> should return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> enforce that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> default branch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> and additionally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> though with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> On Fri, Apr 26,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> The intend was to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> The current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> IMHO, this reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> take any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> `branch()` calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Because I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> might be better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> overlap that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Maybe there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> 'default' is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> a method with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Bummer. Didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Can you add the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> methods? It will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> KIP. For example,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> You did not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> -> KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> nice to get your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> would need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> should discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> "incomplete" to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branched-KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> On 4/13/19 2:13
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> I have updated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Matthias, thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> I can see your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> compatibility?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> deprecated, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Totally agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> reserved word,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Absolutely! I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> please revise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Any new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> 11.04.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> agrees that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> I had a quick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> are some minor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> It's just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> In the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Also, we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Ie, we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> For the issue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> could we extend
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> second argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> `Function`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Finally, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> proposal. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> On 3/31/19
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> I'm a bit of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> revise the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> buy-in from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> could be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> idea overall.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> forward the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> deprecate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> deprecating,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> same thing is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> just need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> is currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Regarding my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> little sloppy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> passing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> complicated to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> On Fri, Mar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> I read your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> looks better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> that KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> going to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> What shall we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Why are you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> as a novice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> Maybe I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> added, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> able to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> The issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> worked around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> [Also, great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> On Mar 28,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> The idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> a way to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> You just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> write in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Consider
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> customers who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> store to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> This is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> 'brancher'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> class
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>   private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>   private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>      
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>       /*In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> a separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> classes'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> /*Alas,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> without the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Does this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> we need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> [BTW Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> KIP I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> so I will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> I tried to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> succeeded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    pretty
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> To be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> I will say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> deprecated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> has the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> On Wed,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> OK, let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> First, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> There are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> 1. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> PROS: 1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> CONS: The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> 2. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> PROS:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> CONS: We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> called,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> I see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Still,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Let me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> 1) OK,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> drop
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Well,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> tests,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         ��      >>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> Hello
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         ��      >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> Does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>   ��            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi all,

I updated the KIP with what we have discussed, see 'How the resulting 
Map is formed' section. Four bullet points to define the rules for Map 
keys, and four bullet points to define the rules for Map values.

I also added the overloads to `Branched` in order to accept Consumers as 
parameters.

Regards,

Ivan


01.06.2020 8:56, Matthias J. Sax пишет:
> For naming:
> 
> (1) _If_ people specify a name for split() _and_ branch(), we _must_
> have a pulic API contract. Otherwise it would be very bad user
> experience if we are allowed to change the names (eg, do from "we don't
> insert a `-` to we insert a `-`). -- We also need to clarify what
> happens is only split() _or_ branch() has a name specified, ie, is there
> a contract for this case or not?
> 
> (2) If we have a contract for how split() and branch() names are put
> together, we can also apply it to auto-generated names (this seems to be
> just consistent).
> 
> (3) Do we want to have a contract for auto-generated names? This is
> debatable, but I personally don't see any harm in having a contract.
> Also, if people want to access the Map, they can easily do it without
> specifying names if there is a contract -- if there is no contract, we
> force people to specify names to use the Map, what seems to make the API
> harder to use.
> 
> But we can of course discuss furhter on the dev list.
> 
> 
> -Matthias
> 
> On 5/30/20 3:53 PM, Ivan Ponomarev wrote:
>> Hi Mathhias,
>>
>> I received your letter just after I sent mine concerning nulls and
>> reintroducing Consumer overload. So, if you're fine with omitting nulls
>> and introducing a with(Consumer) overload, it's great!
>>
>> Concerning the naming algorithm. Well, actually what you propose is very
>> clear and this is what I was going to implement anyway. But are you sure
>> that we will never want to change this algorithm? Here I trust your
>> judgement, since you understand the KStreams API's evolution much more
>> than I do.
>>
>>
>> 31.05.2020 0:50, Matthias J. Sax пишет:
>>> Ivan, John,
>>>
>>> thanks for the details.
>>>
>>> I guess I can be convinced in either direction with regard to `null`
>>> handling. But I am less worried about it, because I assume that a Java
>>> programmer understand the ambiguty of `get(key) == null` -- that's why
>>> there is `containsKey()`. In the end it might be a minor detail and as
>>> long as it's well documented all discussed approaches seem fine.
>>> Including the idea to re-introduce the `Consumer` overload.
>>>
>>> It's Ivan's KIP so I am fine with whatever he picks :)
>>>
>>>
>>> About the naming: Not sure why we don't want to make the naming a public
>>> contract? I guess for split() we could default to the empty String, and
>>> for `branch()` we could just add a counter by default?
>>>
>>> With no passed names at all, we just generte branche names "1", "2",
>>> "3", etc... With only`split(Named.as("foo-"))` (but not branch names) we
>>> generate "foo-1", "foo-2", etc.
>>>
>>> If any branch has a dedicated name, the counting is preserved but just
>>> not used int he name for this brach:
>>>
>>> split("foo-")
>>>     .brach()
>>>     .brach(Branched.as("bar")
>>>     .defaultBranch()
>>>
>>> would generate "foo-1", "foo-bar", "foo-3"?
>>>
>>> Thoughts?
>>>
>>>
>>>
>>> -Matthias
>>>
>>> On 5/29/20 1:20 PM, John Roesler wrote:
>>>> Hi all,
>>>>
>>>> Right, that was my suggestion; sorry for the confusion. I was thinking
>>>> that adding null as a value to maps is generally dubious, for which
>>>> reason a lot of Map implementations actually don't allow it at all.
>>>>
>>>> The reason is that it creates ambiguity, since a lot of code treats
>>>> "map.get(key) == null" as meaning the key is not in the map, but
>>>> if we place a null value in the map (presuming we don't immediately
>>>> get an exception), then you face weird contradictions, like
>>>> get(key) == null, but containsKey(key) can be either true or false, and
>>>> you may or may not see it while iterating. Which would violate the
>>>> expectations of many Java programs. You can just take a look at
>>>> the AK codebase, and you'll find many occurrences where we
>>>> assume a null mapping means the map doesn't contain the key.
>>>>
>>>> Of course, the practical question, which Ivan brought up, is also
>>>> a good one. What would you actually do with these null values
>>>> in the map, besides get an NPE ?
>>>>
>>>> I'm not sure why I didn't think of this before, but an alternative
>>>> to this debate is to go back to adding Consumer into the API, but
>>>> with the same method name, withChain(Consumer<KStream<K,V>>).
>>>>
>>>> Now that we have only static methods, it's less confusing because it's
>>>> not possible to choose _both_ the Function<KStream,KStream> _and_
>>>> the Consumer<KStream> in the same branch (which was one of the
>>>> main drawbacks of the original plan wrt withChain and
>>>> withJavaConsumer as builder methods.
>>>>
>>>> I.e., I was previously concerned about:
>>>> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2)
>>>>
>>>> It's just a little confusing trying to reason about how this is all
>>>> going to wire up.
>>>>
>>>> But with only the static methods, users have only three methods,
>>>> and they are all clear:
>>>> 1. Branched.as(name): just names the branch, the branch-predicated
>>>> stream is the value of the map
>>>> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the
>>>> branch, applies the transformation after the branch predicate, and
>>>> the transformed stream is the value of the map (null is not permitted)
>>>> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch,
>>>> passes the predicated stream in to the Consumer function and omits
>>>> the branch from the map
>>>>
>>>> Sorry for digging this option up again, but it's starting to look more
>>>> attractive after we dropped the builder functions, and I think it also
>>>> resolves the "null" concern.
>>>>
>>>> Thoughts?
>>>> -John
>>>>
>>>>
>>>> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>>>>> Hi Matthias!
>>>>>
>>>>> Thanks for your reply!
>>>>>
>>>>> (1)
>>>>>
>>>>>    > Do you imply that there won't be a `name -> null` entry in the
>>>>> `Map`
>>>>> for the branch? If yes, I am wondering why?
>>>>>
>>>>> Ah, I re-read the discussion and found out that omitting null entries
>>>>> was John's idea, not yours :-)
>>>>>
>>>>> Anyway. As you remember, I was against the 'swallowing' of branches by
>>>>> Consumer. I had a feeling that there are cases when I would like
>>>>> both to
>>>>> consume a branch and post-process it in a Map handler.
>>>>>
>>>>> But then the idea of getting rid of `withJavaConsumer` came up. And
>>>>> then
>>>>> we realized that we should handle nulls somehow.
>>>>>
>>>>> So it looked like taking the best from both approaches: if we want to
>>>>> omit a branch in the resulting map, we just return null, thus emulating
>>>>> a 'swallowing consumer'. If we want to include a branch, we return it.
>>>>> It provides the full control for a user, and it spares the user from
>>>>> null-checking in their code.
>>>>>
>>>>>    > My reasoning is, that it's very clear that `name -> null` would be
>>>>> there, because the user code did execute `return null`.
>>>>>
>>>>> My reasoning is following. The way such lambdas are written, it's hard
>>>>> to return null unintentionally. And if a user returns null
>>>>> intentionally, what `name->null` in the resulting Map can be good for?
>>>>> It's either NPE (which is clearly not intended) or something that
>>>>> should
>>>>> be null-checked and skipped. Ok, let's do it!
>>>>>
>>>>> But, I'm ready to be convinced to drop this rule. It doesn't seem to me
>>>>> to be a principal thing.
>>>>>
>>>>> (2)
>>>>>
>>>>>    > I think it would be good if the KIP could explains the intended
>>>>> default naming schema. This is important because the naming schema must
>>>>> be part of the API contract; otherwise, users cannot rely on the naming
>>>>> when trying to use the returned `Map`.
>>>>>
>>>>> Well, in fact, I intentionally didn't include any naming schema in the
>>>>> KIP. And it's done exactly for the purpose that a user won't be able to
>>>>> rely on default naming. If they want to use certain branches from the
>>>>> Map, they will devise their own naming strategy that will never change
>>>>> and will never let them down.
>>>>>
>>>>> If they don't mind what's in a Map, if they are looking up the Map just
>>>>> for debugging, or if they want to treat a Map just like a Collection of
>>>>> values (a possible use case!), it doesn't matter how the branches
>>>>> are named.
>>>>>
>>>>> This way we are giving ourselves the full freedom to change the naming
>>>>> schema afterwards without violating the API contract.
>>>>>
>>>>>
>>>>> Regards,
>>>>>
>>>>> Ivan
>>>>>
>>>>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>>>>> Thanks for updating the KIP!
>>>>>>
>>>>>> Using covariant generics is a good idea! I am also fine with only
>>>>>> using
>>>>>> static method in `Branched` for now, as it's only two parameters and
>>>>>> thus not too many overloads.
>>>>>>
>>>>>>
>>>>>> (1) What is unclear to me is, what you exaclty mean by:
>>>>>>
>>>>>>> If a function returns `null`, its result is omitted.
>>>>>>
>>>>>> Do you imply that there won't be a `name -> null` entry in the
>>>>>> `Map` for
>>>>>> the branch? If yes, I am wondering why?
>>>>>>
>>>>>> My previous argument was to omit an entry only for the
>>>>>> `withJavaConsumer()` case, because the return type is "void" (and thus
>>>>>> it's unclear what should be added and it would introduce
>>>>>> inconsistencies
>>>>>> in the `Map`).
>>>>>>
>>>>>> However, for a `Function` that returns a `KStream`, we could actually
>>>>>> just add a `name -> null` entry if `null` is returned. My reasoning
>>>>>> is,
>>>>>> that it's very clear that `name -> null` would be there, because the
>>>>>> user code did execute `return null`. Adding a `name -> null` entry for
>>>>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>>>>> exceptional case for handling `null`).
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>>
>>>>>> (2) For `Branched.as()` the KIP says:
>>>>>>
>>>>>>> sets the name of the branch (auto-generated by default, when split
>>>>>>> operation is named, then the names are index-suffixed)
>>>>>>
>>>>>> I think it would be good if the KIP could explains the intended
>>>>>> default
>>>>>> naming schema. This is important because the naming schema must be
>>>>>> part
>>>>>> of the API contract; otherwise, users cannot rely on the naming when
>>>>>> trying to use the returned `Map`.
>>>>>>
>>>>>> Similalry, it should be explained how names are generated if
>>>>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations how the
>>>>>> API can be mixed and matched and it's unclear atm how it would work in
>>>>>> detail.
>>>>>>
>>>>>>
>>>>>> Thanks!
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>>>>> Hi Ivan,
>>>>>>>
>>>>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>>>>> that have been raised in the discussion so far have been
>>>>>>> addressed. And it's been a while since anyone raised a new
>>>>>>> concern. At this point, it seems like a good time to start
>>>>>>> the VOTE thread.
>>>>>>>
>>>>>>> Sometimes, the vote thread will trigger new people to look
>>>>>>> into the KIP, and they may raise new concerns, but it's not a
>>>>>>> problem. We'll just address those lingering concerns if there
>>>>>>> are any, until you have all the votes you need.
>>>>>>>
>>>>>>> Thanks again for the contribution!
>>>>>>>
>>>>>>> -John
>>>>>>>
>>>>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>>>>
>>>>>>>> 1. Per Mathhias's suggestion, if a chain function returns null, the
>>>>>>>> respective result if omitted in the resulting Map.
>>>>>>>>
>>>>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>>>>
>>>>>>>> 3. `Branched` class has only three static methods with all the
>>>>>>>> possible
>>>>>>>> combinations of parameters.
>>>>>>>>
>>>>>>>> 4. Chain function is defined 'fully covariant', let's see if we can
>>>>>>>> implement it this way :-))
>>>>>>>>
>>>>>>>> + code example updates and minor edits.
>>>>>>>>
>>>>>>>>
>>>>>>>> Since this is my first KIP, I'm not sure what should I do next. I
>>>>>>>> feel
>>>>>>>> that we talked over all the details and the consensus is reached.
>>>>>>>> Is it
>>>>>>>> OK to call for VOTE now or is it better to wait for more feedback?
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>>>>> Sounds good to me, Ivan!
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>>>>> John,
>>>>>>>>>>
>>>>>>>>>> ---------------------
>>>>>>>>>>
>>>>>>>>>>      > I'm sorry things have been dragging out a little, but I
>>>>>>>>>> have the
>>>>>>>>>> sense we're very close to the end of this discussion, which is
>>>>>>>>>> exciting.
>>>>>>>>>>
>>>>>>>>>> We are certainly moving forward!  And I'm not in a hurry at
>>>>>>>>>> all. As I
>>>>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>>>>> KafkaStreamBrancher -- the implementation of the first,
>>>>>>>>>> rejected version
>>>>>>>>>> of this KIP. It's inferior to what we are discussing here, but
>>>>>>>>>> it does
>>>>>>>>>> its work. So it's worth to design this KIP really, really well.
>>>>>>>>>> And by
>>>>>>>>>> the way, from this discussion I'm learning the good API designing
>>>>>>>>>> process. For me it has a value per se :-))
>>>>>>>>>>
>>>>>>>>>> ----------------------
>>>>>>>>>>
>>>>>>>>>>      >>   > I'd wonder whether we need the non-static builders
>>>>>>>>>> (like withChain).
>>>>>>>>>>      >>   > Do they provide any benefit over just using the
>>>>>>>>>> right static
>>>>>>>>>> factory?
>>>>>>>>>>      >
>>>>>>>>>>      > I don't have a strong feeling, either. It seems nice to
>>>>>>>>>> offer a better
>>>>>>>>>>      > type inference experience than what we get with
>>>>>>>>>> Materialized, by
>>>>>>>>>>      > offering the static method that takes both name and chain.
>>>>>>>>>>      > Given that, there doesn't seem to be a good reason to
>>>>>>>>>> also offer the
>>>>>>>>>>      > non-static builder-style methods, so I guess I'd prefer
>>>>>>>>>> to drop them.
>>>>>>>>>>
>>>>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I
>>>>>>>>>> found out
>>>>>>>>>> an interesting fact about type inference rules in Java. Funny
>>>>>>>>>> thing is
>>>>>>>>>> that although we need to explicitly set types in a chain like this
>>>>>>>>>>
>>>>>>>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>>>>>>>>>
>>>>>>>>>> (otherwise it won't compile), the composition of static method
>>>>>>>>>> calls
>>>>>>>>>> works just fine, all the types are being calculated correctly:
>>>>>>>>>>
>>>>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>>>>>>>>>
>>>>>>>>>> As I was told, for type inference there is difference between
>>>>>>>>>> qualifiers
>>>>>>>>>> and arguments, 'you go up if you are argument but stop if you are
>>>>>>>>>> qualifier'. And it also seems that we should not bet on any future
>>>>>>>>>> improvements in Java type inference here.
>>>>>>>>>>
>>>>>>>>>> So,
>>>>>>>>>>
>>>>>>>>>> 1) I think we that in this KIP we should provide three static
>>>>>>>>>> methods
>>>>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function,
>>>>>>>>>> String)`, and
>>>>>>>>>> drop any non-static ones.
>>>>>>>>>>
>>>>>>>>>> 2) If anything else will be ever needed, we can easily add
>>>>>>>>>> anything.
>>>>>>>>>> Maybe this can be done in a process of refinement of all the
>>>>>>>>>> parameter
>>>>>>>>>> classes.
>>>>>>>>>>
>>>>>>>>>> ----------------------------
>>>>>>>>>>
>>>>>>>>>>      > we may as well hope for the best, and propose the "fully
>>>>>>>>>>      > covariant" definition for now.
>>>>>>>>>>
>>>>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>>>>
>>>>>>>>>> ----------------------------
>>>>>>>>>>
>>>>>>>>>>      >> Good question, I already thought about it and rejected
>>>>>>>>>> the idea....
>>>>>>>>>>      > (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>      >  "Worst" case scenario: someone
>>>>>>>>>>      > else will wish the return type is something different,
>>>>>>>>>> and we'll go
>>>>>>>>>>      > through a painless deprecation transition to change it
>>>>>>>>>> later.
>>>>>>>>>>
>>>>>>>>>> Of course, we cant' predict all the ways people are going to
>>>>>>>>>> use it.
>>>>>>>>>>      From my own humble experience with Kafka Streams, the worst
>>>>>>>>>> scenario is
>>>>>>>>>> unlikely. Split is split, transform is transform, too much
>>>>>>>>>> flexibility
>>>>>>>>>> is often evil.
>>>>>>>>>>
>>>>>>>>>> ------------------------
>>>>>>>>>>
>>>>>>>>>> So it seems that we are close to the consensus. Two things to
>>>>>>>>>> be altered
>>>>>>>>>> in the current version of KIP:
>>>>>>>>>>
>>>>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>>>>
>>>>>>>>>> Any ideas / questions / objections?
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>>
>>>>>>>>>> Ivan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>>>>
>>>>>>>>>>> I'm sorry things have been dragging out a little, but I have
>>>>>>>>>>> the sense
>>>>>>>>>>> we're very close to the end of this discussion, which is
>>>>>>>>>>> exciting.
>>>>>>>>>>>
>>>>>>>>>>>>       > I'd wonder whether we need the non-static builders
>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>>       > Do they provide any benefit over just using the right
>>>>>>>>>>>> static factory?
>>>>>>>>>>>>
>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>> builders can
>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>
>>>>>>>>>>> I don't have a strong feeling, either. It seems nice to offer
>>>>>>>>>>> a better
>>>>>>>>>>> type inference experience than what we get with Materialized, by
>>>>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>>>>
>>>>>>>>>>> Given that, there doesn't seem to be a good reason to also
>>>>>>>>>>> offer the
>>>>>>>>>>> non-static builder-style methods, so I guess I'd prefer to
>>>>>>>>>>> drop them.
>>>>>>>>>>>
>>>>>>>>>>> I'll defer to Matthias, if he has a chance to consider whether
>>>>>>>>>>> it's more
>>>>>>>>>>> valuable to stick with the existing pattern or break the
>>>>>>>>>>> pattern to offer
>>>>>>>>>>> a better experience.
>>>>>>>>>>>
>>>>>>>>>>>>       > You might as well propose the “ideal” API in the KIP,
>>>>>>>>>>>> which is the
>>>>>>>>>>>>       > covariant typed function
>>>>>>>>>>>>
>>>>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>> implementation
>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>
>>>>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been that
>>>>>>>>>>> we just
>>>>>>>>>>> won't know what works until we really try it, and try it in
>>>>>>>>>>> several different
>>>>>>>>>>> ways. Still, we may as well hope for the best, and propose the
>>>>>>>>>>> "fully
>>>>>>>>>>> covariant" definition for now.
>>>>>>>>>>>
>>>>>>>>>>>>       > Is it necessary to restrict the result key and value
>>>>>>>>>>>> types to be the
>>>>>>>>>>>>       > same as the inputs?
>>>>>>>>>>>>
>>>>>>>>>>>> Good question, I already thought about it and rejected the
>>>>>>>>>>>> idea....
>>>>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>>>>
>>>>>>>>>>> That's fair! It's your KIP, after all. I think I might have
>>>>>>>>>>> made a different
>>>>>>>>>>> call here, but I think this choice is fine. "Worst" case
>>>>>>>>>>> scenario: someone
>>>>>>>>>>> else will wish the return type is something different, and
>>>>>>>>>>> we'll go
>>>>>>>>>>> through a painless deprecation transition to change it later.
>>>>>>>>>>> Thanks
>>>>>>>>>>> to the clean design of your API, this doesn't seem to bad.
>>>>>>>>>>> And, of
>>>>>>>>>>> course, you've actually been using similar functionality
>>>>>>>>>>> already, so it
>>>>>>>>>>> seems we should trust your intuition.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>
>>>>>>>>>>>>       > I'd wonder whether we need the non-static builders
>>>>>>>>>>>> (like withChain).
>>>>>>>>>>>> Do they provide any benefit over just using the right static
>>>>>>>>>>>> factory?
>>>>>>>>>>>>
>>>>>>>>>>>> I don't have a strong opinion here. I think it's just a
>>>>>>>>>>>> matter of taste.
>>>>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static
>>>>>>>>>>>> builders can
>>>>>>>>>>>> be omitted, I agree!
>>>>>>>>>>>>
>>>>>>>>>>>>       > You might as well propose the “ideal” API in the KIP,
>>>>>>>>>>>> which is the
>>>>>>>>>>>> covariant typed function
>>>>>>>>>>>>
>>>>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>>>>> following: 1) accept the 'fully covariant' definition in the
>>>>>>>>>>>> KIP, 2)
>>>>>>>>>>>> then just see if there are any obstacles/pitfalls during
>>>>>>>>>>>> implementation
>>>>>>>>>>>> and unit testing?
>>>>>>>>>>>>
>>>>>>>>>>>>       > Is it necessary to restrict the result key and value
>>>>>>>>>>>> types to be the
>>>>>>>>>>>> same as the inputs?
>>>>>>>>>>>>
>>>>>>>>>>>> Good question, I already thought about it and rejected the idea.
>>>>>>>>>>>>
>>>>>>>>>>>> Look, if we want to keep `withChain`'s function optional,
>>>>>>>>>>>> then we must
>>>>>>>>>>>> keep the result key and value types the same. Because for
>>>>>>>>>>>> now, the
>>>>>>>>>>>> default value for the 'chain function' is Function.identity().
>>>>>>>>>>>>
>>>>>>>>>>>> Of course, we can make the 'chain function' required. But I
>>>>>>>>>>>> think this
>>>>>>>>>>>> is not what `split` method is for. `split` is for splitting, not
>>>>>>>>>>>> transforming, and `chainFunction` in most of the cases should
>>>>>>>>>>>> be either
>>>>>>>>>>>> a consumer or the identity function.
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Ivan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past
>>>>>>>>>>>>> that generic type inference problem is to offer a static
>>>>>>>>>>>>> factory method that takes all the options (both name and
>>>>>>>>>>>>> function).
>>>>>>>>>>>>>
>>>>>>>>>>>>> I’m happy with adding that option, although if we have all
>>>>>>>>>>>>> possible combinations available as static methods, then I’d
>>>>>>>>>>>>> wonder whether we need the non-static builders (like
>>>>>>>>>>>>> withChain). Do they provide any benefit over just using the
>>>>>>>>>>>>> right static factory?
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type
>>>>>>>>>>>>> bounds works algebraically, but I’m not sure whether the
>>>>>>>>>>>>> limitations of Java type inference will actually let you
>>>>>>>>>>>>> pass in all the different functions you would want to.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I recall Bruno and I wrestling with this when he added
>>>>>>>>>>>>> Transform operators. In retrospect, this is a problem we can
>>>>>>>>>>>>> iron out when we write tests in the PR. You might as well
>>>>>>>>>>>>> propose the “ideal” API in the KIP, which is the covariant
>>>>>>>>>>>>> typed function:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>> KStream<? extends K, ? extends V>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3. Actually, I have a new question about the types: Is it
>>>>>>>>>>>>> necessary to restrict the result key and value types to be
>>>>>>>>>>>>> the same as the inputs? I.e., shouldn’t the result types be
>>>>>>>>>>>>> K1,V1?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> John
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>>>>> Hello John,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > Perhaps it would be better to stick with "as" for now
>>>>>>>>>>>>>>        > and just file a Jira to switch them all at the same
>>>>>>>>>>>>>> time [for
>>>>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin
>>>>>>>>>>>>>> they have a
>>>>>>>>>>>>>> standard workaround
>>>>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> So actually this should be a very low priority issue, if an
>>>>>>>>>>>>>> issue at
>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > I don't understand how your new proposed
>>>>>>>>>>>>>>        > methods would work any differently than the ones
>>>>>>>>>>>>>> you already
>>>>>>>>>>>>>>        > had proposed in the KIP. It seems like you'd still
>>>>>>>>>>>>>> have to provide
>>>>>>>>>>>>>>        > the generic type parameters on the first static
>>>>>>>>>>>>>> factory call. Can you
>>>>>>>>>>>>>>        > explain how your new interface proposal differs
>>>>>>>>>>>>>> from the existing KIP?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> In the KIP, I didn't clarify what methods should be static.
>>>>>>>>>>>>>> Now I
>>>>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The overloaded `with` version that provides both Function
>>>>>>>>>>>>>> and name can
>>>>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>> ----------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > Regarding making the K,V types covariant also, yes,
>>>>>>>>>>>>>> that would indeed
>>>>>>>>>>>>>>        > be nice, but I'm not sure it will actually work.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> static KStream<String, Integer> func(KStream<String,
>>>>>>>>>>>>>> Number> s) {
>>>>>>>>>>>>>>                return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>>>>            s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>>>>                       //Won't compile!!
>>>>>>>>>>>>>>                       Branched.with(Me::func));
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The simple workaround here is to change `func`'s return
>>>>>>>>>>>>>> type from
>>>>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [On the other hand, we already agreed to remove
>>>>>>>>>>>>>> `withJavaConsumer` from
>>>>>>>>>>>>>> `Branched`, so during code migration I will have to modify
>>>>>>>>>>>>>> my functions'
>>>>>>>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >  the map you're returning is Map<K,V>, and of
>>>>>>>>>>>>>> course a K is not the
>>>>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I think what you actually meant here is that KStream<?
>>>>>>>>>>>>>> extends K, ?
>>>>>>>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K,
>>>>>>>>>>>>>> V>>. This
>>>>>>>>>>>>>> particularly is not a problem, since KStream<? extends K, ?
>>>>>>>>>>>>>> extends V>
>>>>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put
>>>>>>>>>>>>>> to the map.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard
>>>>>>>>>>>>>> types. So maybe
>>>>>>>>>>>>>> for now it's better to just admit that API is not
>>>>>>>>>>>>>> absolutely perfect and
>>>>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that
>>>>>>>>>>>>>>> "as" would
>>>>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from
>>>>>>>>>>>>>>> convention
>>>>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The convention is that "as(String name)" is used for the
>>>>>>>>>>>>>>> static factory
>>>>>>>>>>>>>>> method, whereas "withName(String name)" is an instance method
>>>>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to
>>>>>>>>>>>>>>> avoid "as"
>>>>>>>>>>>>>>> for compatibility with Kotlin, I might suggest
>>>>>>>>>>>>>>> "fromName(String name)",
>>>>>>>>>>>>>>> although it's somewhat dubious, since all the other
>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick with
>>>>>>>>>>>>>>> "as" for now
>>>>>>>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>>>>> Regarding the type inference problem, yes, it's a blemish
>>>>>>>>>>>>>>> on all of our
>>>>>>>>>>>>>>> configuraion objects. The problem is that Java infers the
>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>> based on the _first_ method in the chain. While it does
>>>>>>>>>>>>>>> consider what
>>>>>>>>>>>>>>> the recipient of the method result wants, it only
>>>>>>>>>>>>>>> considers the _next_
>>>>>>>>>>>>>>> recipient.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>>>>>>>>>> Branched<String,String> variable, java infers the type
>>>>>>>>>>>>>>> correctly. But
>>>>>>>>>>>>>>> when the "next recipient" is a chained method call, like
>>>>>>>>>>>>>>> "withChain",
>>>>>>>>>>>>>>> then the chained method doesn't bound the type (by
>>>>>>>>>>>>>>> definition,
>>>>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java
>>>>>>>>>>>>>>> will take
>>>>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be
>>>>>>>>>>>>>>> revised anymore.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, since it
>>>>>>>>>>>>>>> doesn't seem
>>>>>>>>>>>>>>> that hard to recursively consider the entire context when
>>>>>>>>>>>>>>> inferring the
>>>>>>>>>>>>>>> generic type parameters, but this is what we have to work
>>>>>>>>>>>>>>> with.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> To be honest, though, I don't understand how your new
>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>> methods would work any differently than the ones you already
>>>>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still have to
>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>> the generic type parameters on the first static factory
>>>>>>>>>>>>>>> call. Can you
>>>>>>>>>>>>>>> explain how your new interface proposal differs from the
>>>>>>>>>>>>>>> existing KIP?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that
>>>>>>>>>>>>>>> would indeed
>>>>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You might
>>>>>>>>>>>>>>> want to give it a
>>>>>>>>>>>>>>> try. In the past, we've run into soe truly strange
>>>>>>>>>>>>>>> interactions between the
>>>>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner
>>>>>>>>>>>>>>> classes) in
>>>>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Another issue is that the value type of the map you're
>>>>>>>>>>>>>>> returning is
>>>>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends
>>>>>>>>>>>>>>> K", so it
>>>>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that
>>>>>>>>>>>>>>>> getting rid of
>>>>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as
>>>>>>>>>>>>>>>> possible, but not
>>>>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to
>>>>>>>>>>>>>>>> implement examples
>>>>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1. Having to return something from lambda is not a very
>>>>>>>>>>>>>>>> big deal.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use
>>>>>>>>>>>>>>>> method references
>>>>>>>>>>>>>>>> for already written stream consumers, but then I realized
>>>>>>>>>>>>>>>> that I can
>>>>>>>>>>>>>>>> just change my methods from returning void to returning
>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>> parameter and use references to them. Not very
>>>>>>>>>>>>>>>> convenient, but passable.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no
>>>>>>>>>>>>>>>> consumer 2) when
>>>>>>>>>>>>>>>> function returns null, we don't insert it into the
>>>>>>>>>>>>>>>> resulting map.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but
>>>>>>>>>>>>>>>> workable solution as
>>>>>>>>>>>>>>>> a first approximation. And later we can always add to
>>>>>>>>>>>>>>>> `Branched`
>>>>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes
>>>>>>>>>>>>>>>> like Branched
>>>>>>>>>>>>>>>> should be built? First of all, it seems that `as` now is
>>>>>>>>>>>>>>>> more preferred
>>>>>>>>>>>>>>>> than `withName` (although as you probably know it clashes
>>>>>>>>>>>>>>>> with Kotlin's
>>>>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out that my
>>>>>>>>>>>>>>>> Java cannot
>>>>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>            Branched.as("foo").withChain(s ->
>>>>>>>>>>>>>>>> s.mapValues(...)))
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>>>>            Branched.<String, String>as("foo").withChain(s
>>>>>>>>>>>>>>>> -> s.mapValues(...)))
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of
>>>>>>>>>>>>>>>> `Branched` methods
>>>>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super
>>>>>>>>>>>>>>>> KStream<K, V>, ?
>>>>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>,
>>>>>>>>>>>>>>>> ? extends
>>>>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 4. And one more. What do you think, do we need that
>>>>>>>>>>>>>>>> flexibility:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>>>>>> chain
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends
>>>>>>>>>>>>>>>> KStream<?
>>>>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ??
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the
>>>>>>>>>>>>>>>>> map, because there’s only one thing that could be there,
>>>>>>>>>>>>>>>>> which is whatever is returned from the chain function.
>>>>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way
>>>>>>>>>>>>>>>>> this KIP is designed, it would be trivially easy to add
>>>>>>>>>>>>>>>>> the “terminal” chain later. As you say, fewer concepts
>>>>>>>>>>>>>>>>> leads to an API that is easier to learn.
>>>>>>>>>>>>>>>>> 3. We get to side-step the naming of this method.
>>>>>>>>>>>>>>>>> Although I didn’t complain about withJavaConsumer, it
>>>>>>>>>>>>>>>>> was only because I couldn’t think of a better name.
>>>>>>>>>>>>>>>>> Still, it’s somewhat unsatisfying to name a method after
>>>>>>>>>>>>>>>>> its argument type, since this provides no information at
>>>>>>>>>>>>>>>>> all about what the method does. I was willing to accept
>>>>>>>>>>>>>>>>> it because I didn’t have an alternative, but I would be
>>>>>>>>>>>>>>>>> happy to skip this method for now to avoid the problem
>>>>>>>>>>>>>>>>> until we have more inspiration.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The only con I see is that it makes the code a little
>>>>>>>>>>>>>>>>> less ergonomic to write when you don’t want to return
>>>>>>>>>>>>>>>>> the result of the chain (such as when the chain is
>>>>>>>>>>>>>>>>> terminal), since I’m your example, you have to declare a
>>>>>>>>>>>>>>>>> block with a return statement at the end. It’s not
>>>>>>>>>>>>>>>>> ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing
>>>>>>>>>>>>>>>>> a null result, which would just remove the branch from
>>>>>>>>>>>>>>>>> the returned map. It seems nicer than forcing people to
>>>>>>>>>>>>>>>>> pick a stream to return when their chain is terminal and
>>>>>>>>>>>>>>>>> they don’t want to use the result later.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I guess the only open question is about
>>>>>>>>>>>>>>>>>> `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          (1) split a stream and return the substreams
>>>>>>>>>>>>>>>>>> for futher processing
>>>>>>>>>>>>>>>>>>          (2) split a stream and modify the substreams
>>>>>>>>>>>>>>>>>> with in-place method chaining
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>           -> split a stream, modify the substreams, and
>>>>>>>>>>>>>>>>>> return the _modified_
>>>>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>> split the topology graph at any point. Technically,
>>>>>>>>>>>>>>>>>>> it's OK to do both: feed the KStream to a
>>>>>>>>>>>>>>>>>>> [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>> doesn't need the stream in the Map, one simply does
>>>>>>>>>>>>>>>>>>> not extract it from there
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> That is of course possible. However, it introduces some
>>>>>>>>>>>>>>>>>> "hidded" semantics:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          - using `withChain` I get the modified sub-stream
>>>>>>>>>>>>>>>>>>          - using `withJavaConsumer` I get the unmodifed
>>>>>>>>>>>>>>>>>> sub-stream
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         From my understanding the original idea of
>>>>>>>>>>>>>>>>>> `withJavaConsumer` was to
>>>>>>>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>           s.to();
>>>>>>>>>>>>>>>>>>           return null;
>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> However, I am not sure if we should even allow
>>>>>>>>>>>>>>>>>> `withChain()` to return
>>>>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this
>>>>>>>>>>>>>>>>>> case to avoid a `key
>>>>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Following this train of through, and if we want to
>>>>>>>>>>>>>>>>>> allow the "return
>>>>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>> that does not add
>>>>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Following your proposal, the semantics of
>>>>>>>>>>>>>>>>>> `withJavaConsumer` could also
>>>>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>>>>           s.to();
>>>>>>>>>>>>>>>>>>           return s;
>>>>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely
>>>>>>>>>>>>>>>>>> syntactic sugar,
>>>>>>>>>>>>>>>>>> while for the first proposal it adds new functionality
>>>>>>>>>>>>>>>>>> (if `return null`
>>>>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to
>>>>>>>>>>>>>>>>>> "hide a
>>>>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need
>>>>>>>>>>>>>>>>>> to allow `return
>>>>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a
>>>>>>>>>>>>>>>>>> sub-stream in the Map.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we
>>>>>>>>>>>>>>>>>> follow your
>>>>>>>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer`
>>>>>>>>>>>>>>>>>> at all? Its
>>>>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API
>>>>>>>>>>>>>>>>>> is usually
>>>>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox,
>>>>>>>>>>>>>>>>>>> fortunately we have
>>>>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the
>>>>>>>>>>>>>>>>>>>> existing ‘branch’
>>>>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration
>>>>>>>>>>>>>>>>>>> Plan" section.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to
>>>>>>>>>>>>>>>>>>>> branching directly
>>>>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic
>>>>>>>>>>>>>>>>>>> branching]
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic
>>>>>>>>>>>>>>>>>>> Branching' section.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka
>>>>>>>>>>>>>>>>>>>> Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I
>>>>>>>>>>>>>>>>>>> renamed this
>>>>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases:
>>>>>>>>>>>>>>>>>>>> EITHER using
>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>> `BranchedKStream`]
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of
>>>>>>>>>>>>>>>>>>>> the `branch`
>>>>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that
>>>>>>>>>>>>>>>>>>>> using both
>>>>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of
>>>>>>>>>>>>>>>>>>> `withChain()` will
>>>>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the
>>>>>>>>>>>>>>>>>>>> returned `Map` of
>>>>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a
>>>>>>>>>>>>>>>>>>> reasonable
>>>>>>>>>>>>>>>>>>> implementation would be to not add the "branch" to the
>>>>>>>>>>>>>>>>>>> result map if
>>>>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can
>>>>>>>>>>>>>>>>>>> split the topology
>>>>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both:
>>>>>>>>>>>>>>>>>>> feed the KStream to
>>>>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one
>>>>>>>>>>>>>>>>>>> doesn't need the
>>>>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it from
>>>>>>>>>>>>>>>>>>> there :-)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the
>>>>>>>>>>>>>>>>>>> returned map
>>>>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs
>>>>>>>>>>>>>>>>>>> explicitly set by the
>>>>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids.
>>>>>>>>>>>>>>>>>>> Dealing with this
>>>>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing
>>>>>>>>>>>>>>>>>>> exclusions to this
>>>>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated.
>>>>>>>>>>>>>>>>>>> What if a user
>>>>>>>>>>>>>>>>>>> provides both the name of the branch and a
>>>>>>>>>>>>>>>>>>> [Java]Consumer? What do they
>>>>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or
>>>>>>>>>>>>>>>>>>> save it to the map?
>>>>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so
>>>>>>>>>>>>>>>>>>> maybe just leave
>>>>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` ->
>>>>>>>>>>>>>>>>>>>> `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg,
>>>>>>>>>>>>>>>>>>>> CoGroupedKStream,
>>>>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy"
>>>>>>>>>>>>>>>>>>>> `KGroupedStream`
>>>>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a
>>>>>>>>>>>>>>>>>>>> breaking change...
>>>>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have
>>>>>>>>>>>>>>>>>>>>> overloaded
>>>>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate`
>>>>>>>>>>>>>>>>>>>> and assume you
>>>>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe
>>>>>>>>>>>>>>>>>>>> rephrase it accordingly
>>>>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public
>>>>>>>>>>>>>>>>>>>> Interface" section) of
>>>>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction
>>>>>>>>>>>>>>>>>>>> of `withConsumer()`
>>>>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This
>>>>>>>>>>>>>>>>>>>> related to John's
>>>>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use
>>>>>>>>>>>>>>>>>>>>> cases: EITHER using
>>>>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and
>>>>>>>>>>>>>>>>>>>> `withConsumer()` is the
>>>>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the
>>>>>>>>>>>>>>>>>>>> result of `withChain()`
>>>>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really
>>>>>>>>>>>>>>>>>>>> with the `Consumer`
>>>>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and
>>>>>>>>>>>>>>>>>>>> `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add
>>>>>>>>>>>>>>>>>>>> the "branch" to
>>>>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as
>>>>>>>>>>>>>>>>>>>> we clearly document
>>>>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’
>>>>>>>>>>>>>>>>>>>>> confusing; I thought you
>>>>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface
>>>>>>>>>>>>>>>>>>>>> (which doesn’t make
>>>>>>>>>>>>>>>>>>>>> sense, of course). I get that you were referring to
>>>>>>>>>>>>>>>>>>>>> the java Consumer
>>>>>>>>>>>>>>>>>>>>> interface, but we should still probably to to avoid
>>>>>>>>>>>>>>>>>>>>> the ambiguity.
>>>>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be
>>>>>>>>>>>>>>>>>>>> confused with a "sink
>>>>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I
>>>>>>>>>>>>>>>>>>>>> think it’s close,
>>>>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I believe that now it follows design principles and
>>>>>>>>>>>>>>>>>>>>>> takes into account
>>>>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and
>>>>>>>>>>>>>>>>>>>>>>> have completely
>>>>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> This version includes all the previous discussion
>>>>>>>>>>>>>>>>>>>>>>> results and follows
>>>>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V>
>>>>>>>>>>>>>>>>>>>>>>> branched)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one
>>>>>>>>>>>>>>>>>>>>>>> parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't
>>>>>>>>>>>>>>>>>>>>>>> need to provide one
>>>>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations
>>>>>>>>>>>>>>>>>>>>>>> we may use a single
>>>>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method
>>>>>>>>>>>>>>>>>>>>>>> parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a
>>>>>>>>>>>>>>>>>>>>>>> branch, no
>>>>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.'
>>>>>>>>>>>>>>>>>>>>>>> is expected here
>>>>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter
>>>>>>>>>>>>>>>>>>>>>>> rule'.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to
>>>>>>>>>>>>>>>>>>>>>>>> have you back!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we
>>>>>>>>>>>>>>>>>>>>>>>> already have a
>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems
>>>>>>>>>>>>>>>>>>>>>>>> reasonable to me to
>>>>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design
>>>>>>>>>>>>>>>>>>>>>>>> principles" that we
>>>>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in
>>>>>>>>>>>>>>>>>>>>>>>>> the discussion in
>>>>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then
>>>>>>>>>>>>>>>>>>>>>>>>> and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this,
>>>>>>>>>>>>>>>>>>>>>>>>> because we invested
>>>>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm
>>>>>>>>>>>>>>>>>>>>>>>>> not feel entitled to
>>>>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and
>>>>>>>>>>>>>>>>>>>>>>>>> reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed
>>>>>>>>>>>>>>>>>>>>>>>>> branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the
>>>>>>>>>>>>>>>>>>>>>>>>> original idea for
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why
>>>>>>>>>>>>>>>>>>>>>>>>> I gave up pushing
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the
>>>>>>>>>>>>>>>>>>>>>>>>> problem with the scope of
>>>>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new
>>>>>>>>>>>>>>>>>>>>>>>>> AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>               .branch(....)
>>>>>>>>>>>>>>>>>>>>>>>>>               .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>>>>               .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with
>>>>>>>>>>>>>>>>>>>>>>>>> this approach.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright
>>>>>>>>>>>>>>>>>>>>>>>>> solution in his post
>>>>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>             -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>             -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>             -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional
>>>>>>>>>>>>>>>>>>>>>>>>> names seems to be
>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches
>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope,
>>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making
>>>>>>>>>>>>>>>>>>>>>>>>> your code brittle.
>>>>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging
>>>>>>>>>>>>>>>>>>>>>>>>> purposes. Or,
>>>>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named
>>>>>>>>>>>>>>>>>>>>>>>>> branches in the
>>>>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on
>>>>>>>>>>>>>>>>>>>>>>>>> June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on
>>>>>>>>>>>>>>>>>>>>>>>>> this. The idea seems
>>>>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust
>>>>>>>>>>>>>>>>>>>>>>>>> people who are more
>>>>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than
>>>>>>>>>>>>>>>>>>>>>>>>> me.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status".
>>>>>>>>>>>>>>>>>>>>>>>>>> Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up
>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think
>>>>>>>>>>>>>>>>>>>>>>>>>>> about my proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both
>>>>>>>>>>>>>>>>>>>>>>>>>>> worlds?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded
>>>>>>>>>>>>>>>>>>>>>>>>>>>> chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can
>>>>>>>>>>>>>>>>>>>>>>>>>>>> of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to merge both
>>>>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API
>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be
>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from
>>>>>>>>>>>>>>>>>>>>>>>>>>>> your example.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> The returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why
>>>>>>>>>>>>>>>>>>>>>>>>>>>> should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might
>>>>>>>>>>>>>>>>>>>>>>>>>>>> lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could
>>>>>>>>>>>>>>>>>>>>>>>>>>>> happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead
>>>>>>>>>>>>>>>>>>>>>>>>>>>> to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> default name but
>>>>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not
>>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> what does not
>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not
>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen
>>>>>>>>>>>>>>>>>>>>>>>>>>>> that neither
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set
>>>>>>>>>>>>>>>>>>>>>>>>>>>> independently of all
>>>>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> add new
>>>>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by
>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an
>>>>>>>>>>>>>>>>>>>>>>>>>>>> "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that, although
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> under this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split(Named.withName("mysplit"))
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                          .branch(..., ...,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                          .defaultBranch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                operator(function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              operator(function, function,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> string)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               interface BranchConfig extends
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but it also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                Ivan, I’ll definitely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch(predicate, consumer)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                IMO the one trade off to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                question. I don’t know if I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                in the same scope” since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                seems like a perfectly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                when the branched streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                for the reasons Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                solution - working around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > On May 2, 2019, at 7:00
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Hello everyone, thank you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Well, I don't think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                LinkedHashMap (no other Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                matters) or `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > In my opinion, the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                proposal is that all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                But 1) we rarely need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                workaround for the scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 'Inlining the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                method references instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                tend to split the complex
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > The drawbacks are strong.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                handlers is lost. We have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                handlers in another. This
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > - what if we forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > - what if we misspell a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > - what if we copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > What Michael propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                writing the API in Lua,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                "dynamic naming" approach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                beautiful. But in Java we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                identifiers to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Do we have to invent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > And if we do, what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                all the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Earlier in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                without "start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                the case when we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                current KIP is 'clumsier'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                me address both comments here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 1) "Start branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                for it indeed) is critical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 2) No, dynamic branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                Imagine a real-world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                value (say, RecordType).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > /*John:if we had to start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > KBranchedStream branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > for (RecordType
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >             branched =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                > 02.05.2019 14:40,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> I also agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> current `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> However, I also don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> was more aligned with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> `branch()` statement and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> It makes the code easier
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> `Predicates` (that is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       ��        >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> An open question is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> specified. Atm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> and the call to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> (what is not the case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> because users can just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> About "inlining": So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> preference. I can see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> that clearly make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>> On 5/1/19 6:26 PM, Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>> Perhaps inlining is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that a lambda with the full
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                it can be a method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                The advantage of putting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                (Consumer) together in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>> Ultimately the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch trees will be hard
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> On May 1, 2019, at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> I'm less enthusiastic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> functionality.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>> harder to read as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>           ��    >>>>> Also +1 on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> great framework for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Regarding the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> proposal in the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Obviously some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> doesn't work without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> associativity as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> directly associates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> provides over the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> The KIP solution is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> that it is slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> certainly possible.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> case anyway, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> access your branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branch, and it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> I could also see a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> taken in, branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>    .branch(
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - accessing branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - no double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - downstream branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> to read top to bottom
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> - you can forget to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> API and SortedMap,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> (KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Overall I'm curious
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branched KStream in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> doesn't need to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> user.  I'm sort of in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                <sophie@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> I'd like to +1 what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> method, I agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> trying to alleviate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> able to cleanly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> name->stream), which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> That said, I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> anti-pattern or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> On Tue, Apr 30, 2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> I’d like to propose
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> are three problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 1. If you use it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 2. The way in which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> the ordering of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 3. It is brittle to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Using associative
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> be a stronger
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Map<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Branches are given
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> mapping of names to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> because it’s a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> This solves problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> 2 because you no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> interested in. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> conditional by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> messing with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> One of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> awkward in Java. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> On Tue, Apr 30,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                <john@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Thanks for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> FWIW, I agree with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> confusing when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> like a good name.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> operator at all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Tentatively, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> way, we don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> should return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> enforce that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> default branch.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> and additionally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> though with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> On Fri, Apr 26,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> The intend was to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> The current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> IMHO, this reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> take any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> `branch()` calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Because I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> might be better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> overlap that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Maybe there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> 'default' is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> a method with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Bummer. Didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Can you add the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> methods? It will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> KIP. For example,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> You did not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> -> KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> nice to get your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> would need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> should discuss
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> "incomplete" to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branched-KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> On 4/13/19 2:13
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> I have updated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Matthias, thanks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> I can see your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> that also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> compatibility?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> deprecated, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Renaming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Totally agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> reserved word,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Absolutely! I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> please revise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Any new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>> 11.04.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Thanks for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> agrees that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> I had a quick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> are some minor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> It's just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> In the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> but I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Also, we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Ie, we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> For the issue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> could we extend
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> second argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> `Function`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> Finally, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> proposal. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> On 3/31/19
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> I'm a bit of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> revise the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> buy-in from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> could be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> idea overall.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> forward the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> deprecate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> deprecating,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> same thing is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> just need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> is currently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Regarding my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> little sloppy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> passing in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> complicated to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>> On Fri, Mar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> I read your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> looks better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> that KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> going to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> What shall we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Why are you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> as a novice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> Maybe I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> supports
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> added, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> able to call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> The issue I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> worked around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> [Also, great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> On Mar 28,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> The idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> a way to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> You just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> write in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Consider
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> customers who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> store to give
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> This is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> 'brancher'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> class
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>   private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>   private
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>       /*In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> a separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> classes'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> /*Alas,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> without the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Does this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> we need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> [BTW Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> KIP I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> so I will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>> 28.03.2019
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> I tried to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> succeeded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    with no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>    pretty
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> To be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> I will say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> deprecated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> has the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> On Wed,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> OK, let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> First, it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> There are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> 1. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> PROS: 1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> all the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> CONS: The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> 2. (as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> PROS:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> CONS: We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> called,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> I see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Still,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Maybe
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Let me
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> 1) OK,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> drop
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> message
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Well,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> tests,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         ��      >>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> Hello
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> In my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>         ��      >>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> Does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>               
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>        .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> Hi
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>> On
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>                >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>>
>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <mj...@apache.org>.
Ivan, John,

thanks for the details.

I guess I can be convinced in either direction with regard to `null`
handling. But I am less worried about it, because I assume that a Java
programmer understand the ambiguty of `get(key) == null` -- that's why
there is `containsKey()`. In the end it might be a minor detail and as
long as it's well documented all discussed approaches seem fine.
Including the idea to re-introduce the `Consumer` overload.

It's Ivan's KIP so I am fine with whatever he picks :)


About the naming: Not sure why we don't want to make the naming a public
contract? I guess for split() we could default to the empty String, and
for `branch()` we could just add a counter by default?

With no passed names at all, we just generte branche names "1", "2",
"3", etc... With only`split(Named.as("foo-"))` (but not branch names) we
generate "foo-1", "foo-2", etc.

If any branch has a dedicated name, the counting is preserved but just
not used int he name for this brach:

split("foo-")
  .brach()
  .brach(Branched.as("bar")
  .defaultBranch()

would generate "foo-1", "foo-bar", "foo-3"?

Thoughts?



-Matthias

On 5/29/20 1:20 PM, John Roesler wrote:
> Hi all,
> 
> Right, that was my suggestion; sorry for the confusion. I was thinking
> that adding null as a value to maps is generally dubious, for which
> reason a lot of Map implementations actually don't allow it at all.
> 
> The reason is that it creates ambiguity, since a lot of code treats
> "map.get(key) == null" as meaning the key is not in the map, but
> if we place a null value in the map (presuming we don't immediately
> get an exception), then you face weird contradictions, like
> get(key) == null, but containsKey(key) can be either true or false, and
> you may or may not see it while iterating. Which would violate the
> expectations of many Java programs. You can just take a look at
> the AK codebase, and you'll find many occurrences where we
> assume a null mapping means the map doesn't contain the key.
> 
> Of course, the practical question, which Ivan brought up, is also
> a good one. What would you actually do with these null values
> in the map, besides get an NPE ?
> 
> I'm not sure why I didn't think of this before, but an alternative
> to this debate is to go back to adding Consumer into the API, but
> with the same method name, withChain(Consumer<KStream<K,V>>).
> 
> Now that we have only static methods, it's less confusing because it's
> not possible to choose _both_ the Function<KStream,KStream> _and_
> the Consumer<KStream> in the same branch (which was one of the
> main drawbacks of the original plan wrt withChain and
> withJavaConsumer as builder methods.
> 
> I.e., I was previously concerned about:
> Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2)
> It's just a little confusing trying to reason about how this is all going to wire up.
> 
> But with only the static methods, users have only three methods,
> and they are all clear:
> 1. Branched.as(name): just names the branch, the branch-predicated stream is the value of the map
> 2. Branched.with(name?, Function<KStream,KStream>): Maybe names the branch, applies the transformation after the branch predicate, and the transformed stream is the value of the map (null is not permitted)
> 3. Branched.with(name?, Consumer<KStream>): Maybe names the branch, passes the predicated stream in to the Consumer function and omits the branch from the map
> 
> Sorry for digging this option up again, but it's starting to look more
> attractive after we dropped the builder functions, and I think it also
> resolves the "null" concern.
> 
> Thoughts?
> -John
> 
> 
> On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
>> Hi Matthias!
>>
>> Thanks for your reply!
>>
>> (1)
>>
>>  > Do you imply that there won't be a `name -> null` entry in the `Map` 
>> for the branch? If yes, I am wondering why?
>>
>> Ah, I re-read the discussion and found out that omitting null entries 
>> was John's idea, not yours :-)
>>
>> Anyway. As you remember, I was against the 'swallowing' of branches by 
>> Consumer. I had a feeling that there are cases when I would like both to 
>> consume a branch and post-process it in a Map handler.
>>
>> But then the idea of getting rid of `withJavaConsumer` came up. And then 
>> we realized that we should handle nulls somehow.
>>
>> So it looked like taking the best from both approaches: if we want to 
>> omit a branch in the resulting map, we just return null, thus emulating 
>> a 'swallowing consumer'. If we want to include a branch, we return it. 
>> It provides the full control for a user, and it spares the user from 
>> null-checking in their code.
>>
>>  > My reasoning is, that it's very clear that `name -> null` would be 
>> there, because the user code did execute `return null`.
>>
>> My reasoning is following. The way such lambdas are written, it's hard 
>> to return null unintentionally. And if a user returns null 
>> intentionally, what `name->null` in the resulting Map can be good for? 
>> It's either NPE (which is clearly not intended) or something that should 
>> be null-checked and skipped. Ok, let's do it!
>>
>> But, I'm ready to be convinced to drop this rule. It doesn't seem to me 
>> to be a principal thing.
>>
>> (2)
>>
>>  > I think it would be good if the KIP could explains the intended 
>> default naming schema. This is important because the naming schema must 
>> be part of the API contract; otherwise, users cannot rely on the naming 
>> when trying to use the returned `Map`.
>>
>> Well, in fact, I intentionally didn't include any naming schema in the 
>> KIP. And it's done exactly for the purpose that a user won't be able to 
>> rely on default naming. If they want to use certain branches from the 
>> Map, they will devise their own naming strategy that will never change 
>> and will never let them down.
>>
>> If they don't mind what's in a Map, if they are looking up the Map just 
>> for debugging, or if they want to treat a Map just like a Collection of 
>> values (a possible use case!), it doesn't matter how the branches are named.
>>
>> This way we are giving ourselves the full freedom to change the naming 
>> schema afterwards without violating the API contract.
>>
>>
>> Regards,
>>
>> Ivan
>>
>> 29.05.2020 4:58, Matthias J. Sax пишет:
>>> Thanks for updating the KIP!
>>>
>>> Using covariant generics is a good idea! I am also fine with only using
>>> static method in `Branched` for now, as it's only two parameters and
>>> thus not too many overloads.
>>>
>>>
>>> (1) What is unclear to me is, what you exaclty mean by:
>>>
>>>> If a function returns `null`, its result is omitted.
>>>
>>> Do you imply that there won't be a `name -> null` entry in the `Map` for
>>> the branch? If yes, I am wondering why?
>>>
>>> My previous argument was to omit an entry only for the
>>> `withJavaConsumer()` case, because the return type is "void" (and thus
>>> it's unclear what should be added and it would introduce inconsistencies
>>> in the `Map`).
>>>
>>> However, for a `Function` that returns a `KStream`, we could actually
>>> just add a `name -> null` entry if `null` is returned. My reasoning is,
>>> that it's very clear that `name -> null` would be there, because the
>>> user code did execute `return null`. Adding a `name -> null` entry for
>>> this case is "simpler" as it implies fewer "rules" (ie, avoids an
>>> exceptional case for handling `null`).
>>>
>>> Thoughts?
>>>
>>>
>>> (2) For `Branched.as()` the KIP says:
>>>
>>>> sets the name of the branch (auto-generated by default, when split operation is named, then the names are index-suffixed)
>>>
>>> I think it would be good if the KIP could explains the intended default
>>> naming schema. This is important because the naming schema must be part
>>> of the API contract; otherwise, users cannot rely on the naming when
>>> trying to use the returned `Map`.
>>>
>>> Similalry, it should be explained how names are generated if
>>> `split(Named)` is used. Ie, there might be 4 or 5 combinations how the
>>> API can be mixed and matched and it's unclear atm how it would work in
>>> detail.
>>>
>>>
>>> Thanks!
>>>
>>>
>>> -Matthias
>>>
>>>
>>>
>>>
>>>
>>>
>>> On 5/28/20 7:35 AM, John Roesler wrote:
>>>> Hi Ivan,
>>>>
>>>> Thanks for the updates. I agree, it seems like all the concerns
>>>> that have been raised in the discussion so far have been
>>>> addressed. And it's been a while since anyone raised a new
>>>> concern. At this point, it seems like a good time to start
>>>> the VOTE thread.
>>>>
>>>> Sometimes, the vote thread will trigger new people to look
>>>> into the KIP, and they may raise new concerns, but it's not a
>>>> problem. We'll just address those lingering concerns if there
>>>> are any, until you have all the votes you need.
>>>>
>>>> Thanks again for the contribution!
>>>>
>>>> -John
>>>>
>>>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>>>> Alright, I have updated the KIP with what we have discussed:
>>>>>
>>>>> 1. Per Mathhias's suggestion, if a chain function returns null, the
>>>>> respective result if omitted in the resulting Map.
>>>>>
>>>>> 2. `with[Java]Consumer` method dropped.
>>>>>
>>>>> 3. `Branched` class has only three static methods with all the possible
>>>>> combinations of parameters.
>>>>>
>>>>> 4. Chain function is defined 'fully covariant', let's see if we can
>>>>> implement it this way :-))
>>>>>
>>>>> + code example updates and minor edits.
>>>>>
>>>>>
>>>>> Since this is my first KIP, I'm not sure what should I do next. I feel
>>>>> that we talked over all the details and the consensus is reached. Is it
>>>>> OK to call for VOTE now or is it better to wait for more feedback?
>>>>>
>>>>> Regards,
>>>>>
>>>>> Ivan
>>>>>
>>>>>
>>>>>
>>>>> 28.05.2020 3:26, John Roesler пишет:
>>>>>> Sounds good to me, Ivan!
>>>>>> -John
>>>>>>
>>>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>>>> John,
>>>>>>>
>>>>>>> ---------------------
>>>>>>>
>>>>>>>    > I'm sorry things have been dragging out a little, but I have the
>>>>>>> sense we're very close to the end of this discussion, which is exciting.
>>>>>>>
>>>>>>> We are certainly moving forward!  And I'm not in a hurry at all. As I
>>>>>>> told you before in my projects I'm using Spring Kafka's
>>>>>>> KafkaStreamBrancher -- the implementation of the first, rejected version
>>>>>>> of this KIP. It's inferior to what we are discussing here, but it does
>>>>>>> its work. So it's worth to design this KIP really, really well. And by
>>>>>>> the way, from this discussion I'm learning the good API designing
>>>>>>> process. For me it has a value per se :-))
>>>>>>>
>>>>>>> ----------------------
>>>>>>>
>>>>>>>    >>   > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>>    >>   > Do they provide any benefit over just using the right static
>>>>>>> factory?
>>>>>>>    >
>>>>>>>    > I don't have a strong feeling, either. It seems nice to offer a better
>>>>>>>    > type inference experience than what we get with Materialized, by
>>>>>>>    > offering the static method that takes both name and chain.
>>>>>>>    > Given that, there doesn't seem to be a good reason to also offer the
>>>>>>>    > non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>>>>
>>>>>>> I agree again! From a recent discussion on Twitter
>>>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I found out
>>>>>>> an interesting fact about type inference rules in Java. Funny thing is
>>>>>>> that although we need to explicitly set types in a chain like this
>>>>>>>
>>>>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>>>>>>
>>>>>>> (otherwise it won't compile), the composition of static method calls
>>>>>>> works just fine, all the types are being calculated correctly:
>>>>>>>
>>>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>>>>>>
>>>>>>> As I was told, for type inference there is difference between qualifiers
>>>>>>> and arguments, 'you go up if you are argument but stop if you are
>>>>>>> qualifier'. And it also seems that we should not bet on any future
>>>>>>> improvements in Java type inference here.
>>>>>>>
>>>>>>> So,
>>>>>>>
>>>>>>> 1) I think we that in this KIP we should provide three static methods
>>>>>>> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and
>>>>>>> drop any non-static ones.
>>>>>>>
>>>>>>> 2) If anything else will be ever needed, we can easily add anything.
>>>>>>> Maybe this can be done in a process of refinement of all the parameter
>>>>>>> classes.
>>>>>>>
>>>>>>> ----------------------------
>>>>>>>
>>>>>>>    > we may as well hope for the best, and propose the "fully
>>>>>>>    > covariant" definition for now.
>>>>>>>
>>>>>>> Understood and agreed! I will edit the KIP.
>>>>>>>
>>>>>>> ----------------------------
>>>>>>>
>>>>>>>    >> Good question, I already thought about it and rejected the idea....
>>>>>>>    > (I cut off your quote; the rest is in the chain below)
>>>>>>>    >  "Worst" case scenario: someone
>>>>>>>    > else will wish the return type is something different, and we'll go
>>>>>>>    > through a painless deprecation transition to change it later.
>>>>>>>
>>>>>>> Of course, we cant' predict all the ways people are going to use it.
>>>>>>>    From my own humble experience with Kafka Streams, the worst scenario is
>>>>>>> unlikely. Split is split, transform is transform, too much flexibility
>>>>>>> is often evil.
>>>>>>>
>>>>>>> ------------------------
>>>>>>>
>>>>>>> So it seems that we are close to the consensus. Two things to be altered
>>>>>>> in the current version of KIP:
>>>>>>>
>>>>>>> * list of Branched methods, drop non-static methods.
>>>>>>> * 'fully covariant' definition of `chained`.
>>>>>>>
>>>>>>> Any ideas / questions / objections?
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>>>> Thanks for the reply, Ivan,
>>>>>>>>
>>>>>>>> I'm sorry things have been dragging out a little, but I have the sense
>>>>>>>> we're very close to the end of this discussion, which is exciting.
>>>>>>>>
>>>>>>>>>     > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>>>>     > Do they provide any benefit over just using the right static factory?
>>>>>>>>>
>>>>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>>>>> be omitted, I agree!
>>>>>>>>
>>>>>>>> I don't have a strong feeling, either. It seems nice to offer a better
>>>>>>>> type inference experience than what we get with Materialized, by
>>>>>>>> offering the static method that takes both name and chain.
>>>>>>>>
>>>>>>>> Given that, there doesn't seem to be a good reason to also offer the
>>>>>>>> non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>>>>>
>>>>>>>> I'll defer to Matthias, if he has a chance to consider whether it's more
>>>>>>>> valuable to stick with the existing pattern or break the pattern to offer
>>>>>>>> a better experience.
>>>>>>>>
>>>>>>>>>     > You might as well propose the “ideal” API in the KIP, which is the
>>>>>>>>>     > covariant typed function
>>>>>>>>>
>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>>>>> and unit testing?
>>>>>>>>
>>>>>>>> Yep! I know it's a bit sloppy, but my experience has been that we just
>>>>>>>> won't know what works until we really try it, and try it in several different
>>>>>>>> ways. Still, we may as well hope for the best, and propose the "fully
>>>>>>>> covariant" definition for now.
>>>>>>>>
>>>>>>>>>     > Is it necessary to restrict the result key and value types to be the
>>>>>>>>>     > same as the inputs?
>>>>>>>>>
>>>>>>>>> Good question, I already thought about it and rejected the idea....
>>>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>>>
>>>>>>>> That's fair! It's your KIP, after all. I think I might have made a different
>>>>>>>> call here, but I think this choice is fine. "Worst" case scenario: someone
>>>>>>>> else will wish the return type is something different, and we'll go
>>>>>>>> through a painless deprecation transition to change it later. Thanks
>>>>>>>> to the clean design of your API, this doesn't seem to bad. And, of
>>>>>>>> course, you've actually been using similar functionality already, so it
>>>>>>>> seems we should trust your intuition.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> -John
>>>>>>>>
>>>>>>>>
>>>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>>>> Hi John,
>>>>>>>>>
>>>>>>>>>     > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>>>> Do they provide any benefit over just using the right static factory?
>>>>>>>>>
>>>>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>>>>> be omitted, I agree!
>>>>>>>>>
>>>>>>>>>     > You might as well propose the “ideal” API in the KIP, which is the
>>>>>>>>> covariant typed function
>>>>>>>>>
>>>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>>>>> and unit testing?
>>>>>>>>>
>>>>>>>>>     > Is it necessary to restrict the result key and value types to be the
>>>>>>>>> same as the inputs?
>>>>>>>>>
>>>>>>>>> Good question, I already thought about it and rejected the idea.
>>>>>>>>>
>>>>>>>>> Look, if we want to keep `withChain`'s function optional, then we must
>>>>>>>>> keep the result key and value types the same. Because for now, the
>>>>>>>>> default value for the 'chain function' is Function.identity().
>>>>>>>>>
>>>>>>>>> Of course, we can make the 'chain function' required. But I think this
>>>>>>>>> is not what `split` method is for. `split` is for splitting, not
>>>>>>>>> transforming, and `chainFunction` in most of the cases should be either
>>>>>>>>> a consumer or the identity function.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>>>
>>>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
>>>>>>>>>>
>>>>>>>>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
>>>>>>>>>>
>>>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
>>>>>>>>>>
>>>>>>>>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
>>>>>>>>>>
>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
>>>>>>>>>>
>>>>>>>>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> John
>>>>>>>>>>
>>>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>>>> Hello John,
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 1.
>>>>>>>>>>> ---------------------------------------------
>>>>>>>>>>>
>>>>>>>>>>>      > Perhaps it would be better to stick with "as" for now
>>>>>>>>>>>      > and just file a Jira to switch them all at the same time [for
>>>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>>>
>>>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
>>>>>>>>>>> standard workaround
>>>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>>>>>>> So actually this should be a very low priority issue, if an issue at
>>>>>>>>>>> all.
>>>>>>>>>>>
>>>>>>>>>>>      > I don't understand how your new proposed
>>>>>>>>>>>      > methods would work any differently than the ones you already
>>>>>>>>>>>      > had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>>>>      > the generic type parameters on the first static factory call. Can you
>>>>>>>>>>>      > explain how your new interface proposal differs from the existing KIP?
>>>>>>>>>>>
>>>>>>>>>>> In the KIP, I didn't clarify what methods should be static. Now I
>>>>>>>>>>> propose the following methods:
>>>>>>>>>>>
>>>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>>>
>>>>>>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>>>>>>
>>>>>>>>>>> The overloaded `with` version that provides both Function and name can
>>>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>>>
>>>>>>>>>>> 2.
>>>>>>>>>>> ----------------------------
>>>>>>>>>>>
>>>>>>>>>>>      > Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>>>>      > be nice, but I'm not sure it will actually work.
>>>>>>>>>>>
>>>>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>>>>
>>>>>>>>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
>>>>>>>>>>>              return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>>>          s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>>>                     //Won't compile!!
>>>>>>>>>>>                     Branched.with(Me::func));
>>>>>>>>>>>
>>>>>>>>>>> The simple workaround here is to change `func`'s return type from
>>>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>>>
>>>>>>>>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
>>>>>>>>>>> `Branched`, so during code migration I will have to modify my functions'
>>>>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>>>>>>
>>>>>>>>>>>      >  the map you're returning is Map<K,V>, and of course a K is not the
>>>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>>>
>>>>>>>>>>> I think what you actually meant here is that KStream<? extends K, ?
>>>>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
>>>>>>>>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
>>>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
>>>>>>>>>>>
>>>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
>>>>>>>>>>> for now it's better to just admit that API is not absolutely perfect and
>>>>>>>>>>> accept it as is, that is
>>>>>>>>>>>
>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>>>> Hello Ivan,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the refinement. Actually, I did not know that "as" would
>>>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from convention
>>>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>>>
>>>>>>>>>>>> The convention is that "as(String name)" is used for the static factory
>>>>>>>>>>>> method, whereas "withName(String name)" is an instance method
>>>>>>>>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
>>>>>>>>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
>>>>>>>>>>>> although it's somewhat dubious, since all the other configuration
>>>>>>>>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
>>>>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>>>>
>>>>>>>>>>>> Re. 3:
>>>>>>>>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
>>>>>>>>>>>> configuraion objects. The problem is that Java infers the type
>>>>>>>>>>>> based on the _first_ method in the chain. While it does consider what
>>>>>>>>>>>> the recipient of the method result wants, it only considers the _next_
>>>>>>>>>>>> recipient.
>>>>>>>>>>>>
>>>>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>>>>>>> Branched<String,String> variable, java infers the type correctly. But
>>>>>>>>>>>> when the "next recipient" is a chained method call, like "withChain",
>>>>>>>>>>>> then the chained method doesn't bound the type (by definition,
>>>>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java will take
>>>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
>>>>>>>>>>>>
>>>>>>>>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
>>>>>>>>>>>> that hard to recursively consider the entire context when inferring the
>>>>>>>>>>>> generic type parameters, but this is what we have to work with.
>>>>>>>>>>>>
>>>>>>>>>>>> To be honest, though, I don't understand how your new proposed
>>>>>>>>>>>> methods would work any differently than the ones you already
>>>>>>>>>>>> had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>>>>> the generic type parameters on the first static factory call. Can you
>>>>>>>>>>>> explain how your new interface proposal differs from the existing KIP?
>>>>>>>>>>>>
>>>>>>>>>>>> Re. 4:
>>>>>>>>>>>> Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
>>>>>>>>>>>> try. In the past, we've run into soe truly strange interactions between the
>>>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
>>>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>>>
>>>>>>>>>>>> Another issue is that the value type of the map you're returning is
>>>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
>>>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks again,
>>>>>>>>>>>> -John
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>>>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>>>>>>>>>>>>> simpler'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I made some quick API mocking in my IDE and tried to implement examples
>>>>>>>>>>>>> from KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1. Having to return something from lambda is not a very big deal.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use method references
>>>>>>>>>>>>> for already written stream consumers, but then I realized that I can
>>>>>>>>>>>>> just change my methods from returning void to returning the input
>>>>>>>>>>>>> parameter and use references to them. Not very convenient, but passable.
>>>>>>>>>>>>>
>>>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>>>>>>>>>>>>> function returns null, we don't insert it into the resulting map.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Usually it's better to implement a non-perfect, but workable solution as
>>>>>>>>>>>>> a first approximation. And later we can always add to `Branched`
>>>>>>>>>>>>> anything we want.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
>>>>>>>>>>>>> should be built? First of all, it seems that `as` now is more preferred
>>>>>>>>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
>>>>>>>>>>>>> `as` operator).
>>>>>>>>>>>>>
>>>>>>>>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
>>>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>>>
>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>          Branched.as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> so I have to write
>>>>>>>>>>>>>
>>>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>>>          Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
>>>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>>>
>>>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>>>
>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>>>
>>>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>>>
>>>>>>>>>>>>> //non-static!
>>>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>>>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 4. And one more. What do you think, do we need that flexibility:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>>>>>>>>>>>>
>>>>>>>>>>>>> vs.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>>>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>>>
>>>>>>>>>>>>> ??
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>>>>>>>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        (1) split a stream and return the substreams for futher processing
>>>>>>>>>>>>>>>        (2) split a stream and modify the substreams with in-place method chaining
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         -> split a stream, modify the substreams, and return the _modified_
>>>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        - using `withChain` I get the modified sub-stream
>>>>>>>>>>>>>>>        - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>       From my understanding the original idea of `withJavaConsumer` was to
>>>>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>         s.to();
>>>>>>>>>>>>>>>         return null;
>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
>>>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Following this train of through, and if we want to allow the "return
>>>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>>>         s.to();
>>>>>>>>>>>>>>>         return s;
>>>>>>>>>>>>>>> })
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>>>>>>>>>>>>> while for the first proposal it adds new functionality (if `return null`
>>>>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I guess I can be convinced either way. However, if we follow your
>>>>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
>>>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>>>>>>>>>>>>> implementation would be to not add the "branch" to the result map if
>>>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>>>>>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> In the current version of KIP it is assumed that the returned map
>>>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>>>>>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>>>>>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>>>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>>>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>             .branch(....)
>>>>>>>>>>>>>>>>>>>>>>             .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>>>             .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>              withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>>>                        .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>>>                        .defaultBranch(...) // creates node
>>>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>              operator(function, config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>            operator(function, function, string)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>>>              withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>              withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>              withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>>>            }
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              me address both comments here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> An open question is the case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              record
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              still
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              access
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              between
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>       ��      >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>>
>>>
>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Hi all,

Right, that was my suggestion; sorry for the confusion. I was thinking
that adding null as a value to maps is generally dubious, for which
reason a lot of Map implementations actually don't allow it at all.

The reason is that it creates ambiguity, since a lot of code treats
"map.get(key) == null" as meaning the key is not in the map, but
if we place a null value in the map (presuming we don't immediately
get an exception), then you face weird contradictions, like
get(key) == null, but containsKey(key) can be either true or false, and
you may or may not see it while iterating. Which would violate the
expectations of many Java programs. You can just take a look at
the AK codebase, and you'll find many occurrences where we
assume a null mapping means the map doesn't contain the key.

Of course, the practical question, which Ivan brought up, is also
a good one. What would you actually do with these null values
in the map, besides get an NPE ?

I'm not sure why I didn't think of this before, but an alternative
to this debate is to go back to adding Consumer into the API, but
with the same method name, withChain(Consumer<KStream<K,V>>).

Now that we have only static methods, it's less confusing because it's
not possible to choose _both_ the Function<KStream,KStream> _and_
the Consumer<KStream> in the same branch (which was one of the
main drawbacks of the original plan wrt withChain and
withJavaConsumer as builder methods.

I.e., I was previously concerned about:
Branched.as(name).withChain(chain1).withJavaConsumer(cons1).withChain(chain2)
It's just a little confusing trying to reason about how this is all going to wire up.

But with only the static methods, users have only three methods,
and they are all clear:
1. Branched.as(name): just names the branch, the branch-predicated stream is the value of the map
2. Branched.with(name?, Function<KStream,KStream>): Maybe names the branch, applies the transformation after the branch predicate, and the transformed stream is the value of the map (null is not permitted)
3. Branched.with(name?, Consumer<KStream>): Maybe names the branch, passes the predicated stream in to the Consumer function and omits the branch from the map

Sorry for digging this option up again, but it's starting to look more
attractive after we dropped the builder functions, and I think it also
resolves the "null" concern.

Thoughts?
-John


On Fri, May 29, 2020, at 02:34, Ivan Ponomarev wrote:
> Hi Matthias!
> 
> Thanks for your reply!
> 
> (1)
> 
>  > Do you imply that there won't be a `name -> null` entry in the `Map` 
> for the branch? If yes, I am wondering why?
> 
> Ah, I re-read the discussion and found out that omitting null entries 
> was John's idea, not yours :-)
> 
> Anyway. As you remember, I was against the 'swallowing' of branches by 
> Consumer. I had a feeling that there are cases when I would like both to 
> consume a branch and post-process it in a Map handler.
> 
> But then the idea of getting rid of `withJavaConsumer` came up. And then 
> we realized that we should handle nulls somehow.
> 
> So it looked like taking the best from both approaches: if we want to 
> omit a branch in the resulting map, we just return null, thus emulating 
> a 'swallowing consumer'. If we want to include a branch, we return it. 
> It provides the full control for a user, and it spares the user from 
> null-checking in their code.
> 
>  > My reasoning is, that it's very clear that `name -> null` would be 
> there, because the user code did execute `return null`.
> 
> My reasoning is following. The way such lambdas are written, it's hard 
> to return null unintentionally. And if a user returns null 
> intentionally, what `name->null` in the resulting Map can be good for? 
> It's either NPE (which is clearly not intended) or something that should 
> be null-checked and skipped. Ok, let's do it!
> 
> But, I'm ready to be convinced to drop this rule. It doesn't seem to me 
> to be a principal thing.
> 
> (2)
> 
>  > I think it would be good if the KIP could explains the intended 
> default naming schema. This is important because the naming schema must 
> be part of the API contract; otherwise, users cannot rely on the naming 
> when trying to use the returned `Map`.
> 
> Well, in fact, I intentionally didn't include any naming schema in the 
> KIP. And it's done exactly for the purpose that a user won't be able to 
> rely on default naming. If they want to use certain branches from the 
> Map, they will devise their own naming strategy that will never change 
> and will never let them down.
> 
> If they don't mind what's in a Map, if they are looking up the Map just 
> for debugging, or if they want to treat a Map just like a Collection of 
> values (a possible use case!), it doesn't matter how the branches are named.
> 
> This way we are giving ourselves the full freedom to change the naming 
> schema afterwards without violating the API contract.
> 
> 
> Regards,
> 
> Ivan
> 
> 29.05.2020 4:58, Matthias J. Sax пишет:
> > Thanks for updating the KIP!
> > 
> > Using covariant generics is a good idea! I am also fine with only using
> > static method in `Branched` for now, as it's only two parameters and
> > thus not too many overloads.
> > 
> > 
> > (1) What is unclear to me is, what you exaclty mean by:
> > 
> >> If a function returns `null`, its result is omitted.
> > 
> > Do you imply that there won't be a `name -> null` entry in the `Map` for
> > the branch? If yes, I am wondering why?
> > 
> > My previous argument was to omit an entry only for the
> > `withJavaConsumer()` case, because the return type is "void" (and thus
> > it's unclear what should be added and it would introduce inconsistencies
> > in the `Map`).
> > 
> > However, for a `Function` that returns a `KStream`, we could actually
> > just add a `name -> null` entry if `null` is returned. My reasoning is,
> > that it's very clear that `name -> null` would be there, because the
> > user code did execute `return null`. Adding a `name -> null` entry for
> > this case is "simpler" as it implies fewer "rules" (ie, avoids an
> > exceptional case for handling `null`).
> > 
> > Thoughts?
> > 
> > 
> > (2) For `Branched.as()` the KIP says:
> > 
> >> sets the name of the branch (auto-generated by default, when split operation is named, then the names are index-suffixed)
> > 
> > I think it would be good if the KIP could explains the intended default
> > naming schema. This is important because the naming schema must be part
> > of the API contract; otherwise, users cannot rely on the naming when
> > trying to use the returned `Map`.
> > 
> > Similalry, it should be explained how names are generated if
> > `split(Named)` is used. Ie, there might be 4 or 5 combinations how the
> > API can be mixed and matched and it's unclear atm how it would work in
> > detail.
> > 
> > 
> > Thanks!
> > 
> > 
> > -Matthias
> > 
> > 
> > 
> > 
> > 
> > 
> > On 5/28/20 7:35 AM, John Roesler wrote:
> >> Hi Ivan,
> >>
> >> Thanks for the updates. I agree, it seems like all the concerns
> >> that have been raised in the discussion so far have been
> >> addressed. And it's been a while since anyone raised a new
> >> concern. At this point, it seems like a good time to start
> >> the VOTE thread.
> >>
> >> Sometimes, the vote thread will trigger new people to look
> >> into the KIP, and they may raise new concerns, but it's not a
> >> problem. We'll just address those lingering concerns if there
> >> are any, until you have all the votes you need.
> >>
> >> Thanks again for the contribution!
> >>
> >> -John
> >>
> >> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
> >>> Alright, I have updated the KIP with what we have discussed:
> >>>
> >>> 1. Per Mathhias's suggestion, if a chain function returns null, the
> >>> respective result if omitted in the resulting Map.
> >>>
> >>> 2. `with[Java]Consumer` method dropped.
> >>>
> >>> 3. `Branched` class has only three static methods with all the possible
> >>> combinations of parameters.
> >>>
> >>> 4. Chain function is defined 'fully covariant', let's see if we can
> >>> implement it this way :-))
> >>>
> >>> + code example updates and minor edits.
> >>>
> >>>
> >>> Since this is my first KIP, I'm not sure what should I do next. I feel
> >>> that we talked over all the details and the consensus is reached. Is it
> >>> OK to call for VOTE now or is it better to wait for more feedback?
> >>>
> >>> Regards,
> >>>
> >>> Ivan
> >>>
> >>>
> >>>
> >>> 28.05.2020 3:26, John Roesler пишет:
> >>>> Sounds good to me, Ivan!
> >>>> -John
> >>>>
> >>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
> >>>>> John,
> >>>>>
> >>>>> ---------------------
> >>>>>
> >>>>>    > I'm sorry things have been dragging out a little, but I have the
> >>>>> sense we're very close to the end of this discussion, which is exciting.
> >>>>>
> >>>>> We are certainly moving forward!  And I'm not in a hurry at all. As I
> >>>>> told you before in my projects I'm using Spring Kafka's
> >>>>> KafkaStreamBrancher -- the implementation of the first, rejected version
> >>>>> of this KIP. It's inferior to what we are discussing here, but it does
> >>>>> its work. So it's worth to design this KIP really, really well. And by
> >>>>> the way, from this discussion I'm learning the good API designing
> >>>>> process. For me it has a value per se :-))
> >>>>>
> >>>>> ----------------------
> >>>>>
> >>>>>    >>   > I'd wonder whether we need the non-static builders (like withChain).
> >>>>>    >>   > Do they provide any benefit over just using the right static
> >>>>> factory?
> >>>>>    >
> >>>>>    > I don't have a strong feeling, either. It seems nice to offer a better
> >>>>>    > type inference experience than what we get with Materialized, by
> >>>>>    > offering the static method that takes both name and chain.
> >>>>>    > Given that, there doesn't seem to be a good reason to also offer the
> >>>>>    > non-static builder-style methods, so I guess I'd prefer to drop them.
> >>>>>
> >>>>> I agree again! From a recent discussion on Twitter
> >>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I found out
> >>>>> an interesting fact about type inference rules in Java. Funny thing is
> >>>>> that although we need to explicitly set types in a chain like this
> >>>>>
> >>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
> >>>>>
> >>>>> (otherwise it won't compile), the composition of static method calls
> >>>>> works just fine, all the types are being calculated correctly:
> >>>>>
> >>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
> >>>>>
> >>>>> As I was told, for type inference there is difference between qualifiers
> >>>>> and arguments, 'you go up if you are argument but stop if you are
> >>>>> qualifier'. And it also seems that we should not bet on any future
> >>>>> improvements in Java type inference here.
> >>>>>
> >>>>> So,
> >>>>>
> >>>>> 1) I think we that in this KIP we should provide three static methods
> >>>>> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and
> >>>>> drop any non-static ones.
> >>>>>
> >>>>> 2) If anything else will be ever needed, we can easily add anything.
> >>>>> Maybe this can be done in a process of refinement of all the parameter
> >>>>> classes.
> >>>>>
> >>>>> ----------------------------
> >>>>>
> >>>>>    > we may as well hope for the best, and propose the "fully
> >>>>>    > covariant" definition for now.
> >>>>>
> >>>>> Understood and agreed! I will edit the KIP.
> >>>>>
> >>>>> ----------------------------
> >>>>>
> >>>>>    >> Good question, I already thought about it and rejected the idea....
> >>>>>    > (I cut off your quote; the rest is in the chain below)
> >>>>>    >  "Worst" case scenario: someone
> >>>>>    > else will wish the return type is something different, and we'll go
> >>>>>    > through a painless deprecation transition to change it later.
> >>>>>
> >>>>> Of course, we cant' predict all the ways people are going to use it.
> >>>>>    From my own humble experience with Kafka Streams, the worst scenario is
> >>>>> unlikely. Split is split, transform is transform, too much flexibility
> >>>>> is often evil.
> >>>>>
> >>>>> ------------------------
> >>>>>
> >>>>> So it seems that we are close to the consensus. Two things to be altered
> >>>>> in the current version of KIP:
> >>>>>
> >>>>> * list of Branched methods, drop non-static methods.
> >>>>> * 'fully covariant' definition of `chained`.
> >>>>>
> >>>>> Any ideas / questions / objections?
> >>>>>
> >>>>> Regards,
> >>>>>
> >>>>> Ivan
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> 27.05.2020 7:03, John Roesler пишет:
> >>>>>> Thanks for the reply, Ivan,
> >>>>>>
> >>>>>> I'm sorry things have been dragging out a little, but I have the sense
> >>>>>> we're very close to the end of this discussion, which is exciting.
> >>>>>>
> >>>>>>>     > I'd wonder whether we need the non-static builders (like withChain).
> >>>>>>>     > Do they provide any benefit over just using the right static factory?
> >>>>>>>
> >>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
> >>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
> >>>>>>> be omitted, I agree!
> >>>>>>
> >>>>>> I don't have a strong feeling, either. It seems nice to offer a better
> >>>>>> type inference experience than what we get with Materialized, by
> >>>>>> offering the static method that takes both name and chain.
> >>>>>>
> >>>>>> Given that, there doesn't seem to be a good reason to also offer the
> >>>>>> non-static builder-style methods, so I guess I'd prefer to drop them.
> >>>>>>
> >>>>>> I'll defer to Matthias, if he has a chance to consider whether it's more
> >>>>>> valuable to stick with the existing pattern or break the pattern to offer
> >>>>>> a better experience.
> >>>>>>
> >>>>>>>     > You might as well propose the “ideal” API in the KIP, which is the
> >>>>>>>     > covariant typed function
> >>>>>>>
> >>>>>>> I didn't quite get it. Do I get you right that you propose the
> >>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
> >>>>>>> then just see if there are any obstacles/pitfalls during implementation
> >>>>>>> and unit testing?
> >>>>>>
> >>>>>> Yep! I know it's a bit sloppy, but my experience has been that we just
> >>>>>> won't know what works until we really try it, and try it in several different
> >>>>>> ways. Still, we may as well hope for the best, and propose the "fully
> >>>>>> covariant" definition for now.
> >>>>>>
> >>>>>>>     > Is it necessary to restrict the result key and value types to be the
> >>>>>>>     > same as the inputs?
> >>>>>>>
> >>>>>>> Good question, I already thought about it and rejected the idea....
> >>>>>> (I cut off your quote; the rest is in the chain below)
> >>>>>>
> >>>>>> That's fair! It's your KIP, after all. I think I might have made a different
> >>>>>> call here, but I think this choice is fine. "Worst" case scenario: someone
> >>>>>> else will wish the return type is something different, and we'll go
> >>>>>> through a painless deprecation transition to change it later. Thanks
> >>>>>> to the clean design of your API, this doesn't seem to bad. And, of
> >>>>>> course, you've actually been using similar functionality already, so it
> >>>>>> seems we should trust your intuition.
> >>>>>>
> >>>>>> Thanks,
> >>>>>> -John
> >>>>>>
> >>>>>>
> >>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
> >>>>>>> Hi John,
> >>>>>>>
> >>>>>>>     > I'd wonder whether we need the non-static builders (like withChain).
> >>>>>>> Do they provide any benefit over just using the right static factory?
> >>>>>>>
> >>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
> >>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
> >>>>>>> be omitted, I agree!
> >>>>>>>
> >>>>>>>     > You might as well propose the “ideal” API in the KIP, which is the
> >>>>>>> covariant typed function
> >>>>>>>
> >>>>>>> I didn't quite get it. Do I get you right that you propose the
> >>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
> >>>>>>> then just see if there are any obstacles/pitfalls during implementation
> >>>>>>> and unit testing?
> >>>>>>>
> >>>>>>>     > Is it necessary to restrict the result key and value types to be the
> >>>>>>> same as the inputs?
> >>>>>>>
> >>>>>>> Good question, I already thought about it and rejected the idea.
> >>>>>>>
> >>>>>>> Look, if we want to keep `withChain`'s function optional, then we must
> >>>>>>> keep the result key and value types the same. Because for now, the
> >>>>>>> default value for the 'chain function' is Function.identity().
> >>>>>>>
> >>>>>>> Of course, we can make the 'chain function' required. But I think this
> >>>>>>> is not what `split` method is for. `split` is for splitting, not
> >>>>>>> transforming, and `chainFunction` in most of the cases should be either
> >>>>>>> a consumer or the identity function.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>>
> >>>>>>> Ivan
> >>>>>>>
> >>>>>>>
> >>>>>>> 24.05.2020 17:15, John Roesler пишет:
> >>>>>>>> Thanks for the reply, Ivan.
> >>>>>>>>
> >>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
> >>>>>>>>
> >>>>>>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
> >>>>>>>>
> >>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
> >>>>>>>>
> >>>>>>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
> >>>>>>>>
> >>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
> >>>>>>>>
> >>>>>>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>> John
> >>>>>>>>
> >>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
> >>>>>>>>> Hello John,
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 1.
> >>>>>>>>> ---------------------------------------------
> >>>>>>>>>
> >>>>>>>>>      > Perhaps it would be better to stick with "as" for now
> >>>>>>>>>      > and just file a Jira to switch them all at the same time [for
> >>>>>>>>> compatibility with Kotlin]
> >>>>>>>>>
> >>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
> >>>>>>>>> standard workaround
> >>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
> >>>>>>>>> So actually this should be a very low priority issue, if an issue at
> >>>>>>>>> all.
> >>>>>>>>>
> >>>>>>>>>      > I don't understand how your new proposed
> >>>>>>>>>      > methods would work any differently than the ones you already
> >>>>>>>>>      > had proposed in the KIP. It seems like you'd still have to provide
> >>>>>>>>>      > the generic type parameters on the first static factory call. Can you
> >>>>>>>>>      > explain how your new interface proposal differs from the existing KIP?
> >>>>>>>>>
> >>>>>>>>> In the KIP, I didn't clarify what methods should be static. Now I
> >>>>>>>>> propose the following methods:
> >>>>>>>>>
> >>>>>>>>> non-static: withChain(Function), withName(String).
> >>>>>>>>>
> >>>>>>>>> static: as(String), with(Function), with(Function, String).
> >>>>>>>>>
> >>>>>>>>> The overloaded `with` version that provides both Function and name can
> >>>>>>>>> be used without causing type inference problem!!
> >>>>>>>>>
> >>>>>>>>> 2.
> >>>>>>>>> ----------------------------
> >>>>>>>>>
> >>>>>>>>>      > Regarding making the K,V types covariant also, yes, that would indeed
> >>>>>>>>>      > be nice, but I'm not sure it will actually work.
> >>>>>>>>>
> >>>>>>>>> What I'm keeping in mind is the following example: imagine
> >>>>>>>>>
> >>>>>>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
> >>>>>>>>>              return s.mapValues(n -> (Integer) n + 1);
> >>>>>>>>> }
> >>>>>>>>>
> >>>>>>>>> BranchedKStream<String, Number> b =
> >>>>>>>>>          s.split().branch((k, v) -> isInteger(v),
> >>>>>>>>>                     //Won't compile!!
> >>>>>>>>>                     Branched.with(Me::func));
> >>>>>>>>>
> >>>>>>>>> The simple workaround here is to change `func`'s return type from
> >>>>>>>>> KStream<...Integer> to KStream<...Number>.
> >>>>>>>>>
> >>>>>>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
> >>>>>>>>> `Branched`, so during code migration I will have to modify my functions'
> >>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
> >>>>>>>>>
> >>>>>>>>>      >  the map you're returning is Map<K,V>, and of course a K is not the
> >>>>>>>>> same as "? extends K", so it doesn't seem compatible.
> >>>>>>>>>
> >>>>>>>>> I think what you actually meant here is that KStream<? extends K, ?
> >>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
> >>>>>>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
> >>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
> >>>>>>>>>
> >>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
> >>>>>>>>> for now it's better to just admit that API is not absolutely perfect and
> >>>>>>>>> accept it as is, that is
> >>>>>>>>>
> >>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>>
> >>>>>>>>> Ivan
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
> >>>>>>>>>> Hello Ivan,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the refinement. Actually, I did not know that "as" would
> >>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from convention
> >>>>>>>>>> and just avoid methods named "as" in the future.
> >>>>>>>>>>
> >>>>>>>>>> The convention is that "as(String name)" is used for the static factory
> >>>>>>>>>> method, whereas "withName(String name)" is an instance method
> >>>>>>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
> >>>>>>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
> >>>>>>>>>> although it's somewhat dubious, since all the other configuration
> >>>>>>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
> >>>>>>>>>> and just file a Jira to switch them all at the same time.
> >>>>>>>>>>
> >>>>>>>>>> Re. 3:
> >>>>>>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
> >>>>>>>>>> configuraion objects. The problem is that Java infers the type
> >>>>>>>>>> based on the _first_ method in the chain. While it does consider what
> >>>>>>>>>> the recipient of the method result wants, it only considers the _next_
> >>>>>>>>>> recipient.
> >>>>>>>>>>
> >>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
> >>>>>>>>>> Branched<String,String> variable, java infers the type correctly. But
> >>>>>>>>>> when the "next recipient" is a chained method call, like "withChain",
> >>>>>>>>>> then the chained method doesn't bound the type (by definition,
> >>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java will take
> >>>>>>>>>> the broadest possible inferece and bind the type to
> >>>>>>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
> >>>>>>>>>>
> >>>>>>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
> >>>>>>>>>> that hard to recursively consider the entire context when inferring the
> >>>>>>>>>> generic type parameters, but this is what we have to work with.
> >>>>>>>>>>
> >>>>>>>>>> To be honest, though, I don't understand how your new proposed
> >>>>>>>>>> methods would work any differently than the ones you already
> >>>>>>>>>> had proposed in the KIP. It seems like you'd still have to provide
> >>>>>>>>>> the generic type parameters on the first static factory call. Can you
> >>>>>>>>>> explain how your new interface proposal differs from the existing KIP?
> >>>>>>>>>>
> >>>>>>>>>> Re. 4:
> >>>>>>>>>> Regarding making the K,V types covariant also, yes, that would indeed
> >>>>>>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
> >>>>>>>>>> try. In the past, we've run into soe truly strange interactions between the
> >>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
> >>>>>>>>>> combination with nested covariant types.
> >>>>>>>>>>
> >>>>>>>>>> Another issue is that the value type of the map you're returning is
> >>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
> >>>>>>>>>> doesn't seem compatible.
> >>>>>>>>>>
> >>>>>>>>>> Thanks again,
> >>>>>>>>>> -John
> >>>>>>>>>>
> >>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
> >>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
> >>>>>>>>>>> simpler'.
> >>>>>>>>>>>
> >>>>>>>>>>> I made some quick API mocking in my IDE and tried to implement examples
> >>>>>>>>>>> from KIP.
> >>>>>>>>>>>
> >>>>>>>>>>> 1. Having to return something from lambda is not a very big deal.
> >>>>>>>>>>>
> >>>>>>>>>>> 2. For a moment I thouht that I won't be able to use method references
> >>>>>>>>>>> for already written stream consumers, but then I realized that I can
> >>>>>>>>>>> just change my methods from returning void to returning the input
> >>>>>>>>>>> parameter and use references to them. Not very convenient, but passable.
> >>>>>>>>>>>
> >>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
> >>>>>>>>>>> function returns null, we don't insert it into the resulting map.
> >>>>>>>>>>>
> >>>>>>>>>>> Usually it's better to implement a non-perfect, but workable solution as
> >>>>>>>>>>> a first approximation. And later we can always add to `Branched`
> >>>>>>>>>>> anything we want.
> >>>>>>>>>>>
> >>>>>>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
> >>>>>>>>>>> should be built? First of all, it seems that `as` now is more preferred
> >>>>>>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
> >>>>>>>>>>> `as` operator).
> >>>>>>>>>>>
> >>>>>>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
> >>>>>>>>>>> infer types in the following construction:
> >>>>>>>>>>>
> >>>>>>>>>>> .branch((key, value) -> value == null,
> >>>>>>>>>>>          Branched.as("foo").withChain(s -> s.mapValues(...)))
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> so I have to write
> >>>>>>>>>>>
> >>>>>>>>>>> .branch((key, value) -> value == null,
> >>>>>>>>>>>          Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
> >>>>>>>>>>> that I came to (will you please validate it):
> >>>>>>>>>>>
> >>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
> >>>>>>>>>>>
> >>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>>>>>>>>> extends KStream<K, V>> chain);
> >>>>>>>>>>>
> >>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>>>>>>>>> extends KStream<K, V>> chain, String name);
> >>>>>>>>>>>
> >>>>>>>>>>> //non-static!
> >>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
> >>>>>>>>>>> KStream<K, V>> chain);
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 4. And one more. What do you think, do we need that flexibility:
> >>>>>>>>>>>
> >>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
> >>>>>>>>>>>
> >>>>>>>>>>> vs.
> >>>>>>>>>>>
> >>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
> >>>>>>>>>>> extends K, ? extends V>> chain
> >>>>>>>>>>>
> >>>>>>>>>>> ??
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>>
> >>>>>>>>>>> Ivan
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
> >>>>>>>>>>>> Thanks for this thought, Matthias,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Your idea has a few aspects I find attractive:
> >>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
> >>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
> >>>>>>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
> >>>>>>>>>>>>
> >>>>>>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks again for sharing the idea,
> >>>>>>>>>>>> John
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> >>>>>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
> >>>>>>>>>>>>> its relationship to the returned `Map`.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Originally, we discussed two main patterns:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        (1) split a stream and return the substreams for futher processing
> >>>>>>>>>>>>>        (2) split a stream and modify the substreams with in-place method chaining
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> To combine both patterns we wanted to allow for
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>         -> split a stream, modify the substreams, and return the _modified_
> >>>>>>>>>>>>> substreams for further processing
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        - using `withChain` I get the modified sub-stream
> >>>>>>>>>>>>>        - using `withJavaConsumer` I get the unmodifed sub-stream
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> This seems to be quite subtle to me.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>       From my understanding the original idea of `withJavaConsumer` was to
> >>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Branched.withChain(s -> {
> >>>>>>>>>>>>>         s.to();
> >>>>>>>>>>>>>         return null;
> >>>>>>>>>>>>> })
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
> >>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
> >>>>>>>>>>>>> -> null` entry in the returned Map.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Following this train of through, and if we want to allow the "return
> >>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
> >>>>>>>>>>>>> an entry to the Map.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
> >>>>>>>>>>>>> be achieved with `withChain`:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Branched.withChain(s -> {
> >>>>>>>>>>>>>         s.to();
> >>>>>>>>>>>>>         return s;
> >>>>>>>>>>>>> })
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> >>>>>>>>>>>>> while for the first proposal it adds new functionality (if `return null`
> >>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
> >>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
> >>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I guess I can be convinced either way. However, if we follow your
> >>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
> >>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
> >>>>>>>>>>>>> preferable as it's simpler to learn.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>> Hello, John, hello Matthias!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thank you very much for your detailed feedback!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -----------------------------------------
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> John,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
> >>>>>>>>>>>>>> all the emails on the web.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> >>>>>>>>>>>>>> method?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
> >>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
> >>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
> >>>>>>>>>>>>>> chain and the result map OR using just the sink
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> This is discussed below.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ----------------------------------------------
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Mathhias,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Done.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> >>>>>>>>>>>>>> method]
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Fixed.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Done in `Proposed Changes` section.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
> >>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
> >>>>>>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
> >>>>>>>>>>>>>> be given to the `Consumer`.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Yes, I agree!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
> >>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> >>>>>>>>>>>>>> implementation would be to not add the "branch" to the result map if
> >>>>>>>>>>>>>> `withConsumer` is used?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
> >>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
> >>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> >>>>>>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> In the current version of KIP it is assumed that the returned map
> >>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
> >>>>>>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
> >>>>>>>>>>>>>> map is the user's responsibility.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
> >>>>>>>>>>>>>> general rule, like 'swallowing' some streams by provided
> >>>>>>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
> >>>>>>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
> >>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
> >>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
> >>>>>>>>>>>>>> it as it is?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ----
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Looking forward for your feedback again!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Ivan.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I also have some minor comment:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> >>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> >>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
> >>>>>>>>>>>>>>> so we just keep them.)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (2) Quote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
> >>>>>>>>>>>>>>>> parameterless alternatives.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
> >>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> >>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
> >>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
> >>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> >>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
> >>>>>>>>>>>>>>> 4th comment:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
> >>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
> >>>>>>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
> >>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
> >>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
> >>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
> >>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (5) Reply to John's comments:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> >>>>>>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
> >>>>>>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
> >>>>>>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
> >>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
> >>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
> >>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> >>>>>>>>>>>>>>>> but I had a few last comments.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>> John
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>> Hello everyone,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I believe that now it follows design principles and takes into account
> >>>>>>>>>>>>>>>>> all the arguments discussed here.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
> >>>>>>>>>>>>>>>>>> rewritten the KIP, see
> >>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> This version includes all the previous discussion results and follows
> >>>>>>>>>>>>>>>>>> the design principles, with one exception.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The exception is
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
> >>>>>>>>>>>>>>>>>> here it is justified.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
> >>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
> >>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
> >>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> >>>>>>>>>>>>>>>>>> as it
> >>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
> >>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
> >>>>>>>>>>>>>>>>>>> build on
> >>>>>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> >>>>>>>>>>>>>>>>>>> want to follow:
> >>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>> Hi everyone!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
> >>>>>>>>>>>>>>>>>>>> June
> >>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
> >>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
> >>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
> >>>>>>>>>>>>>>>>>>>> propose other things before this one is finalized.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
> >>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
> >>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>>>>>>>>>>>>>>             .branch(....)
> >>>>>>>>>>>>>>>>>>>>             .defaultBranch(result::set)
> >>>>>>>>>>>>>>>>>>>>             .onTopOf(someStream);
> >>>>>>>>>>>>>>>>>>>> result.get()...
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
> >>>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
> >>>>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
> >>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
> >>>>>>>>>>>>>>>>>>>> Or,
> >>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
> >>>>>>>>>>>>>>>>>>>> finally,
> >>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
> >>>>>>>>>>>>>>>>>>>> original scope.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> >>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
> >>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
> >>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>> at any point.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> >>>>>>>>>>>>>>>>>>>>> do so.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
> >>>>>>>>>>>>>>>>>>>>>> to mix
> >>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> >>>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> >>>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> >>>>>>>>>>>>>>>>>>>>>>> patterns
> >>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
> >>>>>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>>> totally sense.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> >>>>>>>>>>>>>>>>>>>>>>> `Map` only
> >>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
> >>>>>>>>>>>>>>>>>>>>>>> all of
> >>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> >>>>>>>>>>>>>>>>>>>>>>> `Named` is
> >>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> >>>>>>>>>>>>>>>>>>>>>>> counter
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
> >>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
> >>>>>>>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
> >>>>>>>>>>>>>>>>>>>>>>> misses to
> >>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> >>>>>>>>>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> >>>>>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>>>>>>>>>>>>>>>>>>> specifying a
> >>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
> >>>>>>>>>>>>>>>>>>>>>>> hence
> >>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>              withChain(...);
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> >>>>>>>>>>>>>>>>>>>>>>> does not
> >>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> >>>>>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
> >>>>>>>>>>>>>>>>>>>>>>> make sense
> >>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> >>>>>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
> >>>>>>>>>>>>>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
> >>>>>>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> >>>>>>>>>>>>>>>>>>>>>>> object
> >>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
> >>>>>>>>>>>>>>>>>>>>>>> others,
> >>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
> >>>>>>>>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
> >>>>>>>>>>>>>>>>>>>>>>> methods that
> >>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
> >>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> >>>>>>>>>>>>>>>>>>>>>>> two main
> >>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> >>>>>>>>>>>>>>>>>>>>>>>> satisfy
> >>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
> >>>>>>>>>>>>>>>>>>>>>>>> solves
> >>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> >>>>>>>>>>>>>>>>>>>>>>>> to add
> >>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
> >>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
> >>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> >>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
> >>>>>>>>>>>>>>>>>>>>>>>> names for
> >>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
> >>>>>>>>>>>>>>>>>>>>>>>> logic, it
> >>>>>>>>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
> >>>>>>>>>>>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> >>>>>>>>>>>>>>>>>>>>>>>> processing onto
> >>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> >>>>>>>>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
> >>>>>>>>>>>>>>>>>>>>>>>> proposal, we
> >>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
> >>>>>>>>>>>>>>>>>>>>>>>> name,
> >>>>>>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             stream.split(Named.withName("mysplit")) //creates node
> >>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
> >>>>>>>>>>>>>>>>>>>>>>>>                        .branch(..., ..., "abranch") // creates node
> >>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>>>>>>>>>>>>>>                        .defaultBranch(...) // creates node
> >>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> >>>>>>>>>>>>>>>>>>>>>>>> debate
> >>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
> >>>>>>>>>>>>>>>>>>>>>>>> general,
> >>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>              operator(function, config_object?) OR
> >>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
> >>>>>>>>>>>>>>>>>>>>>>>> variant.
> >>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>            operator(function, function, string)
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
> >>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
> >>>>>>>>>>>>>>>>>>>>>>>> specified
> >>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
> >>>>>>>>>>>>>>>>>>>>>>>> arguments
> >>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             interface BranchConfig extends NamedOperation {
> >>>>>>>>>>>>>>>>>>>>>>>>              withPredicate(...);
> >>>>>>>>>>>>>>>>>>>>>>>>              withChain(...);
> >>>>>>>>>>>>>>>>>>>>>>>>              withName(...);
> >>>>>>>>>>>>>>>>>>>>>>>>            }
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> >>>>>>>>>>>>>>>>>>>>>>>> more like
> >>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
> >>>>>>>>>>>>>>>>>>>>>>>> makes us
> >>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> >>>>>>>>>>>>>>>>>>>>>>>> purely
> >>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
> >>>>>>>>>>>>>>>>>>>>>>>> overloads
> >>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> WDYT?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> >>>>>>>>>>>>>>>>>>>>>>>>> view.
> >>>>>>>>>>>>>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> >>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> >>>>>>>>>>>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> >>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> >>>>>>>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>>>>>>>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> >>>>>>>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> >>>>>>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
> >>>>>>>>>>>>>>>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> >>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> >>>>>>>>>>>>>>>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>>>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>>>>>>>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>>>>>>>>>>>>>>> language
> >>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
> >>>>>>>>>>>>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> >>>>>>>>>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> >>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              Ivan, I’ll definitely forfeit my point on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              branch(predicate, consumer) solution, I don’t see
> >>>>>>>>>>>>>>>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              for the dynamic case.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              IMO the one trade off to consider at this point is the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              question. I don’t know if I totally agree that “we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              in the same scope” since merging the branches back
> >>>>>>>>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              seems like a perfectly plausible use case that can
> >>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              when the branched streams are in the same scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              for the reasons Ivan listed, I think it is overall the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              solution - working around the scope thing is easy
> >>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>>>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              to.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > Hello everyone, thank you all for joining the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > Well, I don't think the idea of named branches,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              matters) or `branch` method  taking name and Consumer
> >>>>>>>>>>>>>>>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              advantages than drawbacks.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > In my opinion, the only real positive outcome from
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              proposal is that all the returned branches are in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              But 1) we rarely need them in the same scope 2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              workaround for the scope problem, described in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > 'Inlining the complex logic' is not a problem,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              method references instead of lambdas. In real world
> >>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              tend to split the complex logic to methods anyway,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              going to be clean.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > The drawbacks are strong. The cohesion between
> >>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              handlers is lost. We have to define predicates in one
> >>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              handlers in another. This opens the door for bugs:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we forget to define a handler for a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              a handler?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we misspell a name?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > What Michael propose would have been totally OK
> >>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              writing the API in Lua, Ruby or Python. In those
> >>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              "dynamic naming" approach would have looked most
> >>>>>>>>>>>>>>>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              beautiful. But in Java we expect all the problems
> >>>>>>>>>>>>>>>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              identifiers to be eliminated in compile time.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > And if we do, what advantage are we supposed to get
> >>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              point?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > ---
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > Earlier in this discussion John Roesler also
> >>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              without "start branching" operator, and later Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              the case when we have to add a dynamic number of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              me address both comments here.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > 1) "Start branching" operator (I think that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              for it indeed) is critical when we need to do a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              see example below.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > 2) No, dynamic branching in current KIP is not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              Imagine a real-world scenario when you need one
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>>>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              value (say, RecordType). You can have something
> >>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > /*John:if we had to start with stream.branch(...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              have been much messier.*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > KBranchedStream branched = stream.split();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > /*Not clumsy at all :-)*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >             branched = branched.branch((k, v) ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              recordType,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >                     recordType::processRecords);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> I also agree with Michael's observation about
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> current `branch()` implementation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              thinking
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> was more aligned with Paul's proposal to just
> >>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> `branch()` statement and return a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> It makes the code easier to read, and also make the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> An open question is the case for which no
> >>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> specified. Atm, `split()` and `branch()` would
> >>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              `BranchedKStream`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> and the call to `defaultBranch()` that returns the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> (what is not the case atm). Or is this actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> real
> >>>>>>>>>>>>>>>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> because users can just ignore the branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              `defaultBranch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> in the result `Map` ?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> About "inlining": So far, it seems to be a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> preference. I can see arguments for both, but no
> >>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              argument" yet
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> that clearly make the case for one or the other.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> Perhaps inlining is the wrong terminology. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              that a lambda with the full downstream topology be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              it can be a method reference as with Ivan’s original
> >>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              The advantage of putting the predicate and its
> >>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              (Consumer) together in branch() is that they are
> >>>>>>>>>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              to each other.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> Ultimately the downstream code has to live
> >>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              branch trees will be hard to read regardless.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> I'm less enthusiastic about inlining the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              downstream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> functionality. Programs that have deep branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              quickly become
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> harder to read as a single unit.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              <pgwhalen@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Also +1 on the issues/goals as Michael
> >>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              that sets a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> great framework for the discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Regarding the SortedMap solution, my
> >>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              decisions) is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> roughly this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Obviously some ordering is necessary, since
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              construct
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> doesn't work without it, but this solution seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              provides as much
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> associativity as the SortedMap solution,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> directly associates the "conditional" with
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>>>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              The value it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> provides over the KIP solution is the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              the same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> The KIP solution is less "dynamic" than the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              in the sense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> that it is slightly clumsier to add a dynamic
> >>>>>>>>>>>>>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              branches, but it is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              the "static"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> case anyway, and should make it simple and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              fluently declare and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              ignore a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> solution on top of it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> I could also see a middle ground where
> >>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              SortedMap being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> taken in, branch() takes a name and not a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              like this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Pros for that solution:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - no double brace initialization, hopefully
> >>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              readable than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Cons
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - downstream branch logic cannot be specified
> >>>>>>>>>>>>>>>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              makes it harder
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to read top to bottom (like existing API and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> (KBranchedStreams could even work *both* ways
> >>>>>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              that's overdoing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Overall I'm curious how important it is to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              access the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branched KStream in the same scope as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              possible that it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> doesn't need to be handled directly by the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              left up to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I'd like to +1 what Michael said about the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> method, I agree with what he's outlined and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              proceed by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> trying to alleviate these problems.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              important to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> able to cleanly access the individual
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> name->stream), which I thought was the original
> >>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> That said, I don't think we should so easily
> >>>>>>>>>>>>>>>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              double brace
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> anti-pattern or force ours users into it if
> >>>>>>>>>>>>>>>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> avoid...just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> my two cents.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Sophie
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> I’d like to propose a different way of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              To me,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> are three problems with the existing branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> warnings.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 2. The way in which you use the stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              positionally coupled
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 3. It is brittle to extend existing branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              additional code
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> paths.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Using associative constructs instead of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              constructs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> be a stronger approach. Consider a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              looks like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              Predicate<?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> super K,? super V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Branches are given names in a map, and as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              returns a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> mapping of names to streams. The ordering
> >>>>>>>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> maintained
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> because it’s a sorted map. Insert order
> >>>>>>>>>>>>>>>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the order
> >>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> evaluation.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> This solves problem 1 because there are no
> >>>>>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              solves
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 2 because you no longer lean on ordering to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              branch you’re
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> interested in. It solves problem 3 because
> >>>>>>>>>>>>>>>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              another
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> conditional by simply attaching another
> >>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              structure, rather
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> messing with the existing indices.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              historically
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> awkward in Java. I know it’s an
> >>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              voluminously, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> double brace initialization would clean up the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thanks for the update.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> FWIW, I agree with Matthias that the current
> >>>>>>>>>>>>>>>>>>>>>>>>>>> "start
> >>>>>>>>>>>>>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> operator
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> confusing when named the same way as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              "Split"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> like a good name. Alternatively, we can do
> >>>>>>>>>>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> operator at all, and just do:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Tentatively, I think that this branching
> >>>>>>>>>>>>>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> terminal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> way, we don't create ambiguity about how
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              is, `branch`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              `void`, to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> enforce that it comes last, and that there
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              definition of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              there's no
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> default,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              record
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> falls
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> though with no default.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Thanks for updating the KIP and your
> >>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> this is to make the name similar to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> The intend was to avoid name duplication.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> _not_
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> be an array.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> The current proposal is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> IMHO, this reads a little odd, because
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              `branch()` does
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> take any parameters and has different
> >>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>>>>>>>>>>> than the
> >>>>>>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> hidden
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that the first call is `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the others
> >>>>>>>>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Because I suggested to rename
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              I though
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> might be better to also rename
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> naming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> overlap that seems to be confusing. The
> >>>>>>>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> me:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Maybe there is a better alternative to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>>>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              avoid
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> naming overlap.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> cannot
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> a method with such name :-)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Bummer. Didn't consider this. Maybe we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>>>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              with a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> short
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> name?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Can you add the interface
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              with all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> methods? It will be part of public API and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              contained in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              `KBranchedStream#get(int
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> index)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              branched-KStreams. Would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> nice to get your feedback about it. It
> >>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              that users
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> would need to write custom utility code
> >>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              access them.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>>>>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              to get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> I have updated the KIP-418 according to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> I can see your point: this is to make
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> that also returns an array, right? But
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              loss of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> compatibility? We can have overloaded
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              without
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> affecting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> the existing code. Maybe the old
> >>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> deprecated, but this is a subject for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Totally agree with 'addBranch->branch'
> >>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> however, a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> reserved word, so unfortunately we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              with such
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> name
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> :-)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> defaultBranch() does take an
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> is not required?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Absolutely! I think that was just
> >>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              something.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> please revise the new version of the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Thanks for driving the discussion of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> everybody
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> agrees that the current branch() method
> >>>>>>>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> optimal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> I had a quick look into the PR and I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              proposal.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> There
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> are some minor things we need to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              recommend the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> following renaming:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> It's just a suggestion to get slightly
> >>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> In the current PR, defaultBranch() does
> >>>>>>>>>>>>>>>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>>>>>>>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              accepted and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Ie, we should add overloads that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              parameter.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> For the issue that the created
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>>>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              different
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> scopes:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> could we extend `KBranchedStream` with a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              index)` method
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> returns the corresponding "branched"
> >>>>>>>>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              object?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Maybe,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> second argument of `addBranch()` should
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> a `Function<KStream,KStream>` and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              whatever
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Finally, I would also suggest to update
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              makes sense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> revise the KIP and continue the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              we'll
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> whether
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> could be adopted.  It would be great
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>>>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              think this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> idea overall.  I'm not sure if that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              starting a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> vote,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> or if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> there is generally some indication of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> interest
> >>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> That being said, I'll continue the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              assuming
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> forward the solution of "stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> KBranchedStream",
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> favor
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> accomplish
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              similar
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> just need to be sure we're not making
> >>>>>>>>>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> impossible/difficult
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> just a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> little sloppy overall in terms of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> particular,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> passing in the "predicates" and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> modified
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              KStreamLazyBranch is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> I read your code carefully and now I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              convinced: your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> looks better and should work. We just
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> crucial
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> that KStream consumers are invoked as
> >>>>>>>>>>>>>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              And then
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> What shall we do now? I should
> >>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              resume the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Why are you telling that your PR
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> point
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> we go in this direction'? To me it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              starting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> as a novice in this project I might
> >>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>>>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> details.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> invoked
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> they’re
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> added, not during
> >>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              still
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> ought
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> able to call couponIssuer.coupons()
> >>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              depend on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> The issue I mean to point out is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              access
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streams in the same scope as the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              is, not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> solutions.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> [Also, great to hear additional
> >>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              excited
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> hear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> The idea to postpone the wiring of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>>>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>>>>>>>>>>>>>>> me at
> >>>>>>>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> glance,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              same
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> other.  That is, if we wanted to merge
> >>>>>>>>>>>>>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> again
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> don't see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> You just took the words right out
> >>>>>>>>>>>>>>>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> going
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Consider the example from Bill's
> >>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              we need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> identify
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> customers who have bought coffee and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> electronics
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> This is the code I usually write under
> >>>>>>>>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              circumstances
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>       return
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>       /*In the real world the code
> >>>>>>>>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              complex, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> creation of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              order to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> separate
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> >>>>>>>>>>>>>>>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              everything
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> later,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Does this make sense?  In order to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              initialize the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> we need the terminal operation to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> called
> >>>>>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              essentially
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> KIP I was going to write here. I have
> >>>>>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              based on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> experience,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> so I will join the discussion on KIP-401
> >>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              fluent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> compatibility
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    there aren't any direct ones.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              that Java
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> smart
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    distinguish between a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              returning one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> thing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    with no arguments returning
> >>>>>>>>>>>>>>>>>>>>>>>>>>> another
> >>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> >>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              shares
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> >>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              branching.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    pretty in its current form, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              demonstrates
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>>>>>>>>>>>>>>>>>>> request should
> >>>>>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> final
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> even a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> starting point if we go in this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              wanted to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> I will say though, that I'm not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              solution
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> deprecated in favor of this, which
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>>>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> suggested
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> was a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              streams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              is, if we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> wanted
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> them back together again I don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              that.  The
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> either
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              table.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> First, it seems that it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              branch API
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> needs
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> There are two potential ways to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> >>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>> The code
> >>>>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>          ��   >>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> sense
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> all the necessary ingredients are
> >>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              instance
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> defined.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>>>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              miss the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that one
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> of the terminal methods should be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              methods
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> I see your point when you are
> >>>>>>>>>>>>>>>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              implemented the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> easy
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>> method that
> >>>>>>>>>>>>>>>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> >>>>>>>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the only
> >>>>>>>>>>>>>>>>>>>>>>>>>> option
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> besides
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>>>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              want to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> silently
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              predicate. 2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Throwing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>>>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              looks
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> like a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> bad
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              emit a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> `default`
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              a clear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> compiled
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> run?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              compile if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> used
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              method chain
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> >>>>>>>>>>>>>>>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              between
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> instantly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> project
> >>>>>>>>>>>>>>>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              compilation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              required.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> >>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> defaultBranch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              just as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>       ��      >>>>>>>>>>>>>>>>>>>> think it creates an
> >>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>>>>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> user
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> default
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>>>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              which allows
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              to be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> well
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              a clear
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>>>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>>>>>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> "build
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> step"
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> allow users to do other
> >>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              stream.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>>>>> way all
> >>>>>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> do -
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> want to process off the original
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> times,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> >>>>>>>>>>>>>>>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>>>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              operations
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> >>>>>>>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              always need
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> >>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              so we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> more with the original branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> need of
> >>>>>>>>>>>>>>>>>>>>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> object
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              flow,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              API, but I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> find
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              contrasts the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> fluency
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              just call
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> method on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              cases
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> I think the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              is very
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> nice
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              how we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              terminated by
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              incompatible with
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              different
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> name,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> >>>>>>>>>>>>>>>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              KIP?  It
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              while also
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> allowing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              KBranchedStreams
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              ks){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              String> ks){
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              this::handleFirstCase)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              this::handleSecondCase)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> takes a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Consumer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              example in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> how would
> >>>>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>>>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              we had
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> 418.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Please
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              feedback :)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>              >
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Attachments:
> >>>>>>>>>>>>> * signature.asc
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>
> >>>>>
> >>>
> >>>
> > 
> 
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi Matthias!

Thanks for your reply!

(1)

 > Do you imply that there won't be a `name -> null` entry in the `Map` 
for the branch? If yes, I am wondering why?

Ah, I re-read the discussion and found out that omitting null entries 
was John's idea, not yours :-)

Anyway. As you remember, I was against the 'swallowing' of branches by 
Consumer. I had a feeling that there are cases when I would like both to 
consume a branch and post-process it in a Map handler.

But then the idea of getting rid of `withJavaConsumer` came up. And then 
we realized that we should handle nulls somehow.

So it looked like taking the best from both approaches: if we want to 
omit a branch in the resulting map, we just return null, thus emulating 
a 'swallowing consumer'. If we want to include a branch, we return it. 
It provides the full control for a user, and it spares the user from 
null-checking in their code.

 > My reasoning is, that it's very clear that `name -> null` would be 
there, because the user code did execute `return null`.

My reasoning is following. The way such lambdas are written, it's hard 
to return null unintentionally. And if a user returns null 
intentionally, what `name->null` in the resulting Map can be good for? 
It's either NPE (which is clearly not intended) or something that should 
be null-checked and skipped. Ok, let's do it!

But, I'm ready to be convinced to drop this rule. It doesn't seem to me 
to be a principal thing.

(2)

 > I think it would be good if the KIP could explains the intended 
default naming schema. This is important because the naming schema must 
be part of the API contract; otherwise, users cannot rely on the naming 
when trying to use the returned `Map`.

Well, in fact, I intentionally didn't include any naming schema in the 
KIP. And it's done exactly for the purpose that a user won't be able to 
rely on default naming. If they want to use certain branches from the 
Map, they will devise their own naming strategy that will never change 
and will never let them down.

If they don't mind what's in a Map, if they are looking up the Map just 
for debugging, or if they want to treat a Map just like a Collection of 
values (a possible use case!), it doesn't matter how the branches are named.

This way we are giving ourselves the full freedom to change the naming 
schema afterwards without violating the API contract.


Regards,

Ivan

29.05.2020 4:58, Matthias J. Sax пишет:
> Thanks for updating the KIP!
> 
> Using covariant generics is a good idea! I am also fine with only using
> static method in `Branched` for now, as it's only two parameters and
> thus not too many overloads.
> 
> 
> (1) What is unclear to me is, what you exaclty mean by:
> 
>> If a function returns `null`, its result is omitted.
> 
> Do you imply that there won't be a `name -> null` entry in the `Map` for
> the branch? If yes, I am wondering why?
> 
> My previous argument was to omit an entry only for the
> `withJavaConsumer()` case, because the return type is "void" (and thus
> it's unclear what should be added and it would introduce inconsistencies
> in the `Map`).
> 
> However, for a `Function` that returns a `KStream`, we could actually
> just add a `name -> null` entry if `null` is returned. My reasoning is,
> that it's very clear that `name -> null` would be there, because the
> user code did execute `return null`. Adding a `name -> null` entry for
> this case is "simpler" as it implies fewer "rules" (ie, avoids an
> exceptional case for handling `null`).
> 
> Thoughts?
> 
> 
> (2) For `Branched.as()` the KIP says:
> 
>> sets the name of the branch (auto-generated by default, when split operation is named, then the names are index-suffixed)
> 
> I think it would be good if the KIP could explains the intended default
> naming schema. This is important because the naming schema must be part
> of the API contract; otherwise, users cannot rely on the naming when
> trying to use the returned `Map`.
> 
> Similalry, it should be explained how names are generated if
> `split(Named)` is used. Ie, there might be 4 or 5 combinations how the
> API can be mixed and matched and it's unclear atm how it would work in
> detail.
> 
> 
> Thanks!
> 
> 
> -Matthias
> 
> 
> 
> 
> 
> 
> On 5/28/20 7:35 AM, John Roesler wrote:
>> Hi Ivan,
>>
>> Thanks for the updates. I agree, it seems like all the concerns
>> that have been raised in the discussion so far have been
>> addressed. And it's been a while since anyone raised a new
>> concern. At this point, it seems like a good time to start
>> the VOTE thread.
>>
>> Sometimes, the vote thread will trigger new people to look
>> into the KIP, and they may raise new concerns, but it's not a
>> problem. We'll just address those lingering concerns if there
>> are any, until you have all the votes you need.
>>
>> Thanks again for the contribution!
>>
>> -John
>>
>> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>>> Alright, I have updated the KIP with what we have discussed:
>>>
>>> 1. Per Mathhias's suggestion, if a chain function returns null, the
>>> respective result if omitted in the resulting Map.
>>>
>>> 2. `with[Java]Consumer` method dropped.
>>>
>>> 3. `Branched` class has only three static methods with all the possible
>>> combinations of parameters.
>>>
>>> 4. Chain function is defined 'fully covariant', let's see if we can
>>> implement it this way :-))
>>>
>>> + code example updates and minor edits.
>>>
>>>
>>> Since this is my first KIP, I'm not sure what should I do next. I feel
>>> that we talked over all the details and the consensus is reached. Is it
>>> OK to call for VOTE now or is it better to wait for more feedback?
>>>
>>> Regards,
>>>
>>> Ivan
>>>
>>>
>>>
>>> 28.05.2020 3:26, John Roesler пишет:
>>>> Sounds good to me, Ivan!
>>>> -John
>>>>
>>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>>> John,
>>>>>
>>>>> ---------------------
>>>>>
>>>>>    > I'm sorry things have been dragging out a little, but I have the
>>>>> sense we're very close to the end of this discussion, which is exciting.
>>>>>
>>>>> We are certainly moving forward!  And I'm not in a hurry at all. As I
>>>>> told you before in my projects I'm using Spring Kafka's
>>>>> KafkaStreamBrancher -- the implementation of the first, rejected version
>>>>> of this KIP. It's inferior to what we are discussing here, but it does
>>>>> its work. So it's worth to design this KIP really, really well. And by
>>>>> the way, from this discussion I'm learning the good API designing
>>>>> process. For me it has a value per se :-))
>>>>>
>>>>> ----------------------
>>>>>
>>>>>    >>   > I'd wonder whether we need the non-static builders (like withChain).
>>>>>    >>   > Do they provide any benefit over just using the right static
>>>>> factory?
>>>>>    >
>>>>>    > I don't have a strong feeling, either. It seems nice to offer a better
>>>>>    > type inference experience than what we get with Materialized, by
>>>>>    > offering the static method that takes both name and chain.
>>>>>    > Given that, there doesn't seem to be a good reason to also offer the
>>>>>    > non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>>
>>>>> I agree again! From a recent discussion on Twitter
>>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I found out
>>>>> an interesting fact about type inference rules in Java. Funny thing is
>>>>> that although we need to explicitly set types in a chain like this
>>>>>
>>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>>>>
>>>>> (otherwise it won't compile), the composition of static method calls
>>>>> works just fine, all the types are being calculated correctly:
>>>>>
>>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>>>>
>>>>> As I was told, for type inference there is difference between qualifiers
>>>>> and arguments, 'you go up if you are argument but stop if you are
>>>>> qualifier'. And it also seems that we should not bet on any future
>>>>> improvements in Java type inference here.
>>>>>
>>>>> So,
>>>>>
>>>>> 1) I think we that in this KIP we should provide three static methods
>>>>> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and
>>>>> drop any non-static ones.
>>>>>
>>>>> 2) If anything else will be ever needed, we can easily add anything.
>>>>> Maybe this can be done in a process of refinement of all the parameter
>>>>> classes.
>>>>>
>>>>> ----------------------------
>>>>>
>>>>>    > we may as well hope for the best, and propose the "fully
>>>>>    > covariant" definition for now.
>>>>>
>>>>> Understood and agreed! I will edit the KIP.
>>>>>
>>>>> ----------------------------
>>>>>
>>>>>    >> Good question, I already thought about it and rejected the idea....
>>>>>    > (I cut off your quote; the rest is in the chain below)
>>>>>    >  "Worst" case scenario: someone
>>>>>    > else will wish the return type is something different, and we'll go
>>>>>    > through a painless deprecation transition to change it later.
>>>>>
>>>>> Of course, we cant' predict all the ways people are going to use it.
>>>>>    From my own humble experience with Kafka Streams, the worst scenario is
>>>>> unlikely. Split is split, transform is transform, too much flexibility
>>>>> is often evil.
>>>>>
>>>>> ------------------------
>>>>>
>>>>> So it seems that we are close to the consensus. Two things to be altered
>>>>> in the current version of KIP:
>>>>>
>>>>> * list of Branched methods, drop non-static methods.
>>>>> * 'fully covariant' definition of `chained`.
>>>>>
>>>>> Any ideas / questions / objections?
>>>>>
>>>>> Regards,
>>>>>
>>>>> Ivan
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>>> Thanks for the reply, Ivan,
>>>>>>
>>>>>> I'm sorry things have been dragging out a little, but I have the sense
>>>>>> we're very close to the end of this discussion, which is exciting.
>>>>>>
>>>>>>>     > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>>     > Do they provide any benefit over just using the right static factory?
>>>>>>>
>>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>>> be omitted, I agree!
>>>>>>
>>>>>> I don't have a strong feeling, either. It seems nice to offer a better
>>>>>> type inference experience than what we get with Materialized, by
>>>>>> offering the static method that takes both name and chain.
>>>>>>
>>>>>> Given that, there doesn't seem to be a good reason to also offer the
>>>>>> non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>>>
>>>>>> I'll defer to Matthias, if he has a chance to consider whether it's more
>>>>>> valuable to stick with the existing pattern or break the pattern to offer
>>>>>> a better experience.
>>>>>>
>>>>>>>     > You might as well propose the “ideal” API in the KIP, which is the
>>>>>>>     > covariant typed function
>>>>>>>
>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>>> and unit testing?
>>>>>>
>>>>>> Yep! I know it's a bit sloppy, but my experience has been that we just
>>>>>> won't know what works until we really try it, and try it in several different
>>>>>> ways. Still, we may as well hope for the best, and propose the "fully
>>>>>> covariant" definition for now.
>>>>>>
>>>>>>>     > Is it necessary to restrict the result key and value types to be the
>>>>>>>     > same as the inputs?
>>>>>>>
>>>>>>> Good question, I already thought about it and rejected the idea....
>>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>>
>>>>>> That's fair! It's your KIP, after all. I think I might have made a different
>>>>>> call here, but I think this choice is fine. "Worst" case scenario: someone
>>>>>> else will wish the return type is something different, and we'll go
>>>>>> through a painless deprecation transition to change it later. Thanks
>>>>>> to the clean design of your API, this doesn't seem to bad. And, of
>>>>>> course, you've actually been using similar functionality already, so it
>>>>>> seems we should trust your intuition.
>>>>>>
>>>>>> Thanks,
>>>>>> -John
>>>>>>
>>>>>>
>>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>>> Hi John,
>>>>>>>
>>>>>>>     > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>> Do they provide any benefit over just using the right static factory?
>>>>>>>
>>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>>> be omitted, I agree!
>>>>>>>
>>>>>>>     > You might as well propose the “ideal” API in the KIP, which is the
>>>>>>> covariant typed function
>>>>>>>
>>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>>> and unit testing?
>>>>>>>
>>>>>>>     > Is it necessary to restrict the result key and value types to be the
>>>>>>> same as the inputs?
>>>>>>>
>>>>>>> Good question, I already thought about it and rejected the idea.
>>>>>>>
>>>>>>> Look, if we want to keep `withChain`'s function optional, then we must
>>>>>>> keep the result key and value types the same. Because for now, the
>>>>>>> default value for the 'chain function' is Function.identity().
>>>>>>>
>>>>>>> Of course, we can make the 'chain function' required. But I think this
>>>>>>> is not what `split` method is for. `split` is for splitting, not
>>>>>>> transforming, and `chainFunction` in most of the cases should be either
>>>>>>> a consumer or the identity function.
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan
>>>>>>>
>>>>>>>
>>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>>> Thanks for the reply, Ivan.
>>>>>>>>
>>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
>>>>>>>>
>>>>>>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
>>>>>>>>
>>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
>>>>>>>>
>>>>>>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
>>>>>>>>
>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
>>>>>>>>
>>>>>>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> John
>>>>>>>>
>>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>>> Hello John,
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 1.
>>>>>>>>> ---------------------------------------------
>>>>>>>>>
>>>>>>>>>      > Perhaps it would be better to stick with "as" for now
>>>>>>>>>      > and just file a Jira to switch them all at the same time [for
>>>>>>>>> compatibility with Kotlin]
>>>>>>>>>
>>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
>>>>>>>>> standard workaround
>>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>>>>> So actually this should be a very low priority issue, if an issue at
>>>>>>>>> all.
>>>>>>>>>
>>>>>>>>>      > I don't understand how your new proposed
>>>>>>>>>      > methods would work any differently than the ones you already
>>>>>>>>>      > had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>>      > the generic type parameters on the first static factory call. Can you
>>>>>>>>>      > explain how your new interface proposal differs from the existing KIP?
>>>>>>>>>
>>>>>>>>> In the KIP, I didn't clarify what methods should be static. Now I
>>>>>>>>> propose the following methods:
>>>>>>>>>
>>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>>
>>>>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>>>>
>>>>>>>>> The overloaded `with` version that provides both Function and name can
>>>>>>>>> be used without causing type inference problem!!
>>>>>>>>>
>>>>>>>>> 2.
>>>>>>>>> ----------------------------
>>>>>>>>>
>>>>>>>>>      > Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>>      > be nice, but I'm not sure it will actually work.
>>>>>>>>>
>>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>>
>>>>>>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
>>>>>>>>>              return s.mapValues(n -> (Integer) n + 1);
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>>          s.split().branch((k, v) -> isInteger(v),
>>>>>>>>>                     //Won't compile!!
>>>>>>>>>                     Branched.with(Me::func));
>>>>>>>>>
>>>>>>>>> The simple workaround here is to change `func`'s return type from
>>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>>
>>>>>>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
>>>>>>>>> `Branched`, so during code migration I will have to modify my functions'
>>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>>>>
>>>>>>>>>      >  the map you're returning is Map<K,V>, and of course a K is not the
>>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>>
>>>>>>>>> I think what you actually meant here is that KStream<? extends K, ?
>>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
>>>>>>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
>>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
>>>>>>>>>
>>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
>>>>>>>>> for now it's better to just admit that API is not absolutely perfect and
>>>>>>>>> accept it as is, that is
>>>>>>>>>
>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>>> Hello Ivan,
>>>>>>>>>>
>>>>>>>>>> Thanks for the refinement. Actually, I did not know that "as" would
>>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from convention
>>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>>
>>>>>>>>>> The convention is that "as(String name)" is used for the static factory
>>>>>>>>>> method, whereas "withName(String name)" is an instance method
>>>>>>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
>>>>>>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
>>>>>>>>>> although it's somewhat dubious, since all the other configuration
>>>>>>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
>>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>>
>>>>>>>>>> Re. 3:
>>>>>>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
>>>>>>>>>> configuraion objects. The problem is that Java infers the type
>>>>>>>>>> based on the _first_ method in the chain. While it does consider what
>>>>>>>>>> the recipient of the method result wants, it only considers the _next_
>>>>>>>>>> recipient.
>>>>>>>>>>
>>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>>>>> Branched<String,String> variable, java infers the type correctly. But
>>>>>>>>>> when the "next recipient" is a chained method call, like "withChain",
>>>>>>>>>> then the chained method doesn't bound the type (by definition,
>>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java will take
>>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
>>>>>>>>>>
>>>>>>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
>>>>>>>>>> that hard to recursively consider the entire context when inferring the
>>>>>>>>>> generic type parameters, but this is what we have to work with.
>>>>>>>>>>
>>>>>>>>>> To be honest, though, I don't understand how your new proposed
>>>>>>>>>> methods would work any differently than the ones you already
>>>>>>>>>> had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>>> the generic type parameters on the first static factory call. Can you
>>>>>>>>>> explain how your new interface proposal differs from the existing KIP?
>>>>>>>>>>
>>>>>>>>>> Re. 4:
>>>>>>>>>> Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
>>>>>>>>>> try. In the past, we've run into soe truly strange interactions between the
>>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
>>>>>>>>>> combination with nested covariant types.
>>>>>>>>>>
>>>>>>>>>> Another issue is that the value type of the map you're returning is
>>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
>>>>>>>>>> doesn't seem compatible.
>>>>>>>>>>
>>>>>>>>>> Thanks again,
>>>>>>>>>> -John
>>>>>>>>>>
>>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>>>>>>>>>>> simpler'.
>>>>>>>>>>>
>>>>>>>>>>> I made some quick API mocking in my IDE and tried to implement examples
>>>>>>>>>>> from KIP.
>>>>>>>>>>>
>>>>>>>>>>> 1. Having to return something from lambda is not a very big deal.
>>>>>>>>>>>
>>>>>>>>>>> 2. For a moment I thouht that I won't be able to use method references
>>>>>>>>>>> for already written stream consumers, but then I realized that I can
>>>>>>>>>>> just change my methods from returning void to returning the input
>>>>>>>>>>> parameter and use references to them. Not very convenient, but passable.
>>>>>>>>>>>
>>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>>>>>>>>>>> function returns null, we don't insert it into the resulting map.
>>>>>>>>>>>
>>>>>>>>>>> Usually it's better to implement a non-perfect, but workable solution as
>>>>>>>>>>> a first approximation. And later we can always add to `Branched`
>>>>>>>>>>> anything we want.
>>>>>>>>>>>
>>>>>>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
>>>>>>>>>>> should be built? First of all, it seems that `as` now is more preferred
>>>>>>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
>>>>>>>>>>> `as` operator).
>>>>>>>>>>>
>>>>>>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
>>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>>
>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>          Branched.as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> so I have to write
>>>>>>>>>>>
>>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>>          Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
>>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>>
>>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>>
>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>>
>>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>>
>>>>>>>>>>> //non-static!
>>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 4. And one more. What do you think, do we need that flexibility:
>>>>>>>>>>>
>>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>>>>>>>>>>
>>>>>>>>>>> vs.
>>>>>>>>>>>
>>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>>
>>>>>>>>>>> ??
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>>
>>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>>>>>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>>>>>>>>>>
>>>>>>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>>
>>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>>> John
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>
>>>>>>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        (1) split a stream and return the substreams for futher processing
>>>>>>>>>>>>>        (2) split a stream and modify the substreams with in-place method chaining
>>>>>>>>>>>>>
>>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>>
>>>>>>>>>>>>>         -> split a stream, modify the substreams, and return the _modified_
>>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>>
>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>>>>>>>>>>
>>>>>>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        - using `withChain` I get the modified sub-stream
>>>>>>>>>>>>>        - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>>>>>>>>>>
>>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>       From my understanding the original idea of `withJavaConsumer` was to
>>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>         s.to();
>>>>>>>>>>>>>         return null;
>>>>>>>>>>>>> })
>>>>>>>>>>>>>
>>>>>>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
>>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Following this train of through, and if we want to allow the "return
>>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>>         s.to();
>>>>>>>>>>>>>         return s;
>>>>>>>>>>>>> })
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>>>>>>>>>>> while for the first proposal it adds new functionality (if `return null`
>>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> I guess I can be convinced either way. However, if we follow your
>>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
>>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> John,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>>>>>>>>>>> method?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>>>>>>>>>>> method]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>>>>>>>>>>> implementation would be to not add the "branch" to the result map if
>>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>>>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> In the current version of KIP it is assumed that the returned map
>>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>>>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>>>>>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>>>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ----
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>>             .branch(....)
>>>>>>>>>>>>>>>>>>>>             .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>>             .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>              withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>>                        .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>>                        .defaultBranch(...) // creates node
>>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>              operator(function, config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>            operator(function, function, string)
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>>              withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>>              withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>>              withName(...);
>>>>>>>>>>>>>>>>>>>>>>>>            }
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>>            -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>            -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>>              for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>              question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>>              in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>              seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>>              when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>>              solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>>              to.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>              <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>>              matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>>              advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>>              method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>>              tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>>              a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>>              writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>>              all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>>              point?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>              without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>>              the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>>              me address both comments here.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>>              for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>>              value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>>              have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>>              recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>              > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>>              thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> An open question is the case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>              `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>>              `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>>              argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>>              that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>>              it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>>              (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>>              to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>>              <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>              <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>>              downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>              quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>>              <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>>              that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              current
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>              decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              construct
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              call
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>>              The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>>              the same
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>              in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>              the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>>              SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>>              SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>>              like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>              readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>>              makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>              unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>>              existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>>              that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>>              access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>              left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>>              <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>>              proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>              important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>>              <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>>              additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>>              constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>>              looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>>              returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>>              solves
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>>              another
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              historically
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>>              voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>>              <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>>              is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>>              definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>>              there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              record
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>>              should
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>>              `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              I though
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>>              with a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>              with all
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>>              branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>>              that users
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>>              to get
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>>              without
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>>              with such
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>>              something.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>              proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>              recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>>              BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>>              `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>>              accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>>              parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>>              different
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>>              index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>>              object?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>>              whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              current
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>              we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>>              think this
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>>              starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>              similar
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>>>>              KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>>              convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              And then
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>>              starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>>              still
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              access
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>>              excited
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              same
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>>              just
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>>              we need
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>>              in the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>>              circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>>              order to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>>              everything
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>>              initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>>              based on
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>>>>>>>>              (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>>              that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>              need
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>>              shares
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>>              wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              solution
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>              streams
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>>              is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>>              that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              table.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>>              returns
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>          ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>              instance
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>>              methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>>              predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              looks
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>              a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>>              compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>>              between
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>>              required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              just as
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>>       ��      >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>>              to be
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>>              a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>>              stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>>              operations
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              always need
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              so we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>              API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              just call
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>>              cases
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              is very
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>>              how we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>>              incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              different
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>>              KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>>              while also
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>>              KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              example in
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>>              we had
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>>              feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>              >
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Attachments:
>>>>>>>>>>>>> * signature.asc
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>>
>>>
>>>
> 



Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <mj...@apache.org>.
Thanks for updating the KIP!

Using covariant generics is a good idea! I am also fine with only using
static method in `Branched` for now, as it's only two parameters and
thus not too many overloads.


(1) What is unclear to me is, what you exaclty mean by:

> If a function returns `null`, its result is omitted.

Do you imply that there won't be a `name -> null` entry in the `Map` for
the branch? If yes, I am wondering why?

My previous argument was to omit an entry only for the
`withJavaConsumer()` case, because the return type is "void" (and thus
it's unclear what should be added and it would introduce inconsistencies
in the `Map`).

However, for a `Function` that returns a `KStream`, we could actually
just add a `name -> null` entry if `null` is returned. My reasoning is,
that it's very clear that `name -> null` would be there, because the
user code did execute `return null`. Adding a `name -> null` entry for
this case is "simpler" as it implies fewer "rules" (ie, avoids an
exceptional case for handling `null`).

Thoughts?


(2) For `Branched.as()` the KIP says:

> sets the name of the branch (auto-generated by default, when split operation is named, then the names are index-suffixed)

I think it would be good if the KIP could explains the intended default
naming schema. This is important because the naming schema must be part
of the API contract; otherwise, users cannot rely on the naming when
trying to use the returned `Map`.

Similalry, it should be explained how names are generated if
`split(Named)` is used. Ie, there might be 4 or 5 combinations how the
API can be mixed and matched and it's unclear atm how it would work in
detail.


Thanks!


-Matthias






On 5/28/20 7:35 AM, John Roesler wrote:
> Hi Ivan,
> 
> Thanks for the updates. I agree, it seems like all the concerns
> that have been raised in the discussion so far have been
> addressed. And it's been a while since anyone raised a new
> concern. At this point, it seems like a good time to start
> the VOTE thread.
> 
> Sometimes, the vote thread will trigger new people to look
> into the KIP, and they may raise new concerns, but it's not a
> problem. We'll just address those lingering concerns if there
> are any, until you have all the votes you need.
> 
> Thanks again for the contribution!
> 
> -John
> 
> On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
>> Alright, I have updated the KIP with what we have discussed:
>>
>> 1. Per Mathhias's suggestion, if a chain function returns null, the 
>> respective result if omitted in the resulting Map.
>>
>> 2. `with[Java]Consumer` method dropped.
>>
>> 3. `Branched` class has only three static methods with all the possible 
>> combinations of parameters.
>>
>> 4. Chain function is defined 'fully covariant', let's see if we can 
>> implement it this way :-))
>>
>> + code example updates and minor edits.
>>
>>
>> Since this is my first KIP, I'm not sure what should I do next. I feel 
>> that we talked over all the details and the consensus is reached. Is it 
>> OK to call for VOTE now or is it better to wait for more feedback?
>>
>> Regards,
>>
>> Ivan
>>
>>
>>
>> 28.05.2020 3:26, John Roesler пишет:
>>> Sounds good to me, Ivan!
>>> -John
>>>
>>> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>>>> John,
>>>>
>>>> ---------------------
>>>>
>>>>   > I'm sorry things have been dragging out a little, but I have the
>>>> sense we're very close to the end of this discussion, which is exciting.
>>>>
>>>> We are certainly moving forward!  And I'm not in a hurry at all. As I
>>>> told you before in my projects I'm using Spring Kafka's
>>>> KafkaStreamBrancher -- the implementation of the first, rejected version
>>>> of this KIP. It's inferior to what we are discussing here, but it does
>>>> its work. So it's worth to design this KIP really, really well. And by
>>>> the way, from this discussion I'm learning the good API designing
>>>> process. For me it has a value per se :-))
>>>>
>>>> ----------------------
>>>>
>>>>   >>   > I'd wonder whether we need the non-static builders (like withChain).
>>>>   >>   > Do they provide any benefit over just using the right static
>>>> factory?
>>>>   >
>>>>   > I don't have a strong feeling, either. It seems nice to offer a better
>>>>   > type inference experience than what we get with Materialized, by
>>>>   > offering the static method that takes both name and chain.
>>>>   > Given that, there doesn't seem to be a good reason to also offer the
>>>>   > non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>
>>>> I agree again! From a recent discussion on Twitter
>>>> (https://twitter.com/inponomarev/status/1265220044394545153) I found out
>>>> an interesting fact about type inference rules in Java. Funny thing is
>>>> that although we need to explicitly set types in a chain like this
>>>>
>>>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>>>
>>>> (otherwise it won't compile), the composition of static method calls
>>>> works just fine, all the types are being calculated correctly:
>>>>
>>>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>>>
>>>> As I was told, for type inference there is difference between qualifiers
>>>> and arguments, 'you go up if you are argument but stop if you are
>>>> qualifier'. And it also seems that we should not bet on any future
>>>> improvements in Java type inference here.
>>>>
>>>> So,
>>>>
>>>> 1) I think we that in this KIP we should provide three static methods
>>>> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and
>>>> drop any non-static ones.
>>>>
>>>> 2) If anything else will be ever needed, we can easily add anything.
>>>> Maybe this can be done in a process of refinement of all the parameter
>>>> classes.
>>>>
>>>> ----------------------------
>>>>
>>>>   > we may as well hope for the best, and propose the "fully
>>>>   > covariant" definition for now.
>>>>
>>>> Understood and agreed! I will edit the KIP.
>>>>
>>>> ----------------------------
>>>>
>>>>   >> Good question, I already thought about it and rejected the idea....
>>>>   > (I cut off your quote; the rest is in the chain below)
>>>>   >  "Worst" case scenario: someone
>>>>   > else will wish the return type is something different, and we'll go
>>>>   > through a painless deprecation transition to change it later.
>>>>
>>>> Of course, we cant' predict all the ways people are going to use it.
>>>>   From my own humble experience with Kafka Streams, the worst scenario is
>>>> unlikely. Split is split, transform is transform, too much flexibility
>>>> is often evil.
>>>>
>>>> ------------------------
>>>>
>>>> So it seems that we are close to the consensus. Two things to be altered
>>>> in the current version of KIP:
>>>>
>>>> * list of Branched methods, drop non-static methods.
>>>> * 'fully covariant' definition of `chained`.
>>>>
>>>> Any ideas / questions / objections?
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>>
>>>>
>>>>
>>>> 27.05.2020 7:03, John Roesler пишет:
>>>>> Thanks for the reply, Ivan,
>>>>>
>>>>> I'm sorry things have been dragging out a little, but I have the sense
>>>>> we're very close to the end of this discussion, which is exciting.
>>>>>
>>>>>>    > I'd wonder whether we need the non-static builders (like withChain).
>>>>>>    > Do they provide any benefit over just using the right static factory?
>>>>>>
>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>> be omitted, I agree!
>>>>>
>>>>> I don't have a strong feeling, either. It seems nice to offer a better
>>>>> type inference experience than what we get with Materialized, by
>>>>> offering the static method that takes both name and chain.
>>>>>
>>>>> Given that, there doesn't seem to be a good reason to also offer the
>>>>> non-static builder-style methods, so I guess I'd prefer to drop them.
>>>>>
>>>>> I'll defer to Matthias, if he has a chance to consider whether it's more
>>>>> valuable to stick with the existing pattern or break the pattern to offer
>>>>> a better experience.
>>>>>
>>>>>>    > You might as well propose the “ideal” API in the KIP, which is the
>>>>>>    > covariant typed function
>>>>>>
>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>> and unit testing?
>>>>>
>>>>> Yep! I know it's a bit sloppy, but my experience has been that we just
>>>>> won't know what works until we really try it, and try it in several different
>>>>> ways. Still, we may as well hope for the best, and propose the "fully
>>>>> covariant" definition for now.
>>>>>
>>>>>>    > Is it necessary to restrict the result key and value types to be the
>>>>>>    > same as the inputs?
>>>>>>
>>>>>> Good question, I already thought about it and rejected the idea....
>>>>> (I cut off your quote; the rest is in the chain below)
>>>>>
>>>>> That's fair! It's your KIP, after all. I think I might have made a different
>>>>> call here, but I think this choice is fine. "Worst" case scenario: someone
>>>>> else will wish the return type is something different, and we'll go
>>>>> through a painless deprecation transition to change it later. Thanks
>>>>> to the clean design of your API, this doesn't seem to bad. And, of
>>>>> course, you've actually been using similar functionality already, so it
>>>>> seems we should trust your intuition.
>>>>>
>>>>> Thanks,
>>>>> -John
>>>>>
>>>>>
>>>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>>>> Hi John,
>>>>>>
>>>>>>    > I'd wonder whether we need the non-static builders (like withChain).
>>>>>> Do they provide any benefit over just using the right static factory?
>>>>>>
>>>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>>>> be omitted, I agree!
>>>>>>
>>>>>>    > You might as well propose the “ideal” API in the KIP, which is the
>>>>>> covariant typed function
>>>>>>
>>>>>> I didn't quite get it. Do I get you right that you propose the
>>>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>>>> then just see if there are any obstacles/pitfalls during implementation
>>>>>> and unit testing?
>>>>>>
>>>>>>    > Is it necessary to restrict the result key and value types to be the
>>>>>> same as the inputs?
>>>>>>
>>>>>> Good question, I already thought about it and rejected the idea.
>>>>>>
>>>>>> Look, if we want to keep `withChain`'s function optional, then we must
>>>>>> keep the result key and value types the same. Because for now, the
>>>>>> default value for the 'chain function' is Function.identity().
>>>>>>
>>>>>> Of course, we can make the 'chain function' required. But I think this
>>>>>> is not what `split` method is for. `split` is for splitting, not
>>>>>> transforming, and `chainFunction` in most of the cases should be either
>>>>>> a consumer or the identity function.
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>>
>>>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>>>> Thanks for the reply, Ivan.
>>>>>>>
>>>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
>>>>>>>
>>>>>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
>>>>>>>
>>>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
>>>>>>>
>>>>>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
>>>>>>>
>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
>>>>>>>
>>>>>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
>>>>>>>
>>>>>>> Thanks,
>>>>>>> John
>>>>>>>
>>>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>>>> Hello John,
>>>>>>>>
>>>>>>>>
>>>>>>>> 1.
>>>>>>>> ---------------------------------------------
>>>>>>>>
>>>>>>>>     > Perhaps it would be better to stick with "as" for now
>>>>>>>>     > and just file a Jira to switch them all at the same time [for
>>>>>>>> compatibility with Kotlin]
>>>>>>>>
>>>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
>>>>>>>> standard workaround
>>>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>>>> So actually this should be a very low priority issue, if an issue at
>>>>>>>> all.
>>>>>>>>
>>>>>>>>     > I don't understand how your new proposed
>>>>>>>>     > methods would work any differently than the ones you already
>>>>>>>>     > had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>     > the generic type parameters on the first static factory call. Can you
>>>>>>>>     > explain how your new interface proposal differs from the existing KIP?
>>>>>>>>
>>>>>>>> In the KIP, I didn't clarify what methods should be static. Now I
>>>>>>>> propose the following methods:
>>>>>>>>
>>>>>>>> non-static: withChain(Function), withName(String).
>>>>>>>>
>>>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>>>
>>>>>>>> The overloaded `with` version that provides both Function and name can
>>>>>>>> be used without causing type inference problem!!
>>>>>>>>
>>>>>>>> 2.
>>>>>>>> ----------------------------
>>>>>>>>
>>>>>>>>     > Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>     > be nice, but I'm not sure it will actually work.
>>>>>>>>
>>>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>>>
>>>>>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
>>>>>>>>             return s.mapValues(n -> (Integer) n + 1);
>>>>>>>> }
>>>>>>>>
>>>>>>>> BranchedKStream<String, Number> b =
>>>>>>>>         s.split().branch((k, v) -> isInteger(v),
>>>>>>>>                    //Won't compile!!
>>>>>>>>                    Branched.with(Me::func));
>>>>>>>>
>>>>>>>> The simple workaround here is to change `func`'s return type from
>>>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>>>
>>>>>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
>>>>>>>> `Branched`, so during code migration I will have to modify my functions'
>>>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>>>
>>>>>>>>     >  the map you're returning is Map<K,V>, and of course a K is not the
>>>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>>>
>>>>>>>> I think what you actually meant here is that KStream<? extends K, ?
>>>>>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
>>>>>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
>>>>>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
>>>>>>>>
>>>>>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
>>>>>>>> for now it's better to just admit that API is not absolutely perfect and
>>>>>>>> accept it as is, that is
>>>>>>>>
>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>>
>>>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>>>> Hello Ivan,
>>>>>>>>>
>>>>>>>>> Thanks for the refinement. Actually, I did not know that "as" would
>>>>>>>>> clash with a Kotlin operator. Maybe we should depart from convention
>>>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>>>
>>>>>>>>> The convention is that "as(String name)" is used for the static factory
>>>>>>>>> method, whereas "withName(String name)" is an instance method
>>>>>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
>>>>>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
>>>>>>>>> although it's somewhat dubious, since all the other configuration
>>>>>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
>>>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>>>
>>>>>>>>> Re. 3:
>>>>>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
>>>>>>>>> configuraion objects. The problem is that Java infers the type
>>>>>>>>> based on the _first_ method in the chain. While it does consider what
>>>>>>>>> the recipient of the method result wants, it only considers the _next_
>>>>>>>>> recipient.
>>>>>>>>>
>>>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>>>> Branched<String,String> variable, java infers the type correctly. But
>>>>>>>>> when the "next recipient" is a chained method call, like "withChain",
>>>>>>>>> then the chained method doesn't bound the type (by definition,
>>>>>>>>> withChain is defined on Branched<Object, Object>, so Java will take
>>>>>>>>> the broadest possible inferece and bind the type to
>>>>>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
>>>>>>>>>
>>>>>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
>>>>>>>>> that hard to recursively consider the entire context when inferring the
>>>>>>>>> generic type parameters, but this is what we have to work with.
>>>>>>>>>
>>>>>>>>> To be honest, though, I don't understand how your new proposed
>>>>>>>>> methods would work any differently than the ones you already
>>>>>>>>> had proposed in the KIP. It seems like you'd still have to provide
>>>>>>>>> the generic type parameters on the first static factory call. Can you
>>>>>>>>> explain how your new interface proposal differs from the existing KIP?
>>>>>>>>>
>>>>>>>>> Re. 4:
>>>>>>>>> Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
>>>>>>>>> try. In the past, we've run into soe truly strange interactions between the
>>>>>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
>>>>>>>>> combination with nested covariant types.
>>>>>>>>>
>>>>>>>>> Another issue is that the value type of the map you're returning is
>>>>>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
>>>>>>>>> doesn't seem compatible.
>>>>>>>>>
>>>>>>>>> Thanks again,
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>>>> Hi,
>>>>>>>>>>
>>>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>>>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>>>>>>>>>> simpler'.
>>>>>>>>>>
>>>>>>>>>> I made some quick API mocking in my IDE and tried to implement examples
>>>>>>>>>> from KIP.
>>>>>>>>>>
>>>>>>>>>> 1. Having to return something from lambda is not a very big deal.
>>>>>>>>>>
>>>>>>>>>> 2. For a moment I thouht that I won't be able to use method references
>>>>>>>>>> for already written stream consumers, but then I realized that I can
>>>>>>>>>> just change my methods from returning void to returning the input
>>>>>>>>>> parameter and use references to them. Not very convenient, but passable.
>>>>>>>>>>
>>>>>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>>>>>>>>>> function returns null, we don't insert it into the resulting map.
>>>>>>>>>>
>>>>>>>>>> Usually it's better to implement a non-perfect, but workable solution as
>>>>>>>>>> a first approximation. And later we can always add to `Branched`
>>>>>>>>>> anything we want.
>>>>>>>>>>
>>>>>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
>>>>>>>>>> should be built? First of all, it seems that `as` now is more preferred
>>>>>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
>>>>>>>>>> `as` operator).
>>>>>>>>>>
>>>>>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
>>>>>>>>>> infer types in the following construction:
>>>>>>>>>>
>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>         Branched.as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> so I have to write
>>>>>>>>>>
>>>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>>>         Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
>>>>>>>>>> that I came to (will you please validate it):
>>>>>>>>>>
>>>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>>>
>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>>>
>>>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>>>
>>>>>>>>>> //non-static!
>>>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>>>>>>>>>> KStream<K, V>> chain);
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 4. And one more. What do you think, do we need that flexibility:
>>>>>>>>>>
>>>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>>>>>>>>>
>>>>>>>>>> vs.
>>>>>>>>>>
>>>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>>>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>>>
>>>>>>>>>> ??
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>>
>>>>>>>>>> Ivan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>>>
>>>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>>>>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>>>>>>>>>
>>>>>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>>>>>>>>>
>>>>>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>>>>>>>>>
>>>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>>>> John
>>>>>>>>>>>
>>>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>
>>>>>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>>>
>>>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>>>
>>>>>>>>>>>>       (1) split a stream and return the substreams for futher processing
>>>>>>>>>>>>       (2) split a stream and modify the substreams with in-place method chaining
>>>>>>>>>>>>
>>>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>>>
>>>>>>>>>>>>        -> split a stream, modify the substreams, and return the _modified_
>>>>>>>>>>>> substreams for further processing
>>>>>>>>>>>>
>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>>>>>>>>>
>>>>>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>>>>>>>>>
>>>>>>>>>>>>       - using `withChain` I get the modified sub-stream
>>>>>>>>>>>>       - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>>>>>>>>>
>>>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>      From my understanding the original idea of `withJavaConsumer` was to
>>>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>>>
>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>        s.to();
>>>>>>>>>>>>        return null;
>>>>>>>>>>>> })
>>>>>>>>>>>>
>>>>>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
>>>>>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>>>
>>>>>>>>>>>> Following this train of through, and if we want to allow the "return
>>>>>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>>>>>>>>>> an entry to the Map.
>>>>>>>>>>>>
>>>>>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>>>
>>>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>>>        s.to();
>>>>>>>>>>>>        return s;
>>>>>>>>>>>> })
>>>>>>>>>>>>
>>>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>>>>>>>>>> while for the first proposal it adds new functionality (if `return null`
>>>>>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>>>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>>>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> I guess I can be convinced either way. However, if we follow your
>>>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>>>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
>>>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>>>
>>>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>> John,
>>>>>>>>>>>>>
>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>>>
>>>>>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>>>>>>>>>> method?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>>>>>>>>>
>>>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>>>>>>>>>
>>>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>>>
>>>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>>>
>>>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>>>
>>>>>>>>>>>>> Mathhias,
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>>>>>>>>>
>>>>>>>>>>>>> Done.
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>>>>>>>>>> method]
>>>>>>>>>>>>>
>>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>>>
>>>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>>>
>>>>>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>>>>>>>>>> implementation would be to not add the "branch" to the result map if
>>>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>>>
>>>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>>>>>>>>>
>>>>>>>>>>>>> In the current version of KIP it is assumed that the returned map
>>>>>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>>>
>>>>>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>>>>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>>>>>>>>>> it as it is?
>>>>>>>>>>>>>
>>>>>>>>>>>>> ----
>>>>>>>>>>>>>
>>>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> John
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>>>            .branch(....)
>>>>>>>>>>>>>>>>>>>            .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>>>            .onTopOf(someStream);
>>>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>          -> KBranchedStream
>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>          -> KBranchedStream
>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>             withChain(...);
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>            stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>>>                       .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>>>                       .defaultBranch(...) // creates node
>>>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>             operator(function, config_object?) OR
>>>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>           operator(function, function, string)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>            KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>            interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>>>>>>>>>             withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>>>             withChain(...);
>>>>>>>>>>>>>>>>>>>>>>>             withName(...);
>>>>>>>>>>>>>>>>>>>>>>>           }
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>>>             for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>             question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>>>             in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>             seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>>>             when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>>>             for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>>>             solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>>>             to.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>             <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>>>             LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>>>             matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>>>             advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>>>             proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>             But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>>>             workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>>>             method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>>>             tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>>>             going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>>>             handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>>>             a handler?
>>>>>>>>>>>>>>>>>>>>>>>>>>             > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>             > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>>             writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>>>             "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>>>             identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>>>             all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>>>             point?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > ---
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>             without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>>>             the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>>>             current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>>>             me address both comments here.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>>>             for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>>>             see example below.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>>>             Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>>>             value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>>>             have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>             > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>>>             > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>>>             >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>>>             recordType,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>             > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>>>             thinking
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> An open question is the case for which no
>>>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>             `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>>>             `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>>>             argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>>>             that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>>>             it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>>>             The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>             (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>>>             to each other.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>>>             branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>>>             <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>             <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>>>             downstream
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>             quickly become
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>>             <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>>             that sets a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>>>             current
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>>>             decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>             construct
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>             provides as much
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             call
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>>>             The value it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>>>             the same
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>             in the sense
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>>>             branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>>>             the "static"
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>>>             fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>>>             ignore a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>>>             SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>>>             SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>>>             like this:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>             readable than
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>             makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>>>             unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>>>             existing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>>>             that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>>>             access the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>             possible that it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>             left up to the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>>>             <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>             existing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>>>             proceed by
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>             important to be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>>>             this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>             double brace
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>>             <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>>>             To me,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>>>             positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>             additional code
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>>>             constructs
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>>>             looks like
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>             Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>>>             returns a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>>>             solves
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>>>             branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>>>             another
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>>>             structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>>>             historically
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>>>             voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>>>             <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>>>             "Split"
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>>>             is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>>>             `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>>>             definition of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>>>             there's no
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>>>             record
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>>>             should
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>>>             `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>>>             I though
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>>>             avoid
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>>>             with a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>             with all
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>>>             contained in
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>>>             `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>>>             branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>>>             that users
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>>>             access them.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>>>             to get
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>>>             loss of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>>>             without
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>>>             with such
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>>>             something.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>             proposal.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>             recommend the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>>>             BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>>>             `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>>>             accepted and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>>>             parameter.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>>>             different
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>>>             index)` method
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>>>             object?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>             whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>             current
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>             makes sense
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>             we'll
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>>>             think this
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>>>             starting a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>>>             assuming
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>             similar
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>>>             KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>>>             convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>>>             And then
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>>>             resume the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>>>             starting
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>>>             still
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             depend on
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>>>             access
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>>>             is, not
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>             excited
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>>>             same
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>>>             just
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>>>             we need
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>>>             in the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>>>             circumstances
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>>>             complex, so
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>>>             order to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>>>             everything
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>>>             initialize the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>>>             essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>>>             based on
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>>>             fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>>>>>>>             (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>>>             that Java
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>>>             returning one
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>             need
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>>>             shares
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>>>             demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>>>             wanted to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>>>             solution
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>             streams
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>>>             is, if we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>>>             that.  The
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>>>             table.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             branch API
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>>             returns
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>         ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>             instance
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>>>             miss the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>>>             methods
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>>             implemented the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>>>             want to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>>>             predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>             looks
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>>>             emit a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>             a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>>>             compile if
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>>>             method chain
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>>>             between
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>>>             compilation
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>>>             required.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>>>             just as
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>>>             which allows
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>>>             to be
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>>>             a clear
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>>>             stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>>>             operations
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>>>             always need
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>>>             so we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>>>             flow,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>             API, but I
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>>>             contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>>>             just call
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>>>             cases
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>             is very
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>>>             how we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>>>             terminated by
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>>>             incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>             different
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>>             KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>>>             while also
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>>>             KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>>>             ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>>>             String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>             this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>>>             this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>>>             example in
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>>>             we had
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>>>             feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Attachments:
>>>>>>>>>>>> * signature.asc
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>>
>>>>
>>>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Hi Ivan,

Thanks for the updates. I agree, it seems like all the concerns
that have been raised in the discussion so far have been
addressed. And it's been a while since anyone raised a new
concern. At this point, it seems like a good time to start
the VOTE thread.

Sometimes, the vote thread will trigger new people to look
into the KIP, and they may raise new concerns, but it's not a
problem. We'll just address those lingering concerns if there
are any, until you have all the votes you need.

Thanks again for the contribution!

-John

On Thu, May 28, 2020, at 04:22, Ivan Ponomarev wrote:
> Alright, I have updated the KIP with what we have discussed:
> 
> 1. Per Mathhias's suggestion, if a chain function returns null, the 
> respective result if omitted in the resulting Map.
> 
> 2. `with[Java]Consumer` method dropped.
> 
> 3. `Branched` class has only three static methods with all the possible 
> combinations of parameters.
> 
> 4. Chain function is defined 'fully covariant', let's see if we can 
> implement it this way :-))
> 
> + code example updates and minor edits.
> 
> 
> Since this is my first KIP, I'm not sure what should I do next. I feel 
> that we talked over all the details and the consensus is reached. Is it 
> OK to call for VOTE now or is it better to wait for more feedback?
> 
> Regards,
> 
> Ivan
> 
> 
> 
> 28.05.2020 3:26, John Roesler пишет:
> > Sounds good to me, Ivan!
> > -John
> > 
> > On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
> >> John,
> >>
> >> ---------------------
> >>
> >>   > I'm sorry things have been dragging out a little, but I have the
> >> sense we're very close to the end of this discussion, which is exciting.
> >>
> >> We are certainly moving forward!  And I'm not in a hurry at all. As I
> >> told you before in my projects I'm using Spring Kafka's
> >> KafkaStreamBrancher -- the implementation of the first, rejected version
> >> of this KIP. It's inferior to what we are discussing here, but it does
> >> its work. So it's worth to design this KIP really, really well. And by
> >> the way, from this discussion I'm learning the good API designing
> >> process. For me it has a value per se :-))
> >>
> >> ----------------------
> >>
> >>   >>   > I'd wonder whether we need the non-static builders (like withChain).
> >>   >>   > Do they provide any benefit over just using the right static
> >> factory?
> >>   >
> >>   > I don't have a strong feeling, either. It seems nice to offer a better
> >>   > type inference experience than what we get with Materialized, by
> >>   > offering the static method that takes both name and chain.
> >>   > Given that, there doesn't seem to be a good reason to also offer the
> >>   > non-static builder-style methods, so I guess I'd prefer to drop them.
> >>
> >> I agree again! From a recent discussion on Twitter
> >> (https://twitter.com/inponomarev/status/1265220044394545153) I found out
> >> an interesting fact about type inference rules in Java. Funny thing is
> >> that although we need to explicitly set types in a chain like this
> >>
> >> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
> >>
> >> (otherwise it won't compile), the composition of static method calls
> >> works just fine, all the types are being calculated correctly:
> >>
> >> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
> >>
> >> As I was told, for type inference there is difference between qualifiers
> >> and arguments, 'you go up if you are argument but stop if you are
> >> qualifier'. And it also seems that we should not bet on any future
> >> improvements in Java type inference here.
> >>
> >> So,
> >>
> >> 1) I think we that in this KIP we should provide three static methods
> >> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and
> >> drop any non-static ones.
> >>
> >> 2) If anything else will be ever needed, we can easily add anything.
> >> Maybe this can be done in a process of refinement of all the parameter
> >> classes.
> >>
> >> ----------------------------
> >>
> >>   > we may as well hope for the best, and propose the "fully
> >>   > covariant" definition for now.
> >>
> >> Understood and agreed! I will edit the KIP.
> >>
> >> ----------------------------
> >>
> >>   >> Good question, I already thought about it and rejected the idea....
> >>   > (I cut off your quote; the rest is in the chain below)
> >>   >  "Worst" case scenario: someone
> >>   > else will wish the return type is something different, and we'll go
> >>   > through a painless deprecation transition to change it later.
> >>
> >> Of course, we cant' predict all the ways people are going to use it.
> >>   From my own humble experience with Kafka Streams, the worst scenario is
> >> unlikely. Split is split, transform is transform, too much flexibility
> >> is often evil.
> >>
> >> ------------------------
> >>
> >> So it seems that we are close to the consensus. Two things to be altered
> >> in the current version of KIP:
> >>
> >> * list of Branched methods, drop non-static methods.
> >> * 'fully covariant' definition of `chained`.
> >>
> >> Any ideas / questions / objections?
> >>
> >> Regards,
> >>
> >> Ivan
> >>
> >>
> >>
> >>
> >> 27.05.2020 7:03, John Roesler пишет:
> >>> Thanks for the reply, Ivan,
> >>>
> >>> I'm sorry things have been dragging out a little, but I have the sense
> >>> we're very close to the end of this discussion, which is exciting.
> >>>
> >>>>    > I'd wonder whether we need the non-static builders (like withChain).
> >>>>    > Do they provide any benefit over just using the right static factory?
> >>>>
> >>>> I don't have a strong opinion here. I think it's just a matter of taste.
> >>>> But, if we like to use Occam's razor, then yes, non-static builders can
> >>>> be omitted, I agree!
> >>>
> >>> I don't have a strong feeling, either. It seems nice to offer a better
> >>> type inference experience than what we get with Materialized, by
> >>> offering the static method that takes both name and chain.
> >>>
> >>> Given that, there doesn't seem to be a good reason to also offer the
> >>> non-static builder-style methods, so I guess I'd prefer to drop them.
> >>>
> >>> I'll defer to Matthias, if he has a chance to consider whether it's more
> >>> valuable to stick with the existing pattern or break the pattern to offer
> >>> a better experience.
> >>>
> >>>>    > You might as well propose the “ideal” API in the KIP, which is the
> >>>>    > covariant typed function
> >>>>
> >>>> I didn't quite get it. Do I get you right that you propose the
> >>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
> >>>> then just see if there are any obstacles/pitfalls during implementation
> >>>> and unit testing?
> >>>
> >>> Yep! I know it's a bit sloppy, but my experience has been that we just
> >>> won't know what works until we really try it, and try it in several different
> >>> ways. Still, we may as well hope for the best, and propose the "fully
> >>> covariant" definition for now.
> >>>
> >>>>    > Is it necessary to restrict the result key and value types to be the
> >>>>    > same as the inputs?
> >>>>
> >>>> Good question, I already thought about it and rejected the idea....
> >>> (I cut off your quote; the rest is in the chain below)
> >>>
> >>> That's fair! It's your KIP, after all. I think I might have made a different
> >>> call here, but I think this choice is fine. "Worst" case scenario: someone
> >>> else will wish the return type is something different, and we'll go
> >>> through a painless deprecation transition to change it later. Thanks
> >>> to the clean design of your API, this doesn't seem to bad. And, of
> >>> course, you've actually been using similar functionality already, so it
> >>> seems we should trust your intuition.
> >>>
> >>> Thanks,
> >>> -John
> >>>
> >>>
> >>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
> >>>> Hi John,
> >>>>
> >>>>    > I'd wonder whether we need the non-static builders (like withChain).
> >>>> Do they provide any benefit over just using the right static factory?
> >>>>
> >>>> I don't have a strong opinion here. I think it's just a matter of taste.
> >>>> But, if we like to use Occam's razor, then yes, non-static builders can
> >>>> be omitted, I agree!
> >>>>
> >>>>    > You might as well propose the “ideal” API in the KIP, which is the
> >>>> covariant typed function
> >>>>
> >>>> I didn't quite get it. Do I get you right that you propose the
> >>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
> >>>> then just see if there are any obstacles/pitfalls during implementation
> >>>> and unit testing?
> >>>>
> >>>>    > Is it necessary to restrict the result key and value types to be the
> >>>> same as the inputs?
> >>>>
> >>>> Good question, I already thought about it and rejected the idea.
> >>>>
> >>>> Look, if we want to keep `withChain`'s function optional, then we must
> >>>> keep the result key and value types the same. Because for now, the
> >>>> default value for the 'chain function' is Function.identity().
> >>>>
> >>>> Of course, we can make the 'chain function' required. But I think this
> >>>> is not what `split` method is for. `split` is for splitting, not
> >>>> transforming, and `chainFunction` in most of the cases should be either
> >>>> a consumer or the identity function.
> >>>>
> >>>> Regards,
> >>>>
> >>>> Ivan
> >>>>
> >>>>
> >>>> 24.05.2020 17:15, John Roesler пишет:
> >>>>> Thanks for the reply, Ivan.
> >>>>>
> >>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
> >>>>>
> >>>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
> >>>>>
> >>>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
> >>>>>
> >>>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
> >>>>>
> >>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
> >>>>>
> >>>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
> >>>>>
> >>>>> Thanks,
> >>>>> John
> >>>>>
> >>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
> >>>>>> Hello John,
> >>>>>>
> >>>>>>
> >>>>>> 1.
> >>>>>> ---------------------------------------------
> >>>>>>
> >>>>>>     > Perhaps it would be better to stick with "as" for now
> >>>>>>     > and just file a Jira to switch them all at the same time [for
> >>>>>> compatibility with Kotlin]
> >>>>>>
> >>>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
> >>>>>> standard workaround
> >>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
> >>>>>> So actually this should be a very low priority issue, if an issue at
> >>>>>> all.
> >>>>>>
> >>>>>>     > I don't understand how your new proposed
> >>>>>>     > methods would work any differently than the ones you already
> >>>>>>     > had proposed in the KIP. It seems like you'd still have to provide
> >>>>>>     > the generic type parameters on the first static factory call. Can you
> >>>>>>     > explain how your new interface proposal differs from the existing KIP?
> >>>>>>
> >>>>>> In the KIP, I didn't clarify what methods should be static. Now I
> >>>>>> propose the following methods:
> >>>>>>
> >>>>>> non-static: withChain(Function), withName(String).
> >>>>>>
> >>>>>> static: as(String), with(Function), with(Function, String).
> >>>>>>
> >>>>>> The overloaded `with` version that provides both Function and name can
> >>>>>> be used without causing type inference problem!!
> >>>>>>
> >>>>>> 2.
> >>>>>> ----------------------------
> >>>>>>
> >>>>>>     > Regarding making the K,V types covariant also, yes, that would indeed
> >>>>>>     > be nice, but I'm not sure it will actually work.
> >>>>>>
> >>>>>> What I'm keeping in mind is the following example: imagine
> >>>>>>
> >>>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
> >>>>>>             return s.mapValues(n -> (Integer) n + 1);
> >>>>>> }
> >>>>>>
> >>>>>> BranchedKStream<String, Number> b =
> >>>>>>         s.split().branch((k, v) -> isInteger(v),
> >>>>>>                    //Won't compile!!
> >>>>>>                    Branched.with(Me::func));
> >>>>>>
> >>>>>> The simple workaround here is to change `func`'s return type from
> >>>>>> KStream<...Integer> to KStream<...Number>.
> >>>>>>
> >>>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
> >>>>>> `Branched`, so during code migration I will have to modify my functions'
> >>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
> >>>>>>
> >>>>>>     >  the map you're returning is Map<K,V>, and of course a K is not the
> >>>>>> same as "? extends K", so it doesn't seem compatible.
> >>>>>>
> >>>>>> I think what you actually meant here is that KStream<? extends K, ?
> >>>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
> >>>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
> >>>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
> >>>>>>
> >>>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
> >>>>>> for now it's better to just admit that API is not absolutely perfect and
> >>>>>> accept it as is, that is
> >>>>>>
> >>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
> >>>>>>
> >>>>>> Regards,
> >>>>>>
> >>>>>> Ivan
> >>>>>>
> >>>>>>
> >>>>>> 21.05.2020 17:59, John Roesler пишет:
> >>>>>>> Hello Ivan,
> >>>>>>>
> >>>>>>> Thanks for the refinement. Actually, I did not know that "as" would
> >>>>>>> clash with a Kotlin operator. Maybe we should depart from convention
> >>>>>>> and just avoid methods named "as" in the future.
> >>>>>>>
> >>>>>>> The convention is that "as(String name)" is used for the static factory
> >>>>>>> method, whereas "withName(String name)" is an instance method
> >>>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
> >>>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
> >>>>>>> although it's somewhat dubious, since all the other configuration
> >>>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
> >>>>>>> and just file a Jira to switch them all at the same time.
> >>>>>>>
> >>>>>>> Re. 3:
> >>>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
> >>>>>>> configuraion objects. The problem is that Java infers the type
> >>>>>>> based on the _first_ method in the chain. While it does consider what
> >>>>>>> the recipient of the method result wants, it only considers the _next_
> >>>>>>> recipient.
> >>>>>>>
> >>>>>>> Thus, if you call as("foo") and immediately assign it to a
> >>>>>>> Branched<String,String> variable, java infers the type correctly. But
> >>>>>>> when the "next recipient" is a chained method call, like "withChain",
> >>>>>>> then the chained method doesn't bound the type (by definition,
> >>>>>>> withChain is defined on Branched<Object, Object>, so Java will take
> >>>>>>> the broadest possible inferece and bind the type to
> >>>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
> >>>>>>>
> >>>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
> >>>>>>> that hard to recursively consider the entire context when inferring the
> >>>>>>> generic type parameters, but this is what we have to work with.
> >>>>>>>
> >>>>>>> To be honest, though, I don't understand how your new proposed
> >>>>>>> methods would work any differently than the ones you already
> >>>>>>> had proposed in the KIP. It seems like you'd still have to provide
> >>>>>>> the generic type parameters on the first static factory call. Can you
> >>>>>>> explain how your new interface proposal differs from the existing KIP?
> >>>>>>>
> >>>>>>> Re. 4:
> >>>>>>> Regarding making the K,V types covariant also, yes, that would indeed
> >>>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
> >>>>>>> try. In the past, we've run into soe truly strange interactions between the
> >>>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
> >>>>>>> combination with nested covariant types.
> >>>>>>>
> >>>>>>> Another issue is that the value type of the map you're returning is
> >>>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
> >>>>>>> doesn't seem compatible.
> >>>>>>>
> >>>>>>> Thanks again,
> >>>>>>> -John
> >>>>>>>
> >>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
> >>>>>>>> Hi,
> >>>>>>>>
> >>>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
> >>>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
> >>>>>>>> simpler'.
> >>>>>>>>
> >>>>>>>> I made some quick API mocking in my IDE and tried to implement examples
> >>>>>>>> from KIP.
> >>>>>>>>
> >>>>>>>> 1. Having to return something from lambda is not a very big deal.
> >>>>>>>>
> >>>>>>>> 2. For a moment I thouht that I won't be able to use method references
> >>>>>>>> for already written stream consumers, but then I realized that I can
> >>>>>>>> just change my methods from returning void to returning the input
> >>>>>>>> parameter and use references to them. Not very convenient, but passable.
> >>>>>>>>
> >>>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
> >>>>>>>> function returns null, we don't insert it into the resulting map.
> >>>>>>>>
> >>>>>>>> Usually it's better to implement a non-perfect, but workable solution as
> >>>>>>>> a first approximation. And later we can always add to `Branched`
> >>>>>>>> anything we want.
> >>>>>>>>
> >>>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
> >>>>>>>> should be built? First of all, it seems that `as` now is more preferred
> >>>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
> >>>>>>>> `as` operator).
> >>>>>>>>
> >>>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
> >>>>>>>> infer types in the following construction:
> >>>>>>>>
> >>>>>>>> .branch((key, value) -> value == null,
> >>>>>>>>         Branched.as("foo").withChain(s -> s.mapValues(...)))
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> so I have to write
> >>>>>>>>
> >>>>>>>> .branch((key, value) -> value == null,
> >>>>>>>>         Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
> >>>>>>>> that I came to (will you please validate it):
> >>>>>>>>
> >>>>>>>> static <K, V> Branched<K, V> as(String name);
> >>>>>>>>
> >>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>>>>>> extends KStream<K, V>> chain);
> >>>>>>>>
> >>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>>>>>> extends KStream<K, V>> chain, String name);
> >>>>>>>>
> >>>>>>>> //non-static!
> >>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
> >>>>>>>> KStream<K, V>> chain);
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 4. And one more. What do you think, do we need that flexibility:
> >>>>>>>>
> >>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
> >>>>>>>>
> >>>>>>>> vs.
> >>>>>>>>
> >>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
> >>>>>>>> extends K, ? extends V>> chain
> >>>>>>>>
> >>>>>>>> ??
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>>
> >>>>>>>> Ivan
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 21.05.2020 6:54, John Roesler пишет:
> >>>>>>>>> Thanks for this thought, Matthias,
> >>>>>>>>>
> >>>>>>>>> Your idea has a few aspects I find attractive:
> >>>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
> >>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
> >>>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
> >>>>>>>>>
> >>>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
> >>>>>>>>>
> >>>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
> >>>>>>>>>
> >>>>>>>>> Thanks again for sharing the idea,
> >>>>>>>>> John
> >>>>>>>>>
> >>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> >>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>
> >>>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
> >>>>>>>>>> its relationship to the returned `Map`.
> >>>>>>>>>>
> >>>>>>>>>> Originally, we discussed two main patterns:
> >>>>>>>>>>
> >>>>>>>>>>       (1) split a stream and return the substreams for futher processing
> >>>>>>>>>>       (2) split a stream and modify the substreams with in-place method chaining
> >>>>>>>>>>
> >>>>>>>>>> To combine both patterns we wanted to allow for
> >>>>>>>>>>
> >>>>>>>>>>        -> split a stream, modify the substreams, and return the _modified_
> >>>>>>>>>> substreams for further processing
> >>>>>>>>>>
> >>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> >>>>>>>>>>
> >>>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
> >>>>>>>>>>
> >>>>>>>>>>       - using `withChain` I get the modified sub-stream
> >>>>>>>>>>       - using `withJavaConsumer` I get the unmodifed sub-stream
> >>>>>>>>>>
> >>>>>>>>>> This seems to be quite subtle to me.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>      From my understanding the original idea of `withJavaConsumer` was to
> >>>>>>>>>> model a terminal operation, ie, it should be similar to:
> >>>>>>>>>>
> >>>>>>>>>> Branched.withChain(s -> {
> >>>>>>>>>>        s.to();
> >>>>>>>>>>        return null;
> >>>>>>>>>> })
> >>>>>>>>>>
> >>>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
> >>>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
> >>>>>>>>>> -> null` entry in the returned Map.
> >>>>>>>>>>
> >>>>>>>>>> Following this train of through, and if we want to allow the "return
> >>>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
> >>>>>>>>>> an entry to the Map.
> >>>>>>>>>>
> >>>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
> >>>>>>>>>> be achieved with `withChain`:
> >>>>>>>>>>
> >>>>>>>>>> Branched.withChain(s -> {
> >>>>>>>>>>        s.to();
> >>>>>>>>>>        return s;
> >>>>>>>>>> })
> >>>>>>>>>>
> >>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> >>>>>>>>>> while for the first proposal it adds new functionality (if `return null`
> >>>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
> >>>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
> >>>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> I guess I can be convinced either way. However, if we follow your
> >>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
> >>>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
> >>>>>>>>>> preferable as it's simpler to learn.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>> Hello, John, hello Matthias!
> >>>>>>>>>>>
> >>>>>>>>>>> Thank you very much for your detailed feedback!
> >>>>>>>>>>>
> >>>>>>>>>>> -----------------------------------------
> >>>>>>>>>>>
> >>>>>>>>>>> John,
> >>>>>>>>>>>
> >>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
> >>>>>>>>>>>
> >>>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
> >>>>>>>>>>> all the emails on the web.
> >>>>>>>>>>>
> >>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> >>>>>>>>>>> method?
> >>>>>>>>>>>
> >>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
> >>>>>>>>>>>
> >>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
> >>>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
> >>>>>>>>>>>
> >>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
> >>>>>>>>>>>
> >>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> >>>>>>>>>>>
> >>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
> >>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
> >>>>>>>>>>>
> >>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
> >>>>>>>>>>> chain and the result map OR using just the sink
> >>>>>>>>>>>
> >>>>>>>>>>> This is discussed below.
> >>>>>>>>>>>
> >>>>>>>>>>> ----------------------------------------------
> >>>>>>>>>>>
> >>>>>>>>>>> Mathhias,
> >>>>>>>>>>>
> >>>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> >>>>>>>>>>>
> >>>>>>>>>>> Done.
> >>>>>>>>>>>
> >>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> >>>>>>>>>>> method]
> >>>>>>>>>>>
> >>>>>>>>>>> Fixed.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>> 3. Overview of newly added methods/interfaces
> >>>>>>>>>>>
> >>>>>>>>>>> Done in `Proposed Changes` section.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
> >>>>>>>>>>> `withChain()` and `withConsumer()` is the
> >>>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
> >>>>>>>>>>> be given to the `Consumer`.
> >>>>>>>>>>>
> >>>>>>>>>>> Yes, I agree!
> >>>>>>>>>>>
> >>>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
> >>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> >>>>>>>>>>> implementation would be to not add the "branch" to the result map if
> >>>>>>>>>>> `withConsumer` is used?
> >>>>>>>>>>>
> >>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
> >>>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
> >>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> >>>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
> >>>>>>>>>>>
> >>>>>>>>>>> In the current version of KIP it is assumed that the returned map
> >>>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
> >>>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
> >>>>>>>>>>> map is the user's responsibility.
> >>>>>>>>>>>
> >>>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
> >>>>>>>>>>> general rule, like 'swallowing' some streams by provided
> >>>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
> >>>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
> >>>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
> >>>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
> >>>>>>>>>>> it as it is?
> >>>>>>>>>>>
> >>>>>>>>>>> ----
> >>>>>>>>>>>
> >>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
> >>>>>>>>>>>
> >>>>>>>>>>> Looking forward for your feedback again!
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>>
> >>>>>>>>>>> Ivan.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
> >>>>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>>>
> >>>>>>>>>>>> I also have some minor comment:
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> >>>>>>>>>>>>
> >>>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> >>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> >>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
> >>>>>>>>>>>> so we just keep them.)
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> (2) Quote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
> >>>>>>>>>>>>> parameterless alternatives.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
> >>>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> >>>>>>>>>>>> as `branch()` would not be "parameterless".
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
> >>>>>>>>>>>> newly added and deprecated methods/classes (cf.
> >>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> >>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
> >>>>>>>>>>>> 4th comment:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
> >>>>>>>>>>>>> chain and the result map OR using just the sink.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
> >>>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
> >>>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
> >>>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
> >>>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
> >>>>>>>>>>>> it in the JavaDocs, this might be fine?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> (5) Reply to John's comments:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> >>>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
> >>>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
> >>>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
> >>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
> >>>>>>>>>>>>
> >>>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
> >>>>>>>>>>>> node", ie., writing the KStream to a topic.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
> >>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> >>>>>>>>>>>>> but I had a few last comments.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>> John
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>> Hello everyone,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I believe that now it follows design principles and takes into account
> >>>>>>>>>>>>>> all the arguments discussed here.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
> >>>>>>>>>>>>>>> rewritten the KIP, see
> >>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> This version includes all the previous discussion results and follows
> >>>>>>>>>>>>>>> the design principles, with one exception.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The exception is
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
> >>>>>>>>>>>>>>> here it is justified.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
> >>>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
> >>>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
> >>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> >>>>>>>>>>>>>>> as it
> >>>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
> >>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
> >>>>>>>>>>>>>>>> build on
> >>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> >>>>>>>>>>>>>>>> want to follow:
> >>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -- might be worth to checkout.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>> Hi everyone!
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
> >>>>>>>>>>>>>>>>> June
> >>>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
> >>>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
> >>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
> >>>>>>>>>>>>>>>>> propose other things before this one is finalized.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
> >>>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
> >>>>>>>>>>>>>>>>> branches, I worked around it this way:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>>>>>>>>>>>            .branch(....)
> >>>>>>>>>>>>>>>>>            .defaultBranch(result::set)
> >>>>>>>>>>>>>>>>>            .onTopOf(someStream);
> >>>>>>>>>>>>>>>>> result.get()...
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
> >>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>>>          -> KBranchedStream
> >>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>          -> KBranchedStream
> >>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
> >>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
> >>>>>>>>>>>>>>>>> Or,
> >>>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
> >>>>>>>>>>>>>>>>> finally,
> >>>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
> >>>>>>>>>>>>>>>>> original scope.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> >>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
> >>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
> >>>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>> at any point.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> >>>>>>>>>>>>>>>>>> do so.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
> >>>>>>>>>>>>>>>>>>> to mix
> >>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> >>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> >>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> >>>>>>>>>>>>>>>>>>>> patterns
> >>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
> >>>>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>> totally sense.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> >>>>>>>>>>>>>>>>>>>> `Map` only
> >>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
> >>>>>>>>>>>>>>>>>>>> all of
> >>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> >>>>>>>>>>>>>>>>>>>> `Named` is
> >>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> >>>>>>>>>>>>>>>>>>>> counter
> >>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
> >>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
> >>>>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>>>>>>>>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
> >>>>>>>>>>>>>>>>>>>> misses to
> >>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> >>>>>>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> >>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>>>>>>>>>>>>>>>> specifying a
> >>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
> >>>>>>>>>>>>>>>>>>>> hence
> >>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>             withChain(...);
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> >>>>>>>>>>>>>>>>>>>> does not
> >>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
> >>>>>>>>>>>>>>>>>>>> make sense
> >>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> >>>>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
> >>>>>>>>>>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
> >>>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> >>>>>>>>>>>>>>>>>>>> object
> >>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
> >>>>>>>>>>>>>>>>>>>> others,
> >>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
> >>>>>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
> >>>>>>>>>>>>>>>>>>>> methods that
> >>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
> >>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> >>>>>>>>>>>>>>>>>>>> two main
> >>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> >>>>>>>>>>>>>>>>>>>>> satisfy
> >>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
> >>>>>>>>>>>>>>>>>>>>> solves
> >>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> >>>>>>>>>>>>>>>>>>>>> to add
> >>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
> >>>>>>>>>>>>>>>>>>>>> Otherwise,
> >>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> >>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
> >>>>>>>>>>>>>>>>>>>>> names for
> >>>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
> >>>>>>>>>>>>>>>>>>>>> logic, it
> >>>>>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
> >>>>>>>>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> >>>>>>>>>>>>>>>>>>>>> processing onto
> >>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> >>>>>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
> >>>>>>>>>>>>>>>>>>>>> proposal, we
> >>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
> >>>>>>>>>>>>>>>>>>>>> name,
> >>>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>            stream.split(Named.withName("mysplit")) //creates node
> >>>>>>>>>>>>>>>>>>>>> "mysplit"
> >>>>>>>>>>>>>>>>>>>>>                       .branch(..., ..., "abranch") // creates node
> >>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>>>>>>>>>>>                       .defaultBranch(...) // creates node
> >>>>>>>>>>>>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> >>>>>>>>>>>>>>>>>>>>> debate
> >>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
> >>>>>>>>>>>>>>>>>>>>> general,
> >>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>             operator(function, config_object?) OR
> >>>>>>>>>>>>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
> >>>>>>>>>>>>>>>>>>>>> variant.
> >>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>           operator(function, function, string)
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> where the string is the name.
> >>>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
> >>>>>>>>>>>>>>>>>>>>> specified
> >>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
> >>>>>>>>>>>>>>>>>>>>> arguments
> >>>>>>>>>>>>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>            KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>            interface BranchConfig extends NamedOperation {
> >>>>>>>>>>>>>>>>>>>>>             withPredicate(...);
> >>>>>>>>>>>>>>>>>>>>>             withChain(...);
> >>>>>>>>>>>>>>>>>>>>>             withName(...);
> >>>>>>>>>>>>>>>>>>>>>           }
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> >>>>>>>>>>>>>>>>>>>>> more like
> >>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
> >>>>>>>>>>>>>>>>>>>>> makes us
> >>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> >>>>>>>>>>>>>>>>>>>>> purely
> >>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
> >>>>>>>>>>>>>>>>>>>>> overloads
> >>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> WDYT?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> >>>>>>>>>>>>>>>>>>>>>> view.
> >>>>>>>>>>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> >>>>>>>>>>>>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> >>>>>>>>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> >>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>>>>>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>>>>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> >>>>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
> >>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> >>>>>>>>>>>>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>>>>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>>>>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>>>>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
> >>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> >>>>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> >>>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>>>>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>>>>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
> >>>>>>>>>>>>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> >>>>>>>>>>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>>>>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> >>>>>>>>>>>>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>>>>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> >>>>>>>>>>>>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>>>>>>>>>>>> language
> >>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> >>>>>>>>>>>>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
> >>>>>>>>>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>>>>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> >>>>>>>>>>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>>>>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>>>>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> >>>>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> >>>>>>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> >>>>>>>>>>>>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>>>>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             Ivan, I’ll definitely forfeit my point on the
> >>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             branch(predicate, consumer) solution, I don’t see
> >>>>>>>>>>>>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>>>>>>>>>>>             for the dynamic case.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             IMO the one trade off to consider at this point is the
> >>>>>>>>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>             question. I don’t know if I totally agree that “we
> >>>>>>>>>>>>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>>>>>>>>>             in the same scope” since merging the branches back
> >>>>>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>>             seems like a perfectly plausible use case that can
> >>>>>>>>>>>>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>>>>>>>>>>>             when the branched streams are in the same scope.
> >>>>>>>>>>>>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>>>>>>>             for the reasons Ivan listed, I think it is overall the
> >>>>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>>>             solution - working around the scope thing is easy
> >>>>>>>>>>>>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>>>>>>>>>>>             to.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>             <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > Hello everyone, thank you all for joining the
> >>>>>>>>>>>>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > Well, I don't think the idea of named branches,
> >>>>>>>>>>>>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>>>>>>>>>>>             LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>>>>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>>>>>>>>>             matters) or `branch` method  taking name and Consumer
> >>>>>>>>>>>>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>>>>>>>>>>>             advantages than drawbacks.
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > In my opinion, the only real positive outcome from
> >>>>>>>>>>>>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>>>>>>>>>>>             proposal is that all the returned branches are in
> >>>>>>>>>>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>             But 1) we rarely need them in the same scope 2)
> >>>>>>>>>>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>>>>>>>>>             workaround for the scope problem, described in the
> >>>>>>>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > 'Inlining the complex logic' is not a problem,
> >>>>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>>>>>>>>>>>             method references instead of lambdas. In real world
> >>>>>>>>>>>>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>>>>>>>>>>>             tend to split the complex logic to methods anyway,
> >>>>>>>>>>>>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>>>>>>>>>>>             going to be clean.
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > The drawbacks are strong. The cohesion between
> >>>>>>>>>>>>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             handlers is lost. We have to define predicates in one
> >>>>>>>>>>>>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>>>>>>>>>>>             handlers in another. This opens the door for bugs:
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > - what if we forget to define a handler for a
> >>>>>>>>>>>>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>>>>>>>>>>>             a handler?
> >>>>>>>>>>>>>>>>>>>>>>>>             > - what if we misspell a name?
> >>>>>>>>>>>>>>>>>>>>>>>>             > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > What Michael propose would have been totally OK
> >>>>>>>>>>>>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>             writing the API in Lua, Ruby or Python. In those
> >>>>>>>>>>>>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>>>>>>>>>>>             "dynamic naming" approach would have looked most
> >>>>>>>>>>>>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             beautiful. But in Java we expect all the problems
> >>>>>>>>>>>>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>>>>>>>>>>>             identifiers to be eliminated in compile time.
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > And if we do, what advantage are we supposed to get
> >>>>>>>>>>>>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>>>>>>>>>>>             all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>>>>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>>>>>>>>>>>             point?
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > ---
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > Earlier in this discussion John Roesler also
> >>>>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>>>             without "start branching" operator, and later Paul
> >>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>>>>>>>>>>>             the case when we have to add a dynamic number of
> >>>>>>>>>>>>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>>>>>>>>>>>             current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>>>>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>>>>>>>>>>>             me address both comments here.
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > 1) "Start branching" operator (I think that
> >>>>>>>>>>>>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>>>>>>>>>>>             for it indeed) is critical when we need to do a
> >>>>>>>>>>>>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>>>>>>>>>>>             see example below.
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > 2) No, dynamic branching in current KIP is not
> >>>>>>>>>>>>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>>>>>             Imagine a real-world scenario when you need one
> >>>>>>>>>>>>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>>>>>>>>>>>             value (say, RecordType). You can have something
> >>>>>>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > /*John:if we had to start with stream.branch(...)
> >>>>>>>>>>>>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>>>>>>>>>>>             have been much messier.*/
> >>>>>>>>>>>>>>>>>>>>>>>>             > KBranchedStream branched = stream.split();
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > /*Not clumsy at all :-)*/
> >>>>>>>>>>>>>>>>>>>>>>>>             > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>>>>>>>>>>>>>>>             >             branched = branched.branch((k, v) ->
> >>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>>>>>>>>>>>             recordType,
> >>>>>>>>>>>>>>>>>>>>>>>>             >                     recordType::processRecords);
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>             > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >> I also agree with Michael's observation about
> >>>>>>>>>>>>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>>>>>>>>>>>             >> current `branch()` implementation.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>>>>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>>>>>>>>>>>             thinking
> >>>>>>>>>>>>>>>>>>>>>>>>             >> was more aligned with Paul's proposal to just
> >>>>>>>>>>>>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>>>>>>>>>>>             >> `branch()` statement and return a
> >>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >> It makes the code easier to read, and also make the
> >>>>>>>>>>>>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>>>>>>>>>>>             >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>>>>>>>>>>>             >> An open question is the case for which no
> >>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>             >> specified. Atm, `split()` and `branch()` would
> >>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>             `BranchedKStream`
> >>>>>>>>>>>>>>>>>>>>>>>>             >> and the call to `defaultBranch()` that returns the
> >>>>>>>>>>>>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>>>>>>>>>>>             >> (what is not the case atm). Or is this actually
> >>>>>>>>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>>>>>> real
> >>>>>>>>>>>>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>>>>>>>>>>>             >> because users can just ignore the branch
> >>>>>>>>>>>>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>>>>>>>>>>>             `defaultBranch()`
> >>>>>>>>>>>>>>>>>>>>>>>>             >> in the result `Map` ?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >> About "inlining": So far, it seems to be a
> >>>>>>>>>>>>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>>>>>>>>>>>             >> preference. I can see arguments for both, but no
> >>>>>>>>>>>>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>>>>>>>>>>>             argument" yet
> >>>>>>>>>>>>>>>>>>>>>>>>             >> that clearly make the case for one or the other.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>             >>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>> Perhaps inlining is the wrong terminology. It
> >>>>>>>>>>>>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>>>             that a lambda with the full downstream topology be
> >>>>>>>>>>>>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>>>>>>>>>>>             it can be a method reference as with Ivan’s original
> >>>>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>>>             The advantage of putting the predicate and its
> >>>>>>>>>>>>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>>             (Consumer) together in branch() is that they are
> >>>>>>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>>>>>>>>>>>             to each other.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>> Ultimately the downstream code has to live
> >>>>>>>>>>>>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>>>>>>>>>>>             branch trees will be hard to read regardless.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>>>>>>>>>>>             <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>             <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>> I'm less enthusiastic about inlining the
> >>>>>>>>>>>>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>>>>>>>>>>>             downstream
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>> functionality. Programs that have deep branch
> >>>>>>>>>>>>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>             quickly become
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>> harder to read as a single unit.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>>             <pgwhalen@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Also +1 on the issues/goals as Michael
> >>>>>>>>>>>>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>>>>             that sets a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> great framework for the discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Regarding the SortedMap solution, my
> >>>>>>>>>>>>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>>>>>>>>>             current
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>>>>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>>>>>>>>>>>             decisions) is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> roughly this:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Obviously some ordering is necessary, since
> >>>>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>             construct
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> doesn't work without it, but this solution seems
> >>>>>>>>>>>>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>>>             provides as much
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> associativity as the SortedMap solution,
> >>>>>>>>>>>>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>             call
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> directly associates the "conditional" with
> >>>>>>>>>>>>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>>>>>>>>>>>             The value it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> provides over the KIP solution is the
> >>>>>>>>>>>>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>>>>>>>>>             the same
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> The KIP solution is less "dynamic" than the
> >>>>>>>>>>>>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>             in the sense
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> that it is slightly clumsier to add a dynamic
> >>>>>>>>>>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>>>>>>>             branches, but it is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>>>>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>>>>>>>>>>>             the "static"
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> case anyway, and should make it simple and
> >>>>>>>>>>>>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>>>>>>>>>>>             fluently declare and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>>>>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>>>>>>>>>>>             ignore a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>>>>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>>>>>>>>>>>             SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> solution on top of it.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> I could also see a middle ground where
> >>>>>>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>>>>>>>>>>>             SortedMap being
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> taken in, branch() takes a name and not a
> >>>>>>>>>>>>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>>>>>>>>>>>             like this:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Pros for that solution:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - no double brace initialization, hopefully
> >>>>>>>>>>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>             readable than
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Cons
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - downstream branch logic cannot be specified
> >>>>>>>>>>>>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>             makes it harder
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to read top to bottom (like existing API and
> >>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>>>>>>>>>>>             unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>>>>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>>>>>>>>>>>             existing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> (KBranchedStreams could even work *both* ways
> >>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>>>>>>>>>>>             that's overdoing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it).
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Overall I'm curious how important it is to be
> >>>>>>>>>>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>>>>>>>>>             access the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branched KStream in the same scope as the
> >>>>>>>>>>>>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>>>             possible that it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> doesn't need to be handled directly by the
> >>>>>>>>>>>>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>             left up to the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> >>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>>>>>>>>>>>             <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I'd like to +1 what Michael said about the
> >>>>>>>>>>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>             existing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branch
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> method, I agree with what he's outlined and
> >>>>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>>>>>>>>>>>             proceed by
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> trying to alleviate these problems.
> >>>>>>>>>>>>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>             important to be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> able to cleanly access the individual
> >>>>>>>>>>>>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> name->stream), which I thought was the original
> >>>>>>>>>>>>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>>>>>>>>>>>             this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> That said, I don't think we should so easily
> >>>>>>>>>>>>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>>>             double brace
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> anti-pattern or force ours users into it if
> >>>>>>>>>>>>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> avoid...just
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> my two cents.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Sophie
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> >>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>>             <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> I’d like to propose a different way of
> >>>>>>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>>>>>>>>>>>             To me,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> there
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> are three problems with the existing branch
> >>>>>>>>>>>>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> warnings.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 2. The way in which you use the stream
> >>>>>>>>>>>>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>>>>>>>>>>>             positionally coupled
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 3. It is brittle to extend existing branch
> >>>>>>>>>>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>             additional code
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> paths.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Using associative constructs instead of
> >>>>>>>>>>>>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>>>>>>>>>>>             constructs
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> be a stronger approach. Consider a
> >>>>>>>>>>>>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>>>             looks like
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>>>>>>>>>>>             Predicate<?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> super K,? super V>>);
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Branches are given names in a map, and as a
> >>>>>>>>>>>>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>>>>>>>>>>>             returns a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> mapping of names to streams. The ordering
> >>>>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> maintained
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> because it’s a sorted map. Insert order
> >>>>>>>>>>>>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>>>>>>>>>>>> the order
> >>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> evaluation.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> This solves problem 1 because there are no
> >>>>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>>>>>>>>>>>             solves
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> problem
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 2 because you no longer lean on ordering to
> >>>>>>>>>>>>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>>>>>>>>>>>             branch you’re
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> interested in. It solves problem 3 because
> >>>>>>>>>>>>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>>>>>>>>>             another
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> conditional by simply attaching another
> >>>>>>>>>>>>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>>>>>>>>>>>             structure, rather
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> messing with the existing indices.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>>>>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>>>>>>>>>>>             historically
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> awkward in Java. I know it’s an
> >>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>>>>>>>>>>>             voluminously, but
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> double brace initialization would clean up the
> >>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>>>>>>>>>>>             <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thanks for the update.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> FWIW, I agree with Matthias that the current
> >>>>>>>>>>>>>>>>>>>>>>>> "start
> >>>>>>>>>>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> operator
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> confusing when named the same way as the
> >>>>>>>>>>>>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>>>>>>>>>>>             "Split"
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> like a good name. Alternatively, we can do
> >>>>>>>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> operator at all, and just do:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Tentatively, I think that this branching
> >>>>>>>>>>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> terminal.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> way, we don't create ambiguity about how
> >>>>>>>>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>>>>>>>>>>>             is, `branch`
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>>>>>>>>>>>             `void`, to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> enforce that it comes last, and that there
> >>>>>>>>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>>>>>             definition of
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>>>>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>>>>>>>>>>>             there's no
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> default,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>>>>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>>>>>>>>>>>             record
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> falls
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> though with no default.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Thanks for updating the KIP and your
> >>>>>>>>>>>>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> this is to make the name similar to
> >>>>>>>>>>>>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> The intend was to avoid name duplication.
> >>>>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>>>>>>>>>>>             should
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> _not_
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> be an array.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> The current proposal is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> IMHO, this reads a little odd, because
> >>>>>>>>>>>>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>>>>>>>>>>>             `branch()` does
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> take any parameters and has different
> >>>>>>>>>>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>>>>>>>> than the
> >>>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>>>>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> hidden
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that the first call is `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>>>>>>>> the others
> >>>>>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Because I suggested to rename
> >>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>>>>>>>>>>>             I though
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> might be better to also rename
> >>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> naming
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> overlap that seems to be confusing. The
> >>>>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> me:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Maybe there is a better alternative to
> >>>>>>>>>>>>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>>>>>>>>>>>             avoid
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> naming overlap.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> cannot
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> a method with such name :-)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Bummer. Didn't consider this. Maybe we
> >>>>>>>>>>>>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>>>>>>>>>>>             with a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> short
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> name?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Can you add the interface
> >>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>>             with all
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> methods? It will be part of public API and
> >>>>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>>>             contained in
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>>>>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>>>>>>>>>>>>>>>             `KBranchedStream#get(int
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> index)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>>>>>>>>>>>>>>>             branched-KStreams. Would
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> nice to get your feedback about it. It
> >>>>>>>>>>>>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>>>>>>>             that users
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> would need to write custom utility code
> >>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>>>>>>>>>>>             access them.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>>>>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>>>>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>>>>>>>>>>>             to get
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> I have updated the KIP-418 according to
> >>>>>>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> I can see your point: this is to make
> >>>>>>>>>>>>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> that also returns an array, right? But
> >>>>>>>>>>>>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>>>>>>>>>>>             loss of
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> compatibility? We can have overloaded
> >>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>>>>>>>>>>>             without
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> affecting
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> the existing code. Maybe the old
> >>>>>>>>>>>>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> should
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> deprecated, but this is a subject for
> >>>>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Totally agree with 'addBranch->branch'
> >>>>>>>>>>>>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> however, a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> reserved word, so unfortunately we
> >>>>>>>>>>>>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>>>             with such
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> name
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> :-)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> defaultBranch() does take an
> >>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> is not required?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Absolutely! I think that was just
> >>>>>>>>>>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>>>>>>>>>>>             something.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> please revise the new version of the KIP
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Thanks for driving the discussion of
> >>>>>>>>>>>>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> everybody
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> agrees that the current branch() method
> >>>>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> optimal.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> I had a quick look into the PR and I
> >>>>>>>>>>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>>>>>>>             proposal.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> There
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> are some minor things we need to
> >>>>>>>>>>>>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>             recommend the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> following renaming:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>>>             BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> It's just a suggestion to get slightly
> >>>>>>>>>>>>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> In the current PR, defaultBranch() does
> >>>>>>>>>>>>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>>>>>>>>>>>             `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>>>>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>>>>>>>>>>>             accepted and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Ie, we should add overloads that
> >>>>>>>>>>>>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>>>>>>>             parameter.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> For the issue that the created
> >>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>>>>>>>>>>>             different
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> scopes:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> could we extend `KBranchedStream` with a
> >>>>>>>>>>>>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>>>>>>>>>>>             index)` method
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> returns the corresponding "branched"
> >>>>>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>>>>>>>>>>>             object?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Maybe,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> second argument of `addBranch()` should
> >>>>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> a `Function<KStream,KStream>` and
> >>>>>>>>>>>>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>>>             whatever
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Finally, I would also suggest to update
> >>>>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>             current
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>>>             makes sense
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> revise the KIP and continue the
> >>>>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>>>>>>>>>>>             we'll
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> need
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>>>>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> whether
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> could be adopted.  It would be great
> >>>>>>>>>>>>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>>>>>>>>>>>             think this
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> idea overall.  I'm not sure if that
> >>>>>>>>>>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>>>>>>>>>>>             starting a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> vote,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> or if
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> there is generally some indication of
> >>>>>>>>>>>>>>>>>>>>>>>> interest
> >>>>>>>>>>>>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> That being said, I'll continue the
> >>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>>>>>>>>>>>             assuming
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> forward the solution of "stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> KBranchedStream",
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> favor
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> accomplish
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>>>>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>>>>>>>>>>>             similar
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> just need to be sure we're not making
> >>>>>>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> impossible/difficult
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>>>>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> just a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> little sloppy overall in terms of
> >>>>>>>>>>>>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> particular,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> passing in the "predicates" and
> >>>>>>>>>>>>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> modified
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>>>>>>>>>>>>>>>             KStreamLazyBranch is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> I read your code carefully and now I
> >>>>>>>>>>>>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>>>>>>>>>>>             convinced: your
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> looks better and should work. We just
> >>>>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> crucial
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> that KStream consumers are invoked as
> >>>>>>>>>>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>>>>>>>             And then
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> What shall we do now? I should
> >>>>>>>>>>>>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>>>>>>>>>>>             resume the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Why are you telling that your PR
> >>>>>>>>>>>>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> point
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> we go in this direction'? To me it
> >>>>>>>>>>>>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>>>>>>>             starting
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> as a novice in this project I might
> >>>>>>>>>>>>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> details.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> >>>>>>>>>>>>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>>>>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> invoked
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> they’re
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> added, not during
> >>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>>>>>>>>>>>             still
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> ought
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> able to call couponIssuer.coupons()
> >>>>>>>>>>>>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             depend on
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> The issue I mean to point out is
> >>>>>>>>>>>>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>>>>>>>>>>>             access
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streams in the same scope as the
> >>>>>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>>>>>>>>>>>             is, not
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> solutions.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> [Also, great to hear additional
> >>>>>>>>>>>>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>>>>>>>>>>>             excited
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> hear
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> The idea to postpone the wiring of
> >>>>>>>>>>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>>>>>>>>>>>> me at
> >>>>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> glance,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>>>>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>>>>>>>>>>>             same
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> other.  That is, if we wanted to merge
> >>>>>>>>>>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> again
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> don't see
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> You just took the words right out
> >>>>>>>>>>>>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>>>>>>>>>>>             just
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> going
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Consider the example from Bill's
> >>>>>>>>>>>>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>>>>>>>>>>>             we need
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> identify
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> customers who have bought coffee and
> >>>>>>>>>>>>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>>>>>>>>>>>             in the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> electronics
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> This is the code I usually write under
> >>>>>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>>>             circumstances
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>       return
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>       /*In the real world the code
> >>>>>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>>>             complex, so
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> creation of
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>>>>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>>>>>>>>>>>             order to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> separate
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> >>>>>>>>>>>>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>>>>>>>>>>>             everything
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> later,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Does this make sense?  In order to
> >>>>>>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>>>>>             initialize the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> we need the terminal operation to be
> >>>>>>>>>>>>>>>>>>>>>>>> called
> >>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>>>>>>>>>>>             essentially
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> KIP I was going to write here. I have
> >>>>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>>>>>>>>>>>             based on
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> my
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> experience,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> so I will join the discussion on KIP-401
> >>>>>>>>>>>>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>>>>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>>>>>>>>>>>             fluent
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> API
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>>>>>>>>>>>>>>>             (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>>>>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> compatibility
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    there aren't any direct ones.
> >>>>>>>>>>>>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>>>>>>>>>>>             that Java
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> smart
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    distinguish between a
> >>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>>>>>>>>>>>             returning one
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> thing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    with no arguments returning
> >>>>>>>>>>>>>>>>>>>>>>>> another
> >>>>>>>>>>>>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> >>>>>>>>>>>>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>             need
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>>>>>>>>>>>             shares
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> >>>>>>>>>>>>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             branching.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    pretty in its current form, but I
> >>>>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>>>             demonstrates
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>>>>>>>>>>>>>>>> request should
> >>>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> final
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> even a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> starting point if we go in this
> >>>>>>>>>>>>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>>>>>>>>>>>             wanted to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> I will say though, that I'm not
> >>>>>>>>>>>>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>>>>>             solution
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> deprecated in favor of this, which
> >>>>>>>>>>>>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> suggested
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> was a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>             streams
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>>>>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>>>>>>>>>>>             is, if we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> wanted
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> them back together again I don't
> >>>>>>>>>>>>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>>>             that.  The
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>>>>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> either
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>>>>>>>>>>>             table.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>>>>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> First, it seems that it's
> >>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             branch API
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> needs
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> There are two potential ways to
> >>>>>>>>>>>>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>>>>>>>>>>>             returns
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> >>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>>>>>>>>>>>> The code
> >>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>         ��   >>>>> make
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> sense
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> all the necessary ingredients are
> >>>>>>>>>>>>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>             instance
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> >>>>>>>>>>>>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> defined.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> >>>>>>>>>>>>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>>>>>>>>>>>             miss the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that one
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> of the terminal methods should be
> >>>>>>>>>>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>>>>>>>>>>>             methods
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>>>>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> I see your point when you are
> >>>>>>>>>>>>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> >>>>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>>>>             implemented the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> easy
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> >>>>>>>>>>>>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> >>>>>>>>>>>>>>>>>>>>>>>> method that
> >>>>>>>>>>>>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> >>>>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>>>> the only
> >>>>>>>>>>>>>>>>>>>>>>> option
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> besides
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>>>>>>>>>>>             want to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> silently
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> >>>>>>>>>>>>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>>>>>>>>>>>             predicate. 2)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Throwing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>>             looks
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> like a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> bad
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> >>>>>>>>>>>>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>>>>>>>>>>>             emit a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> >>>>>>>>>>>>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> `default`
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>>>             a clear
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> >>>>>>>>>>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> compiled
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> run?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> >>>>>>>>>>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>>>             compile if
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> used
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>>>>>>>>>>>             method chain
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> >>>>>>>>>>>>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>>>             between
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> instantly
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> >>>>>>>>>>>>>>>>>>>>>>>> project
> >>>>>>>>>>>>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>>>>>>>>>>>             compilation
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>>>>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>>>>>>>>>>>             required.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> >>>>>>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> defaultBranch
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>>>>>>>>>>>             just as
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> think it creates an
> >>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> user
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> default
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>>>>>>>>>             which allows
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>>>>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>>>>>>>>>>>             to be
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> well
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>>>             a clear
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> "build
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> step"
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> allow users to do other
> >>>>>>>>>>>>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>>>>>>>>>             stream.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>> way all
> >>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> do -
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> want to process off the original
> >>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> times,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> >>>>>>>>>>>>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>>>>>>>>>>>             operations
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> >>>>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>>>>>>>             always need
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> >>>>>>>>>>>>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>>>             so we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> more with the original branch
> >>>>>>>>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> >>>>>>>>>>>>>>>>>>>>>>>> need of
> >>>>>>>>>>>>>>>>>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> object
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> here
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>>>>>>>>>>>             flow,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> >>>>>>>>>>>>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>             API, but I
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> find
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>>>>>>>>>             contrasts the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> fluency
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
> >>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>>>>>>>>>             just call
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> method on
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> >>>>>>>>>>>>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>>>>>>>>>>>             cases
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> I think the
> >>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>>>>>>>>>>>             is very
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> nice
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> >>>>>>>>>>>>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>>>             how we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>>>>>>>>>>>             terminated by
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>>>>>>>>>>>             incompatible with
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> >>>>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>>             different
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> name,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> >>>>>>>>>>>>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> >>>>>>>>>>>>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>             KIP?  It
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>>>             while also
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> allowing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>>>>>>>>>>>             KBranchedStreams
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>>>>>>>>>>>             ks){
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>>>>>>>>>>>             String> ks){
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>             this::handleFirstCase)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>>>             this::handleSecondCase)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> takes a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Consumer
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>>>             example in
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>>>>>>>>>>>> how would
> >>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>>>>>>>>>>>             we had
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> 418.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Please
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>>>>>>>>>>>             feedback :)
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>             >
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Attachments:
> >>>>>>>>>> * signature.asc
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>>
> >>
> >>
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Alright, I have updated the KIP with what we have discussed:

1. Per Mathhias's suggestion, if a chain function returns null, the 
respective result if omitted in the resulting Map.

2. `with[Java]Consumer` method dropped.

3. `Branched` class has only three static methods with all the possible 
combinations of parameters.

4. Chain function is defined 'fully covariant', let's see if we can 
implement it this way :-))

+ code example updates and minor edits.


Since this is my first KIP, I'm not sure what should I do next. I feel 
that we talked over all the details and the consensus is reached. Is it 
OK to call for VOTE now or is it better to wait for more feedback?

Regards,

Ivan



28.05.2020 3:26, John Roesler пишет:
> Sounds good to me, Ivan!
> -John
> 
> On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
>> John,
>>
>> ---------------------
>>
>>   > I'm sorry things have been dragging out a little, but I have the
>> sense we're very close to the end of this discussion, which is exciting.
>>
>> We are certainly moving forward!  And I'm not in a hurry at all. As I
>> told you before in my projects I'm using Spring Kafka's
>> KafkaStreamBrancher -- the implementation of the first, rejected version
>> of this KIP. It's inferior to what we are discussing here, but it does
>> its work. So it's worth to design this KIP really, really well. And by
>> the way, from this discussion I'm learning the good API designing
>> process. For me it has a value per se :-))
>>
>> ----------------------
>>
>>   >>   > I'd wonder whether we need the non-static builders (like withChain).
>>   >>   > Do they provide any benefit over just using the right static
>> factory?
>>   >
>>   > I don't have a strong feeling, either. It seems nice to offer a better
>>   > type inference experience than what we get with Materialized, by
>>   > offering the static method that takes both name and chain.
>>   > Given that, there doesn't seem to be a good reason to also offer the
>>   > non-static builder-style methods, so I guess I'd prefer to drop them.
>>
>> I agree again! From a recent discussion on Twitter
>> (https://twitter.com/inponomarev/status/1265220044394545153) I found out
>> an interesting fact about type inference rules in Java. Funny thing is
>> that although we need to explicitly set types in a chain like this
>>
>> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
>>
>> (otherwise it won't compile), the composition of static method calls
>> works just fine, all the types are being calculated correctly:
>>
>> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
>>
>> As I was told, for type inference there is difference between qualifiers
>> and arguments, 'you go up if you are argument but stop if you are
>> qualifier'. And it also seems that we should not bet on any future
>> improvements in Java type inference here.
>>
>> So,
>>
>> 1) I think we that in this KIP we should provide three static methods
>> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and
>> drop any non-static ones.
>>
>> 2) If anything else will be ever needed, we can easily add anything.
>> Maybe this can be done in a process of refinement of all the parameter
>> classes.
>>
>> ----------------------------
>>
>>   > we may as well hope for the best, and propose the "fully
>>   > covariant" definition for now.
>>
>> Understood and agreed! I will edit the KIP.
>>
>> ----------------------------
>>
>>   >> Good question, I already thought about it and rejected the idea....
>>   > (I cut off your quote; the rest is in the chain below)
>>   >  "Worst" case scenario: someone
>>   > else will wish the return type is something different, and we'll go
>>   > through a painless deprecation transition to change it later.
>>
>> Of course, we cant' predict all the ways people are going to use it.
>>   From my own humble experience with Kafka Streams, the worst scenario is
>> unlikely. Split is split, transform is transform, too much flexibility
>> is often evil.
>>
>> ------------------------
>>
>> So it seems that we are close to the consensus. Two things to be altered
>> in the current version of KIP:
>>
>> * list of Branched methods, drop non-static methods.
>> * 'fully covariant' definition of `chained`.
>>
>> Any ideas / questions / objections?
>>
>> Regards,
>>
>> Ivan
>>
>>
>>
>>
>> 27.05.2020 7:03, John Roesler пишет:
>>> Thanks for the reply, Ivan,
>>>
>>> I'm sorry things have been dragging out a little, but I have the sense
>>> we're very close to the end of this discussion, which is exciting.
>>>
>>>>    > I'd wonder whether we need the non-static builders (like withChain).
>>>>    > Do they provide any benefit over just using the right static factory?
>>>>
>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>> be omitted, I agree!
>>>
>>> I don't have a strong feeling, either. It seems nice to offer a better
>>> type inference experience than what we get with Materialized, by
>>> offering the static method that takes both name and chain.
>>>
>>> Given that, there doesn't seem to be a good reason to also offer the
>>> non-static builder-style methods, so I guess I'd prefer to drop them.
>>>
>>> I'll defer to Matthias, if he has a chance to consider whether it's more
>>> valuable to stick with the existing pattern or break the pattern to offer
>>> a better experience.
>>>
>>>>    > You might as well propose the “ideal” API in the KIP, which is the
>>>>    > covariant typed function
>>>>
>>>> I didn't quite get it. Do I get you right that you propose the
>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>> then just see if there are any obstacles/pitfalls during implementation
>>>> and unit testing?
>>>
>>> Yep! I know it's a bit sloppy, but my experience has been that we just
>>> won't know what works until we really try it, and try it in several different
>>> ways. Still, we may as well hope for the best, and propose the "fully
>>> covariant" definition for now.
>>>
>>>>    > Is it necessary to restrict the result key and value types to be the
>>>>    > same as the inputs?
>>>>
>>>> Good question, I already thought about it and rejected the idea....
>>> (I cut off your quote; the rest is in the chain below)
>>>
>>> That's fair! It's your KIP, after all. I think I might have made a different
>>> call here, but I think this choice is fine. "Worst" case scenario: someone
>>> else will wish the return type is something different, and we'll go
>>> through a painless deprecation transition to change it later. Thanks
>>> to the clean design of your API, this doesn't seem to bad. And, of
>>> course, you've actually been using similar functionality already, so it
>>> seems we should trust your intuition.
>>>
>>> Thanks,
>>> -John
>>>
>>>
>>> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>>>> Hi John,
>>>>
>>>>    > I'd wonder whether we need the non-static builders (like withChain).
>>>> Do they provide any benefit over just using the right static factory?
>>>>
>>>> I don't have a strong opinion here. I think it's just a matter of taste.
>>>> But, if we like to use Occam's razor, then yes, non-static builders can
>>>> be omitted, I agree!
>>>>
>>>>    > You might as well propose the “ideal” API in the KIP, which is the
>>>> covariant typed function
>>>>
>>>> I didn't quite get it. Do I get you right that you propose the
>>>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>>>> then just see if there are any obstacles/pitfalls during implementation
>>>> and unit testing?
>>>>
>>>>    > Is it necessary to restrict the result key and value types to be the
>>>> same as the inputs?
>>>>
>>>> Good question, I already thought about it and rejected the idea.
>>>>
>>>> Look, if we want to keep `withChain`'s function optional, then we must
>>>> keep the result key and value types the same. Because for now, the
>>>> default value for the 'chain function' is Function.identity().
>>>>
>>>> Of course, we can make the 'chain function' required. But I think this
>>>> is not what `split` method is for. `split` is for splitting, not
>>>> transforming, and `chainFunction` in most of the cases should be either
>>>> a consumer or the identity function.
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>>
>>>> 24.05.2020 17:15, John Roesler пишет:
>>>>> Thanks for the reply, Ivan.
>>>>>
>>>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
>>>>>
>>>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
>>>>>
>>>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
>>>>>
>>>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
>>>>>
>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
>>>>>
>>>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
>>>>>
>>>>> Thanks,
>>>>> John
>>>>>
>>>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>>>> Hello John,
>>>>>>
>>>>>>
>>>>>> 1.
>>>>>> ---------------------------------------------
>>>>>>
>>>>>>     > Perhaps it would be better to stick with "as" for now
>>>>>>     > and just file a Jira to switch them all at the same time [for
>>>>>> compatibility with Kotlin]
>>>>>>
>>>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
>>>>>> standard workaround
>>>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>>>> So actually this should be a very low priority issue, if an issue at
>>>>>> all.
>>>>>>
>>>>>>     > I don't understand how your new proposed
>>>>>>     > methods would work any differently than the ones you already
>>>>>>     > had proposed in the KIP. It seems like you'd still have to provide
>>>>>>     > the generic type parameters on the first static factory call. Can you
>>>>>>     > explain how your new interface proposal differs from the existing KIP?
>>>>>>
>>>>>> In the KIP, I didn't clarify what methods should be static. Now I
>>>>>> propose the following methods:
>>>>>>
>>>>>> non-static: withChain(Function), withName(String).
>>>>>>
>>>>>> static: as(String), with(Function), with(Function, String).
>>>>>>
>>>>>> The overloaded `with` version that provides both Function and name can
>>>>>> be used without causing type inference problem!!
>>>>>>
>>>>>> 2.
>>>>>> ----------------------------
>>>>>>
>>>>>>     > Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>     > be nice, but I'm not sure it will actually work.
>>>>>>
>>>>>> What I'm keeping in mind is the following example: imagine
>>>>>>
>>>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
>>>>>>             return s.mapValues(n -> (Integer) n + 1);
>>>>>> }
>>>>>>
>>>>>> BranchedKStream<String, Number> b =
>>>>>>         s.split().branch((k, v) -> isInteger(v),
>>>>>>                    //Won't compile!!
>>>>>>                    Branched.with(Me::func));
>>>>>>
>>>>>> The simple workaround here is to change `func`'s return type from
>>>>>> KStream<...Integer> to KStream<...Number>.
>>>>>>
>>>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
>>>>>> `Branched`, so during code migration I will have to modify my functions'
>>>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>>>
>>>>>>     >  the map you're returning is Map<K,V>, and of course a K is not the
>>>>>> same as "? extends K", so it doesn't seem compatible.
>>>>>>
>>>>>> I think what you actually meant here is that KStream<? extends K, ?
>>>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
>>>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
>>>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
>>>>>>
>>>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
>>>>>> for now it's better to just admit that API is not absolutely perfect and
>>>>>> accept it as is, that is
>>>>>>
>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>>
>>>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>>>> Hello Ivan,
>>>>>>>
>>>>>>> Thanks for the refinement. Actually, I did not know that "as" would
>>>>>>> clash with a Kotlin operator. Maybe we should depart from convention
>>>>>>> and just avoid methods named "as" in the future.
>>>>>>>
>>>>>>> The convention is that "as(String name)" is used for the static factory
>>>>>>> method, whereas "withName(String name)" is an instance method
>>>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
>>>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
>>>>>>> although it's somewhat dubious, since all the other configuration
>>>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
>>>>>>> and just file a Jira to switch them all at the same time.
>>>>>>>
>>>>>>> Re. 3:
>>>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
>>>>>>> configuraion objects. The problem is that Java infers the type
>>>>>>> based on the _first_ method in the chain. While it does consider what
>>>>>>> the recipient of the method result wants, it only considers the _next_
>>>>>>> recipient.
>>>>>>>
>>>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>>>> Branched<String,String> variable, java infers the type correctly. But
>>>>>>> when the "next recipient" is a chained method call, like "withChain",
>>>>>>> then the chained method doesn't bound the type (by definition,
>>>>>>> withChain is defined on Branched<Object, Object>, so Java will take
>>>>>>> the broadest possible inferece and bind the type to
>>>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
>>>>>>>
>>>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
>>>>>>> that hard to recursively consider the entire context when inferring the
>>>>>>> generic type parameters, but this is what we have to work with.
>>>>>>>
>>>>>>> To be honest, though, I don't understand how your new proposed
>>>>>>> methods would work any differently than the ones you already
>>>>>>> had proposed in the KIP. It seems like you'd still have to provide
>>>>>>> the generic type parameters on the first static factory call. Can you
>>>>>>> explain how your new interface proposal differs from the existing KIP?
>>>>>>>
>>>>>>> Re. 4:
>>>>>>> Regarding making the K,V types covariant also, yes, that would indeed
>>>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
>>>>>>> try. In the past, we've run into soe truly strange interactions between the
>>>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
>>>>>>> combination with nested covariant types.
>>>>>>>
>>>>>>> Another issue is that the value type of the map you're returning is
>>>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
>>>>>>> doesn't seem compatible.
>>>>>>>
>>>>>>> Thanks again,
>>>>>>> -John
>>>>>>>
>>>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>>>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>>>>>>>> simpler'.
>>>>>>>>
>>>>>>>> I made some quick API mocking in my IDE and tried to implement examples
>>>>>>>> from KIP.
>>>>>>>>
>>>>>>>> 1. Having to return something from lambda is not a very big deal.
>>>>>>>>
>>>>>>>> 2. For a moment I thouht that I won't be able to use method references
>>>>>>>> for already written stream consumers, but then I realized that I can
>>>>>>>> just change my methods from returning void to returning the input
>>>>>>>> parameter and use references to them. Not very convenient, but passable.
>>>>>>>>
>>>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>>>>>>>> function returns null, we don't insert it into the resulting map.
>>>>>>>>
>>>>>>>> Usually it's better to implement a non-perfect, but workable solution as
>>>>>>>> a first approximation. And later we can always add to `Branched`
>>>>>>>> anything we want.
>>>>>>>>
>>>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
>>>>>>>> should be built? First of all, it seems that `as` now is more preferred
>>>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
>>>>>>>> `as` operator).
>>>>>>>>
>>>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
>>>>>>>> infer types in the following construction:
>>>>>>>>
>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>         Branched.as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>
>>>>>>>>
>>>>>>>> so I have to write
>>>>>>>>
>>>>>>>> .branch((key, value) -> value == null,
>>>>>>>>         Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>>>>>>>
>>>>>>>>
>>>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
>>>>>>>> that I came to (will you please validate it):
>>>>>>>>
>>>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>>>
>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>> extends KStream<K, V>> chain);
>>>>>>>>
>>>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>>>
>>>>>>>> //non-static!
>>>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>>>>>>>> KStream<K, V>> chain);
>>>>>>>>
>>>>>>>>
>>>>>>>> 4. And one more. What do you think, do we need that flexibility:
>>>>>>>>
>>>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>>>>>>>
>>>>>>>> vs.
>>>>>>>>
>>>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>>>>>>>> extends K, ? extends V>> chain
>>>>>>>>
>>>>>>>> ??
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>>
>>>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>>>> Thanks for this thought, Matthias,
>>>>>>>>>
>>>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>>>>>>>
>>>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>>>>>>>
>>>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>>>>>>>
>>>>>>>>> Thanks again for sharing the idea,
>>>>>>>>> John
>>>>>>>>>
>>>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>
>>>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>>>
>>>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>>>
>>>>>>>>>>       (1) split a stream and return the substreams for futher processing
>>>>>>>>>>       (2) split a stream and modify the substreams with in-place method chaining
>>>>>>>>>>
>>>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>>>
>>>>>>>>>>        -> split a stream, modify the substreams, and return the _modified_
>>>>>>>>>> substreams for further processing
>>>>>>>>>>
>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>>>>>>>
>>>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>>>>>>>
>>>>>>>>>>       - using `withChain` I get the modified sub-stream
>>>>>>>>>>       - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>>>>>>>
>>>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>      From my understanding the original idea of `withJavaConsumer` was to
>>>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>>>
>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>        s.to();
>>>>>>>>>>        return null;
>>>>>>>>>> })
>>>>>>>>>>
>>>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
>>>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>>>
>>>>>>>>>> Following this train of through, and if we want to allow the "return
>>>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>>>>>>>> an entry to the Map.
>>>>>>>>>>
>>>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>>>>>>>> be achieved with `withChain`:
>>>>>>>>>>
>>>>>>>>>> Branched.withChain(s -> {
>>>>>>>>>>        s.to();
>>>>>>>>>>        return s;
>>>>>>>>>> })
>>>>>>>>>>
>>>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>>>>>>>> while for the first proposal it adds new functionality (if `return null`
>>>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> I guess I can be convinced either way. However, if we follow your
>>>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
>>>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>>>
>>>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>>>
>>>>>>>>>>> -----------------------------------------
>>>>>>>>>>>
>>>>>>>>>>> John,
>>>>>>>>>>>
>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>>>
>>>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>>>>>>>> all the emails on the web.
>>>>>>>>>>>
>>>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>>>>>>>> method?
>>>>>>>>>>>
>>>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>>>>>>>
>>>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>>>>>>>
>>>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>>>>>>>
>>>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>>>>>>>
>>>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>>>
>>>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>>>
>>>>>>>>>>> This is discussed below.
>>>>>>>>>>>
>>>>>>>>>>> ----------------------------------------------
>>>>>>>>>>>
>>>>>>>>>>> Mathhias,
>>>>>>>>>>>
>>>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>>>>>>>
>>>>>>>>>>> Done.
>>>>>>>>>>>
>>>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>>>>>>>> method]
>>>>>>>>>>>
>>>>>>>>>>> Fixed.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>>>
>>>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>>>
>>>>>>>>>>> Yes, I agree!
>>>>>>>>>>>
>>>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>>>>>>>> implementation would be to not add the "branch" to the result map if
>>>>>>>>>>> `withConsumer` is used?
>>>>>>>>>>>
>>>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>>>>>>>
>>>>>>>>>>> In the current version of KIP it is assumed that the returned map
>>>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>>>>>>>> map is the user's responsibility.
>>>>>>>>>>>
>>>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>>>>>>>> it as it is?
>>>>>>>>>>>
>>>>>>>>>>> ----
>>>>>>>>>>>
>>>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>>>
>>>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>>>
>>>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>>>>>>>
>>>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>>>>>>>> so we just keep them.)
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (2) Quote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>>>
>>>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>>>>>>>> 4th comment:
>>>>>>>>>>>>
>>>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>>>
>>>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>>>>>>>
>>>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>>>
>>>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>>>
>>>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>>>
>>>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> John
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>>>>>>>> as it
>>>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>>>            .branch(....)
>>>>>>>>>>>>>>>>>            .defaultBranch(result::set)
>>>>>>>>>>>>>>>>>            .onTopOf(someStream);
>>>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>          -> KBranchedStream
>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>          -> KBranchedStream
>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>             withChain(...);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>            stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>>>                       .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>>>                       .defaultBranch(...) // creates node
>>>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>             operator(function, config_object?) OR
>>>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>           operator(function, function, string)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>            KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>            interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>>>>>>>             withPredicate(...);
>>>>>>>>>>>>>>>>>>>>>             withChain(...);
>>>>>>>>>>>>>>>>>>>>>             withName(...);
>>>>>>>>>>>>>>>>>>>>>           }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>>>           -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>           -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>>>             for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>             question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>>>             in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>             seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>>>             when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>>>             for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>>>             solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>>>             to.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>             <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>>>             LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>>>             matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>>>             advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>>>             proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>             But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>>>             workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>>>             method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>>>             tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>>>             going to be clean.
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>>>             handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>>>             a handler?
>>>>>>>>>>>>>>>>>>>>>>>>             > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>>>>>>>             > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>             writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>>>             "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>>>             identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>>>             all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>>>             point?
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > ---
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>             without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>>>             the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>>>             current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>>>             me address both comments here.
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>>>             for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>>>             see example below.
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>>>             Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>>>             value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>>>             have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>>>             > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>>>             > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>>>             >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>>>             recordType,
>>>>>>>>>>>>>>>>>>>>>>>>             >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>             > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>>>             >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>>>             thinking
>>>>>>>>>>>>>>>>>>>>>>>>             >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>>>             >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>>>             >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>>>             >> An open question is the case for which no
>>>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>             >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>             `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>>>             >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>>>             >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>>>             >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>>>             `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>>>             >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>>>             >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>>>             argument" yet
>>>>>>>>>>>>>>>>>>>>>>>>             >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>             >>
>>>>>>>>>>>>>>>>>>>>>>>>             >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>>>             that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>>>             it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>>>             The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>             (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>>>             to each other.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>>>             branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>>>             <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>             <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>>>             downstream
>>>>>>>>>>>>>>>>>>>>>>>>             >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>             quickly become
>>>>>>>>>>>>>>>>>>>>>>>>             >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>>             <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>>             that sets a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>>>             current
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>>>             decisions) is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>             construct
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>             provides as much
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>             call
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>>>             The value it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>>>             the same
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>             in the sense
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>>>             branches, but it is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>>>             the "static"
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>>>             fluently declare and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>>>             ignore a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>>>             SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>>>             SortedMap being
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>>>             like this:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>             readable than
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>             makes it harder
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>>>             unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>>>             existing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>>>             that's overdoing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>>>             access the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>             possible that it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>             left up to the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>>>             <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>             existing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>>>             proceed by
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>             important to be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>>>             this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>             double brace
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>>             <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>>>             To me,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> there
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>>>             positionally coupled
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>             additional code
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>>>             constructs
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>>>             looks like
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>>>             Predicate<?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>>>             returns a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>>>             solves
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>>>             branch you’re
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>>             another
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>>>             structure, rather
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>>>             historically
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>>>             voluminously, but
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>>>             <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>>>             "Split"
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>>>             is, `branch`
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>>>             `void`, to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>>>             definition of
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>>>             there's no
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>>>             record
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>>>             should
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>>>             `branch()` does
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>>>             I though
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>>>             avoid
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>>>             with a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> short
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>>             with all
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>>>             contained in
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>>>             `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>>>>>>>             branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>>>             that users
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>>>             access them.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>>>             to get
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>>>             loss of
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>>>             without
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> should
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>>>             with such
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>>>             something.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>             proposal.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>             recommend the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>>>             BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>>>             `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>>>             accepted and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>>>             parameter.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>>>             different
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>>>             index)` method
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>>>             object?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>             whatever
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>             current
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>             makes sense
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>>>             we'll
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> need
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>>>             think this
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>>>             starting a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>>>             assuming
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>>>             similar
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>>>             KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>>>             convinced: your
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>>>             And then
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>>>             resume the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>>>             starting
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>>>             still
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             depend on
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>>>             access
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>>>             is, not
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>>>             excited
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>>>             same
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> again
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>>>             just
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>>>             we need
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>>>             in the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>>>             circumstances
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>>>             complex, so
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>>>             order to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>>>             everything
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>>>             initialize the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>>>             essentially
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>>>             based on
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>>>             fluent
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> API
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>>>>>             (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>>>             that Java
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>>>             returning one
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>             need
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>>>             shares
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             branching.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>>>             demonstrates
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> final
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>>>             wanted to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>>             solution
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>             streams
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>>>             is, if we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>>>             that.  The
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>>>             table.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             branch API
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>>>             returns
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>         ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>             instance
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>>>             miss the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>>>             methods
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>>             implemented the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>>>             want to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>>>             predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>             looks
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>>>             emit a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>             a clear
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>>>             compile if
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> used
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>>>             method chain
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>>>             between
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>>>             compilation
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>>>             required.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>>>             just as
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>>>             which allows
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>>>             to be
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>>>             a clear
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>>>             stream.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>>>             operations
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>>>             always need
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>>>             so we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> object
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> here
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>>>             flow,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>             API, but I
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>>>             contrasts the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>>>             just call
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>>>             cases
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> are
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>>>             is very
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>>>             how we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>>>             terminated by
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>>>             incompatible with
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>             different
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>             KIP?  It
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>>>             while also
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>>>             KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>>>             ks){
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>>>             String> ks){
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>             this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>>>             this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>>>             example in
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>>>             we had
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>>>             feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>             >
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Attachments:
>>>>>>>>>> * signature.asc
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Sounds good to me, Ivan!
-John

On Wed, May 27, 2020, at 07:13, Ivan Ponomarev wrote:
> John,
> 
> ---------------------
> 
>  > I'm sorry things have been dragging out a little, but I have the 
> sense we're very close to the end of this discussion, which is exciting.
> 
> We are certainly moving forward!  And I'm not in a hurry at all. As I 
> told you before in my projects I'm using Spring Kafka's 
> KafkaStreamBrancher -- the implementation of the first, rejected version 
> of this KIP. It's inferior to what we are discussing here, but it does 
> its work. So it's worth to design this KIP really, really well. And by 
> the way, from this discussion I'm learning the good API designing 
> process. For me it has a value per se :-))
> 
> ----------------------
> 
>  >>   > I'd wonder whether we need the non-static builders (like withChain).
>  >>   > Do they provide any benefit over just using the right static 
> factory?
>  >
>  > I don't have a strong feeling, either. It seems nice to offer a better
>  > type inference experience than what we get with Materialized, by
>  > offering the static method that takes both name and chain.
>  > Given that, there doesn't seem to be a good reason to also offer the
>  > non-static builder-style methods, so I guess I'd prefer to drop them.
> 
> I agree again! From a recent discussion on Twitter 
> (https://twitter.com/inponomarev/status/1265220044394545153) I found out 
> an interesting fact about type inference rules in Java. Funny thing is 
> that although we need to explicitly set types in a chain like this
> 
> foo.branch(..., Branched.<...,...>named("foo").withChain(...));
> 
> (otherwise it won't compile), the composition of static method calls 
> works just fine, all the types are being calculated correctly:
> 
> foo.branch(..., Branched.withChain(Branched.named("foo"), ...));
> 
> As I was told, for type inference there is difference between qualifiers 
> and arguments, 'you go up if you are argument but stop if you are 
> qualifier'. And it also seems that we should not bet on any future 
> improvements in Java type inference here.
> 
> So,
> 
> 1) I think we that in this KIP we should provide three static methods 
> only: `as(String)`, `with(Function)`, and `with(Function, String)`, and 
> drop any non-static ones.
> 
> 2) If anything else will be ever needed, we can easily add anything. 
> Maybe this can be done in a process of refinement of all the parameter 
> classes.
> 
> ----------------------------
> 
>  > we may as well hope for the best, and propose the "fully
>  > covariant" definition for now.
> 
> Understood and agreed! I will edit the KIP.
> 
> ----------------------------
> 
>  >> Good question, I already thought about it and rejected the idea....
>  > (I cut off your quote; the rest is in the chain below)
>  >  "Worst" case scenario: someone
>  > else will wish the return type is something different, and we'll go
>  > through a painless deprecation transition to change it later.
> 
> Of course, we cant' predict all the ways people are going to use it. 
>  From my own humble experience with Kafka Streams, the worst scenario is 
> unlikely. Split is split, transform is transform, too much flexibility 
> is often evil.
> 
> ------------------------
> 
> So it seems that we are close to the consensus. Two things to be altered 
> in the current version of KIP:
> 
> * list of Branched methods, drop non-static methods.
> * 'fully covariant' definition of `chained`.
> 
> Any ideas / questions / objections?
> 
> Regards,
> 
> Ivan
> 
> 
> 
> 
> 27.05.2020 7:03, John Roesler пишет:
> > Thanks for the reply, Ivan,
> > 
> > I'm sorry things have been dragging out a little, but I have the sense
> > we're very close to the end of this discussion, which is exciting.
> > 
> >>   > I'd wonder whether we need the non-static builders (like withChain).
> >>   > Do they provide any benefit over just using the right static factory?
> >>
> >> I don't have a strong opinion here. I think it's just a matter of taste.
> >> But, if we like to use Occam's razor, then yes, non-static builders can
> >> be omitted, I agree!
> > 
> > I don't have a strong feeling, either. It seems nice to offer a better
> > type inference experience than what we get with Materialized, by
> > offering the static method that takes both name and chain.
> > 
> > Given that, there doesn't seem to be a good reason to also offer the
> > non-static builder-style methods, so I guess I'd prefer to drop them.
> > 
> > I'll defer to Matthias, if he has a chance to consider whether it's more
> > valuable to stick with the existing pattern or break the pattern to offer
> > a better experience.
> > 
> >>   > You might as well propose the “ideal” API in the KIP, which is the
> >>   > covariant typed function
> >>
> >> I didn't quite get it. Do I get you right that you propose the
> >> following: 1) accept the 'fully covariant' definition in the KIP, 2)
> >> then just see if there are any obstacles/pitfalls during implementation
> >> and unit testing?
> > 
> > Yep! I know it's a bit sloppy, but my experience has been that we just
> > won't know what works until we really try it, and try it in several different
> > ways. Still, we may as well hope for the best, and propose the "fully
> > covariant" definition for now.
> > 
> >>   > Is it necessary to restrict the result key and value types to be the
> >>   > same as the inputs?
> >>
> >> Good question, I already thought about it and rejected the idea....
> > (I cut off your quote; the rest is in the chain below)
> > 
> > That's fair! It's your KIP, after all. I think I might have made a different
> > call here, but I think this choice is fine. "Worst" case scenario: someone
> > else will wish the return type is something different, and we'll go
> > through a painless deprecation transition to change it later. Thanks
> > to the clean design of your API, this doesn't seem to bad. And, of
> > course, you've actually been using similar functionality already, so it
> > seems we should trust your intuition.
> > 
> > Thanks,
> > -John
> > 
> > 
> > On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
> >> Hi John,
> >>
> >>   > I'd wonder whether we need the non-static builders (like withChain).
> >> Do they provide any benefit over just using the right static factory?
> >>
> >> I don't have a strong opinion here. I think it's just a matter of taste.
> >> But, if we like to use Occam's razor, then yes, non-static builders can
> >> be omitted, I agree!
> >>
> >>   > You might as well propose the “ideal” API in the KIP, which is the
> >> covariant typed function
> >>
> >> I didn't quite get it. Do I get you right that you propose the
> >> following: 1) accept the 'fully covariant' definition in the KIP, 2)
> >> then just see if there are any obstacles/pitfalls during implementation
> >> and unit testing?
> >>
> >>   > Is it necessary to restrict the result key and value types to be the
> >> same as the inputs?
> >>
> >> Good question, I already thought about it and rejected the idea.
> >>
> >> Look, if we want to keep `withChain`'s function optional, then we must
> >> keep the result key and value types the same. Because for now, the
> >> default value for the 'chain function' is Function.identity().
> >>
> >> Of course, we can make the 'chain function' required. But I think this
> >> is not what `split` method is for. `split` is for splitting, not
> >> transforming, and `chainFunction` in most of the cases should be either
> >> a consumer or the identity function.
> >>
> >> Regards,
> >>
> >> Ivan
> >>
> >>
> >> 24.05.2020 17:15, John Roesler пишет:
> >>> Thanks for the reply, Ivan.
> >>>
> >>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
> >>>
> >>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
> >>>
> >>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
> >>>
> >>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
> >>>
> >>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
> >>>
> >>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
> >>>
> >>> Thanks,
> >>> John
> >>>
> >>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
> >>>> Hello John,
> >>>>
> >>>>
> >>>> 1.
> >>>> ---------------------------------------------
> >>>>
> >>>>    > Perhaps it would be better to stick with "as" for now
> >>>>    > and just file a Jira to switch them all at the same time [for
> >>>> compatibility with Kotlin]
> >>>>
> >>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
> >>>> standard workaround
> >>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
> >>>> So actually this should be a very low priority issue, if an issue at
> >>>> all.
> >>>>
> >>>>    > I don't understand how your new proposed
> >>>>    > methods would work any differently than the ones you already
> >>>>    > had proposed in the KIP. It seems like you'd still have to provide
> >>>>    > the generic type parameters on the first static factory call. Can you
> >>>>    > explain how your new interface proposal differs from the existing KIP?
> >>>>
> >>>> In the KIP, I didn't clarify what methods should be static. Now I
> >>>> propose the following methods:
> >>>>
> >>>> non-static: withChain(Function), withName(String).
> >>>>
> >>>> static: as(String), with(Function), with(Function, String).
> >>>>
> >>>> The overloaded `with` version that provides both Function and name can
> >>>> be used without causing type inference problem!!
> >>>>
> >>>> 2.
> >>>> ----------------------------
> >>>>
> >>>>    > Regarding making the K,V types covariant also, yes, that would indeed
> >>>>    > be nice, but I'm not sure it will actually work.
> >>>>
> >>>> What I'm keeping in mind is the following example: imagine
> >>>>
> >>>> static KStream<String, Integer> func(KStream<String, Number> s) {
> >>>>            return s.mapValues(n -> (Integer) n + 1);
> >>>> }
> >>>>
> >>>> BranchedKStream<String, Number> b =
> >>>>        s.split().branch((k, v) -> isInteger(v),
> >>>>                   //Won't compile!!
> >>>>                   Branched.with(Me::func));
> >>>>
> >>>> The simple workaround here is to change `func`'s return type from
> >>>> KStream<...Integer> to KStream<...Number>.
> >>>>
> >>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
> >>>> `Branched`, so during code migration I will have to modify my functions'
> >>>> return types anyway -- I mean, from `void` to `KStream`!! ]
> >>>>
> >>>>    >  the map you're returning is Map<K,V>, and of course a K is not the
> >>>> same as "? extends K", so it doesn't seem compatible.
> >>>>
> >>>> I think what you actually meant here is that KStream<? extends K, ?
> >>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
> >>>> particularly is not a problem, since KStream<? extends K, ? extends V>
> >>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
> >>>>
> >>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
> >>>> for now it's better to just admit that API is not absolutely perfect and
> >>>> accept it as is, that is
> >>>>
> >>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
> >>>>
> >>>> Regards,
> >>>>
> >>>> Ivan
> >>>>
> >>>>
> >>>> 21.05.2020 17:59, John Roesler пишет:
> >>>>> Hello Ivan,
> >>>>>
> >>>>> Thanks for the refinement. Actually, I did not know that "as" would
> >>>>> clash with a Kotlin operator. Maybe we should depart from convention
> >>>>> and just avoid methods named "as" in the future.
> >>>>>
> >>>>> The convention is that "as(String name)" is used for the static factory
> >>>>> method, whereas "withName(String name)" is an instance method
> >>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
> >>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
> >>>>> although it's somewhat dubious, since all the other configuration
> >>>>> classes use "as". Perhaps it would be better to stick with "as" for now
> >>>>> and just file a Jira to switch them all at the same time.
> >>>>>
> >>>>> Re. 3:
> >>>>> Regarding the type inference problem, yes, it's a blemish on all of our
> >>>>> configuraion objects. The problem is that Java infers the type
> >>>>> based on the _first_ method in the chain. While it does consider what
> >>>>> the recipient of the method result wants, it only considers the _next_
> >>>>> recipient.
> >>>>>
> >>>>> Thus, if you call as("foo") and immediately assign it to a
> >>>>> Branched<String,String> variable, java infers the type correctly. But
> >>>>> when the "next recipient" is a chained method call, like "withChain",
> >>>>> then the chained method doesn't bound the type (by definition,
> >>>>> withChain is defined on Branched<Object, Object>, so Java will take
> >>>>> the broadest possible inferece and bind the type to
> >>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
> >>>>>
> >>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
> >>>>> that hard to recursively consider the entire context when inferring the
> >>>>> generic type parameters, but this is what we have to work with.
> >>>>>
> >>>>> To be honest, though, I don't understand how your new proposed
> >>>>> methods would work any differently than the ones you already
> >>>>> had proposed in the KIP. It seems like you'd still have to provide
> >>>>> the generic type parameters on the first static factory call. Can you
> >>>>> explain how your new interface proposal differs from the existing KIP?
> >>>>>
> >>>>> Re. 4:
> >>>>> Regarding making the K,V types covariant also, yes, that would indeed
> >>>>> be nice, but I'm not sure it will actually work. You might want to give it a
> >>>>> try. In the past, we've run into soe truly strange interactions between the
> >>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
> >>>>> combination with nested covariant types.
> >>>>>
> >>>>> Another issue is that the value type of the map you're returning is
> >>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
> >>>>> doesn't seem compatible.
> >>>>>
> >>>>> Thanks again,
> >>>>> -John
> >>>>>
> >>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
> >>>>>> Hi,
> >>>>>>
> >>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
> >>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
> >>>>>> simpler'.
> >>>>>>
> >>>>>> I made some quick API mocking in my IDE and tried to implement examples
> >>>>>> from KIP.
> >>>>>>
> >>>>>> 1. Having to return something from lambda is not a very big deal.
> >>>>>>
> >>>>>> 2. For a moment I thouht that I won't be able to use method references
> >>>>>> for already written stream consumers, but then I realized that I can
> >>>>>> just change my methods from returning void to returning the input
> >>>>>> parameter and use references to them. Not very convenient, but passable.
> >>>>>>
> >>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
> >>>>>> function returns null, we don't insert it into the resulting map.
> >>>>>>
> >>>>>> Usually it's better to implement a non-perfect, but workable solution as
> >>>>>> a first approximation. And later we can always add to `Branched`
> >>>>>> anything we want.
> >>>>>>
> >>>>>> 3. Do we have any guidelines on how parameter classes like Branched
> >>>>>> should be built? First of all, it seems that `as` now is more preferred
> >>>>>> than `withName` (although as you probably know it clashes with Kotlin's
> >>>>>> `as` operator).
> >>>>>>
> >>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
> >>>>>> infer types in the following construction:
> >>>>>>
> >>>>>> .branch((key, value) -> value == null,
> >>>>>>        Branched.as("foo").withChain(s -> s.mapValues(...)))
> >>>>>>
> >>>>>>
> >>>>>> so I have to write
> >>>>>>
> >>>>>> .branch((key, value) -> value == null,
> >>>>>>        Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
> >>>>>>
> >>>>>>
> >>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
> >>>>>> that I came to (will you please validate it):
> >>>>>>
> >>>>>> static <K, V> Branched<K, V> as(String name);
> >>>>>>
> >>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>>>> extends KStream<K, V>> chain);
> >>>>>>
> >>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>>>> extends KStream<K, V>> chain, String name);
> >>>>>>
> >>>>>> //non-static!
> >>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
> >>>>>> KStream<K, V>> chain);
> >>>>>>
> >>>>>>
> >>>>>> 4. And one more. What do you think, do we need that flexibility:
> >>>>>>
> >>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
> >>>>>>
> >>>>>> vs.
> >>>>>>
> >>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
> >>>>>> extends K, ? extends V>> chain
> >>>>>>
> >>>>>> ??
> >>>>>>
> >>>>>> Regards,
> >>>>>>
> >>>>>> Ivan
> >>>>>>
> >>>>>>
> >>>>>> 21.05.2020 6:54, John Roesler пишет:
> >>>>>>> Thanks for this thought, Matthias,
> >>>>>>>
> >>>>>>> Your idea has a few aspects I find attractive:
> >>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
> >>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
> >>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
> >>>>>>>
> >>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
> >>>>>>>
> >>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
> >>>>>>>
> >>>>>>> Thanks again for sharing the idea,
> >>>>>>> John
> >>>>>>>
> >>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> >>>>>>>> Thanks for updating the KIP!
> >>>>>>>>
> >>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
> >>>>>>>> its relationship to the returned `Map`.
> >>>>>>>>
> >>>>>>>> Originally, we discussed two main patterns:
> >>>>>>>>
> >>>>>>>>      (1) split a stream and return the substreams for futher processing
> >>>>>>>>      (2) split a stream and modify the substreams with in-place method chaining
> >>>>>>>>
> >>>>>>>> To combine both patterns we wanted to allow for
> >>>>>>>>
> >>>>>>>>       -> split a stream, modify the substreams, and return the _modified_
> >>>>>>>> substreams for further processing
> >>>>>>>>
> >>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> >>>>>>>>
> >>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
> >>>>>>>>
> >>>>>>>>      - using `withChain` I get the modified sub-stream
> >>>>>>>>      - using `withJavaConsumer` I get the unmodifed sub-stream
> >>>>>>>>
> >>>>>>>> This seems to be quite subtle to me.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>     From my understanding the original idea of `withJavaConsumer` was to
> >>>>>>>> model a terminal operation, ie, it should be similar to:
> >>>>>>>>
> >>>>>>>> Branched.withChain(s -> {
> >>>>>>>>       s.to();
> >>>>>>>>       return null;
> >>>>>>>> })
> >>>>>>>>
> >>>>>>>> However, I am not sure if we should even allow `withChain()` to return
> >>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
> >>>>>>>> -> null` entry in the returned Map.
> >>>>>>>>
> >>>>>>>> Following this train of through, and if we want to allow the "return
> >>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
> >>>>>>>> an entry to the Map.
> >>>>>>>>
> >>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
> >>>>>>>> be achieved with `withChain`:
> >>>>>>>>
> >>>>>>>> Branched.withChain(s -> {
> >>>>>>>>       s.to();
> >>>>>>>>       return s;
> >>>>>>>> })
> >>>>>>>>
> >>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> >>>>>>>> while for the first proposal it adds new functionality (if `return null`
> >>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
> >>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
> >>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> I guess I can be convinced either way. However, if we follow your
> >>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
> >>>>>>>> benefit seems to be small? Also, having a reduced API is usually
> >>>>>>>> preferable as it's simpler to learn.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> >>>>>>>>> Hello, John, hello Matthias!
> >>>>>>>>>
> >>>>>>>>> Thank you very much for your detailed feedback!
> >>>>>>>>>
> >>>>>>>>> -----------------------------------------
> >>>>>>>>>
> >>>>>>>>> John,
> >>>>>>>>>
> >>>>>>>>>> It looks like you missed my reply on Apr 23rd.
> >>>>>>>>>
> >>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
> >>>>>>>>> all the emails on the web.
> >>>>>>>>>
> >>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> >>>>>>>>> method?
> >>>>>>>>>
> >>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
> >>>>>>>>>
> >>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
> >>>>>>>>> off of the parent KStream for the needs of dynamic branching]
> >>>>>>>>>
> >>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
> >>>>>>>>>
> >>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> >>>>>>>>>
> >>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
> >>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
> >>>>>>>>>
> >>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
> >>>>>>>>> chain and the result map OR using just the sink
> >>>>>>>>>
> >>>>>>>>> This is discussed below.
> >>>>>>>>>
> >>>>>>>>> ----------------------------------------------
> >>>>>>>>>
> >>>>>>>>> Mathhias,
> >>>>>>>>>
> >>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> >>>>>>>>>
> >>>>>>>>> Done.
> >>>>>>>>>
> >>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> >>>>>>>>> method]
> >>>>>>>>>
> >>>>>>>>> Fixed.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> 3. Overview of newly added methods/interfaces
> >>>>>>>>>
> >>>>>>>>> Done in `Proposed Changes` section.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
> >>>>>>>>> `withChain()` and `withConsumer()` is the
> >>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
> >>>>>>>>> be given to the `Consumer`.
> >>>>>>>>>
> >>>>>>>>> Yes, I agree!
> >>>>>>>>>
> >>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
> >>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> >>>>>>>>> implementation would be to not add the "branch" to the result map if
> >>>>>>>>> `withConsumer` is used?
> >>>>>>>>>
> >>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
> >>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
> >>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> >>>>>>>>> stream in the Map, one simply does not extract it from there :-)
> >>>>>>>>>
> >>>>>>>>> In the current version of KIP it is assumed that the returned map
> >>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
> >>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
> >>>>>>>>> map is the user's responsibility.
> >>>>>>>>>
> >>>>>>>>> What seems to me to be an issue is introducing exclusions to this
> >>>>>>>>> general rule, like 'swallowing' some streams by provided
> >>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
> >>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
> >>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
> >>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
> >>>>>>>>> it as it is?
> >>>>>>>>>
> >>>>>>>>> ----
> >>>>>>>>>
> >>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
> >>>>>>>>>
> >>>>>>>>> Looking forward for your feedback again!
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>>
> >>>>>>>>> Ivan.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
> >>>>>>>>>> Thanks for updating the KIP!
> >>>>>>>>>>
> >>>>>>>>>> I also have some minor comment:
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> >>>>>>>>>>
> >>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> >>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> >>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
> >>>>>>>>>> so we just keep them.)
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> (2) Quote:
> >>>>>>>>>>
> >>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
> >>>>>>>>>>> parameterless alternatives.
> >>>>>>>>>>
> >>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
> >>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> >>>>>>>>>> as `branch()` would not be "parameterless".
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
> >>>>>>>>>> newly added and deprecated methods/classes (cf.
> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> >>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
> >>>>>>>>>> 4th comment:
> >>>>>>>>>>
> >>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
> >>>>>>>>>>> chain and the result map OR using just the sink.
> >>>>>>>>>>
> >>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
> >>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
> >>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
> >>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> >>>>>>>>>>
> >>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
> >>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
> >>>>>>>>>> it in the JavaDocs, this might be fine?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> (5) Reply to John's comments:
> >>>>>>>>>>
> >>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> >>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
> >>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
> >>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
> >>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
> >>>>>>>>>>
> >>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
> >>>>>>>>>> node", ie., writing the KStream to a topic.
> >>>>>>>>>>
> >>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
> >>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>
> >>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> >>>>>>>>>>> but I had a few last comments.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> John
> >>>>>>>>>>>
> >>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> >>>>>>>>>>>> Hello everyone,
> >>>>>>>>>>>>
> >>>>>>>>>>>> will someone please take a look at the reworked KIP?
> >>>>>>>>>>>>
> >>>>>>>>>>>> I believe that now it follows design principles and takes into account
> >>>>>>>>>>>> all the arguments discussed here.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Ivan
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>>>>>>>>>> Hi,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
> >>>>>>>>>>>>> rewritten the KIP, see
> >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> This version includes all the previous discussion results and follows
> >>>>>>>>>>>>> the design principles, with one exception.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The exception is
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
> >>>>>>>>>>>>> here it is justified.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
> >>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
> >>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
> >>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> >>>>>>>>>>>>> as it
> >>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
> >>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
> >>>>>>>>>>>>>> build on
> >>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> >>>>>>>>>>>>>> want to follow:
> >>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -- might be worth to checkout.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>> Hi everyone!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
> >>>>>>>>>>>>>>> June
> >>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
> >>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
> >>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
> >>>>>>>>>>>>>>> propose other things before this one is finalized.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
> >>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
> >>>>>>>>>>>>>>> branches, I worked around it this way:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>>>>>>>>>           .branch(....)
> >>>>>>>>>>>>>>>           .defaultBranch(result::set)
> >>>>>>>>>>>>>>>           .onTopOf(someStream);
> >>>>>>>>>>>>>>> result.get()...
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
> >>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>         -> KBranchedStream
> >>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>         -> KBranchedStream
> >>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
> >>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
> >>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
> >>>>>>>>>>>>>>> Or,
> >>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
> >>>>>>>>>>>>>>> finally,
> >>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
> >>>>>>>>>>>>>>> original scope.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> >>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
> >>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
> >>>>>>>>>>>>>>> familiar with Streams API design principles than me.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>> at any point.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> >>>>>>>>>>>>>>>> do so.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
> >>>>>>>>>>>>>>>>> to mix
> >>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> >>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> >>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> >>>>>>>>>>>>>>>>>> patterns
> >>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
> >>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>> totally sense.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> >>>>>>>>>>>>>>>>>> `Map` only
> >>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
> >>>>>>>>>>>>>>>>>> all of
> >>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>>>>>>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> >>>>>>>>>>>>>>>>>> `Named` is
> >>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> >>>>>>>>>>>>>>>>>> counter
> >>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
> >>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
> >>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>>>>>>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
> >>>>>>>>>>>>>>>>>> misses to
> >>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> >>>>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> >>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>>>>>>>>>>>>>> specifying a
> >>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
> >>>>>>>>>>>>>>>>>> hence
> >>>>>>>>>>>>>>>>>> would not include it in a configuration object.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>            withChain(...);
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> >>>>>>>>>>>>>>>>>> does not
> >>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> >>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
> >>>>>>>>>>>>>>>>>> make sense
> >>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> >>>>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
> >>>>>>>>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
> >>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> >>>>>>>>>>>>>>>>>> object
> >>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
> >>>>>>>>>>>>>>>>>> others,
> >>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
> >>>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
> >>>>>>>>>>>>>>>>>> methods that
> >>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
> >>>>>>>>>>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> >>>>>>>>>>>>>>>>>> two main
> >>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> >>>>>>>>>>>>>>>>>>> satisfy
> >>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
> >>>>>>>>>>>>>>>>>>> solves
> >>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> >>>>>>>>>>>>>>>>>>> to add
> >>>>>>>>>>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
> >>>>>>>>>>>>>>>>>>> Otherwise,
> >>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> >>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
> >>>>>>>>>>>>>>>>>>> names for
> >>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
> >>>>>>>>>>>>>>>>>>> logic, it
> >>>>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
> >>>>>>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> >>>>>>>>>>>>>>>>>>> processing onto
> >>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> >>>>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
> >>>>>>>>>>>>>>>>>>> proposal, we
> >>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
> >>>>>>>>>>>>>>>>>>> name,
> >>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           stream.split(Named.withName("mysplit")) //creates node
> >>>>>>>>>>>>>>>>>>> "mysplit"
> >>>>>>>>>>>>>>>>>>>                      .branch(..., ..., "abranch") // creates node
> >>>>>>>>>>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>>>>>>>>>                      .defaultBranch(...) // creates node
> >>>>>>>>>>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> >>>>>>>>>>>>>>>>>>> debate
> >>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
> >>>>>>>>>>>>>>>>>>> general,
> >>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>            operator(function, config_object?) OR
> >>>>>>>>>>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
> >>>>>>>>>>>>>>>>>>> variant.
> >>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>          operator(function, function, string)
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> where the string is the name.
> >>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
> >>>>>>>>>>>>>>>>>>> specified
> >>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
> >>>>>>>>>>>>>>>>>>> arguments
> >>>>>>>>>>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>           interface BranchConfig extends NamedOperation {
> >>>>>>>>>>>>>>>>>>>            withPredicate(...);
> >>>>>>>>>>>>>>>>>>>            withChain(...);
> >>>>>>>>>>>>>>>>>>>            withName(...);
> >>>>>>>>>>>>>>>>>>>          }
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> >>>>>>>>>>>>>>>>>>> more like
> >>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
> >>>>>>>>>>>>>>>>>>> makes us
> >>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> >>>>>>>>>>>>>>>>>>> purely
> >>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
> >>>>>>>>>>>>>>>>>>> overloads
> >>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> WDYT?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> >>>>>>>>>>>>>>>>>>>> view.
> >>>>>>>>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> >>>>>>>>>>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>          -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> >>>>>>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>>>>>          -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>>>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> >>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
> >>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> >>>>>>>>>>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
> >>>>>>>>>>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> >>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> >>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
> >>>>>>>>>>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> >>>>>>>>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> >>>>>>>>>>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> >>>>>>>>>>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>>>>>>>>>> language
> >>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> >>>>>>>>>>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
> >>>>>>>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> >>>>>>>>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> >>>>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> >>>>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> >>>>>>>>>>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            Ivan, I’ll definitely forfeit my point on the
> >>>>>>>>>>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            branch(predicate, consumer) solution, I don’t see
> >>>>>>>>>>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>>>>>>>>>            for the dynamic case.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            IMO the one trade off to consider at this point is the
> >>>>>>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>            question. I don’t know if I totally agree that “we
> >>>>>>>>>>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>>>>>>>            in the same scope” since merging the branches back
> >>>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>            seems like a perfectly plausible use case that can
> >>>>>>>>>>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>>>>>>>>>            when the branched streams are in the same scope.
> >>>>>>>>>>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>>>>>            for the reasons Ivan listed, I think it is overall the
> >>>>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>>>            solution - working around the scope thing is easy
> >>>>>>>>>>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>>>>>>>>>            to.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>            <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > Hello everyone, thank you all for joining the
> >>>>>>>>>>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > Well, I don't think the idea of named branches,
> >>>>>>>>>>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>>>>>>>>>            LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>>>>>>>            matters) or `branch` method  taking name and Consumer
> >>>>>>>>>>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>>>>>>>>>            advantages than drawbacks.
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > In my opinion, the only real positive outcome from
> >>>>>>>>>>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>>>>>>>>>            proposal is that all the returned branches are in
> >>>>>>>>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>            But 1) we rarely need them in the same scope 2)
> >>>>>>>>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>>>>>>>            workaround for the scope problem, described in the
> >>>>>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > 'Inlining the complex logic' is not a problem,
> >>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>>>>>>>>>            method references instead of lambdas. In real world
> >>>>>>>>>>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>>>>>>>>>            tend to split the complex logic to methods anyway,
> >>>>>>>>>>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>>>>>>>>>            going to be clean.
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > The drawbacks are strong. The cohesion between
> >>>>>>>>>>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            handlers is lost. We have to define predicates in one
> >>>>>>>>>>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>>>>>>>>>            handlers in another. This opens the door for bugs:
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > - what if we forget to define a handler for a
> >>>>>>>>>>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>>>>>>>>>            a handler?
> >>>>>>>>>>>>>>>>>>>>>>            > - what if we misspell a name?
> >>>>>>>>>>>>>>>>>>>>>>            > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > What Michael propose would have been totally OK
> >>>>>>>>>>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>            writing the API in Lua, Ruby or Python. In those
> >>>>>>>>>>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>>>>>>>>>            "dynamic naming" approach would have looked most
> >>>>>>>>>>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            beautiful. But in Java we expect all the problems
> >>>>>>>>>>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>>>>>>>>>            identifiers to be eliminated in compile time.
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > And if we do, what advantage are we supposed to get
> >>>>>>>>>>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>>>>>>>>>            all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>>>>>>>>>            point?
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > ---
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > Earlier in this discussion John Roesler also
> >>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>            without "start branching" operator, and later Paul
> >>>>>>>>>>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>>>>>>>>>            the case when we have to add a dynamic number of
> >>>>>>>>>>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>>>>>>>>>            current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>>>>>>>>>            me address both comments here.
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > 1) "Start branching" operator (I think that
> >>>>>>>>>>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>>>>>>>>>            for it indeed) is critical when we need to do a
> >>>>>>>>>>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>>>>>>>>>            see example below.
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > 2) No, dynamic branching in current KIP is not
> >>>>>>>>>>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>>>            Imagine a real-world scenario when you need one
> >>>>>>>>>>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>>>>>>>>>            value (say, RecordType). You can have something
> >>>>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > /*John:if we had to start with stream.branch(...)
> >>>>>>>>>>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>>>>>>>>>            have been much messier.*/
> >>>>>>>>>>>>>>>>>>>>>>            > KBranchedStream branched = stream.split();
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > /*Not clumsy at all :-)*/
> >>>>>>>>>>>>>>>>>>>>>>            > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>>>>>>>>>>>>>            >             branched = branched.branch((k, v) ->
> >>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>>>>>>>>>            recordType,
> >>>>>>>>>>>>>>>>>>>>>>            >                     recordType::processRecords);
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>            > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >> I also agree with Michael's observation about
> >>>>>>>>>>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>>>>>>>>>            >> current `branch()` implementation.
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>>>>>>>>>            thinking
> >>>>>>>>>>>>>>>>>>>>>>            >> was more aligned with Paul's proposal to just
> >>>>>>>>>>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>>>>>>>>>            >> `branch()` statement and return a
> >>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >> It makes the code easier to read, and also make the
> >>>>>>>>>>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>>>>>>>>>            >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>>>>>>>>>            >> An open question is the case for which no
> >>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>            >> specified. Atm, `split()` and `branch()` would
> >>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>            `BranchedKStream`
> >>>>>>>>>>>>>>>>>>>>>>            >> and the call to `defaultBranch()` that returns the
> >>>>>>>>>>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>>>>>>>>>            >> (what is not the case atm). Or is this actually
> >>>>>>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>>>> real
> >>>>>>>>>>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>>>>>>>>>            >> because users can just ignore the branch
> >>>>>>>>>>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>>>>>>>>>            `defaultBranch()`
> >>>>>>>>>>>>>>>>>>>>>>            >> in the result `Map` ?
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >> About "inlining": So far, it seems to be a
> >>>>>>>>>>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>>>>>>>>>            >> preference. I can see arguments for both, but no
> >>>>>>>>>>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>>>>>>>>>            argument" yet
> >>>>>>>>>>>>>>>>>>>>>>            >> that clearly make the case for one or the other.
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>            >>
> >>>>>>>>>>>>>>>>>>>>>>            >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>> Perhaps inlining is the wrong terminology. It
> >>>>>>>>>>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>>>            that a lambda with the full downstream topology be
> >>>>>>>>>>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>>>>>>>>>            it can be a method reference as with Ivan’s original
> >>>>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>>>            The advantage of putting the predicate and its
> >>>>>>>>>>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>>            (Consumer) together in branch() is that they are
> >>>>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>>>>>>>>>            to each other.
> >>>>>>>>>>>>>>>>>>>>>>            >>>
> >>>>>>>>>>>>>>>>>>>>>>            >>> Ultimately the downstream code has to live
> >>>>>>>>>>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>>>>>>>>>            branch trees will be hard to read regardless.
> >>>>>>>>>>>>>>>>>>>>>>            >>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>>>>>>>>>            <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>            <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>> I'm less enthusiastic about inlining the
> >>>>>>>>>>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>>>>>>>>>            downstream
> >>>>>>>>>>>>>>>>>>>>>>            >>>> functionality. Programs that have deep branch
> >>>>>>>>>>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>            quickly become
> >>>>>>>>>>>>>>>>>>>>>>            >>>> harder to read as a single unit.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>>            <pgwhalen@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Also +1 on the issues/goals as Michael
> >>>>>>>>>>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>>            that sets a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> great framework for the discussion.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Regarding the SortedMap solution, my
> >>>>>>>>>>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>>>>>>>            current
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>>>>>>>>>            decisions) is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> roughly this:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Obviously some ordering is necessary, since
> >>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>            construct
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> doesn't work without it, but this solution seems
> >>>>>>>>>>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>            provides as much
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> associativity as the SortedMap solution,
> >>>>>>>>>>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>            call
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> directly associates the "conditional" with
> >>>>>>>>>>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>>>>>>>>>            The value it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> provides over the KIP solution is the
> >>>>>>>>>>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>>>>>>>            the same
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> scope.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> The KIP solution is less "dynamic" than the
> >>>>>>>>>>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>            in the sense
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> that it is slightly clumsier to add a dynamic
> >>>>>>>>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>>>>>            branches, but it is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>>>>>>>>>            the "static"
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> case anyway, and should make it simple and
> >>>>>>>>>>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>>>>>>>>>            fluently declare and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>>>>>>>>>            ignore a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>>>>>>>>>            SortedMap
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> solution on top of it.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> I could also see a middle ground where
> >>>>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>>>>>>>>>            SortedMap being
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> taken in, branch() takes a name and not a
> >>>>>>>>>>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>>>>>>>>>            like this:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Pros for that solution:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> - no double brace initialization, hopefully
> >>>>>>>>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>            readable than
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Cons
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> - downstream branch logic cannot be specified
> >>>>>>>>>>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>            makes it harder
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> to read top to bottom (like existing API and
> >>>>>>>>>>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>>>>>>>>>            unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>>>>>>>>>            existing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> (KBranchedStreams could even work *both* ways
> >>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>>>>>>>>>            that's overdoing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> it).
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Overall I'm curious how important it is to be
> >>>>>>>>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>>>>>>>            access the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> branched KStream in the same scope as the
> >>>>>>>>>>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>            possible that it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> doesn't need to be handled directly by the
> >>>>>>>>>>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>            left up to the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> >>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>>>>>>>>>            <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> I'd like to +1 what Michael said about the
> >>>>>>>>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>            existing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> branch
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> method, I agree with what he's outlined and
> >>>>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>>>>>>>>>            proceed by
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> trying to alleviate these problems.
> >>>>>>>>>>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>            important to be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> able to cleanly access the individual
> >>>>>>>>>>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> name->stream), which I thought was the original
> >>>>>>>>>>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>>>>>>>>>            this KIP.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> That said, I don't think we should so easily
> >>>>>>>>>>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>            double brace
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> anti-pattern or force ours users into it if
> >>>>>>>>>>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> avoid...just
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> my two cents.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> Sophie
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> >>>>>>>>>>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>>>            <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> I’d like to propose a different way of
> >>>>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>>>>>>>>>            To me,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> there
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> are three problems with the existing branch
> >>>>>>>>>>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> warnings.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 2. The way in which you use the stream
> >>>>>>>>>>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>>>>>>>>>            positionally coupled
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 3. It is brittle to extend existing branch
> >>>>>>>>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>            additional code
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> paths.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Using associative constructs instead of
> >>>>>>>>>>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>>>>>>>>>            constructs
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> be a stronger approach. Consider a
> >>>>>>>>>>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>>>            looks like
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>>>>>>>>>            Predicate<?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> super K,? super V>>);
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Branches are given names in a map, and as a
> >>>>>>>>>>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>>>>>>>>>            returns a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> mapping of names to streams. The ordering
> >>>>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> maintained
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> because it’s a sorted map. Insert order
> >>>>>>>>>>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>>>>>>>>>> the order
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> evaluation.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> This solves problem 1 because there are no
> >>>>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>>>>>>>>>            solves
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> problem
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 2 because you no longer lean on ordering to
> >>>>>>>>>>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>>>>>>>>>            branch you’re
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> interested in. It solves problem 3 because
> >>>>>>>>>>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>>>>>>>            another
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> conditional by simply attaching another
> >>>>>>>>>>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>>>>>>>>>            structure, rather
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> messing with the existing indices.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>>>>>>>>>            historically
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> awkward in Java. I know it’s an
> >>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>>>>>>>>>            voluminously, but
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> double brace initialization would clean up the
> >>>>>>>>>>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>>>>>>>>>            <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Thanks for the update.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> FWIW, I agree with Matthias that the current
> >>>>>>>>>>>>>>>>>>>>>> "start
> >>>>>>>>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> operator
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> confusing when named the same way as the
> >>>>>>>>>>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>>>>>>>>>            "Split"
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> like a good name. Alternatively, we can do
> >>>>>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> branching"
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> operator at all, and just do:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Tentatively, I think that this branching
> >>>>>>>>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> terminal.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> way, we don't create ambiguity about how
> >>>>>>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>>>>>>>>>            is, `branch`
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>>>>>>>>>            `void`, to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> enforce that it comes last, and that there
> >>>>>>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>>>            definition of
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>>>>>>>>>            there's no
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> default,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>>>>>>>>>            record
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> falls
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> though with no default.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Thanks for updating the KIP and your
> >>>>>>>>>>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> this is to make the name similar to
> >>>>>>>>>>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> The intend was to avoid name duplication.
> >>>>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>>>>>>>>>            should
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> _not_
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> be an array.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> The current proposal is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> IMHO, this reads a little odd, because
> >>>>>>>>>>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>>>>>>>>>            `branch()` does
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> not
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> take any parameters and has different
> >>>>>>>>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>>>>>> than the
> >>>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> hidden
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that the first call is `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>>>>>> the others
> >>>>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Because I suggested to rename
> >>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>>>>>>>>>            I though
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> might be better to also rename
> >>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> naming
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> overlap that seems to be confusing. The
> >>>>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> me:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Maybe there is a better alternative to
> >>>>>>>>>>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>>>>>>>>>            avoid
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> naming overlap.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> cannot
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> a method with such name :-)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Bummer. Didn't consider this. Maybe we
> >>>>>>>>>>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>>>>>>>>>            with a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> short
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> name?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Can you add the interface
> >>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>>            with all
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> methods? It will be part of public API and
> >>>>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>>>            contained in
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>>>>>>>>>>>>>            `KBranchedStream#get(int
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> index)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>>>>>>>>>>>>>            branched-KStreams. Would
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> nice to get your feedback about it. It
> >>>>>>>>>>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>>>>>            that users
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> would need to write custom utility code
> >>>>>>>>>>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>>>>>>>>>            access them.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> We
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>>>>>>>>>            to get
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> I have updated the KIP-418 according to
> >>>>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> I can see your point: this is to make
> >>>>>>>>>>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> that also returns an array, right? But
> >>>>>>>>>>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>>>>>>>>>            loss of
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> compatibility? We can have overloaded
> >>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>>>>>>>>>            without
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> affecting
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> the existing code. Maybe the old
> >>>>>>>>>>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> should
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> deprecated, but this is a subject for
> >>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Totally agree with 'addBranch->branch'
> >>>>>>>>>>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> however, a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> reserved word, so unfortunately we
> >>>>>>>>>>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>>>            with such
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> name
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> :-)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> defaultBranch() does take an
> >>>>>>>>>>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> think
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> is not required?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Absolutely! I think that was just
> >>>>>>>>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>>>>>>>>>            something.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> please revise the new version of the KIP
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Thanks for driving the discussion of
> >>>>>>>>>>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> everybody
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> agrees that the current branch() method
> >>>>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> optimal.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> I had a quick look into the PR and I
> >>>>>>>>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>>>>>            proposal.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> There
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> are some minor things we need to
> >>>>>>>>>>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>            recommend the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> following renaming:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>>>            BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> It's just a suggestion to get slightly
> >>>>>>>>>>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> In the current PR, defaultBranch() does
> >>>>>>>>>>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>>>>>>>>>            `Predicate` as
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>>>>>>>>>            accepted and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Ie, we should add overloads that
> >>>>>>>>>>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>>>>>            parameter.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> For the issue that the created
> >>>>>>>>>>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>>>>>>>>>            different
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> scopes:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> could we extend `KBranchedStream` with a
> >>>>>>>>>>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>>>>>>>>>            index)` method
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> returns the corresponding "branched"
> >>>>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>>>>>>>>>            object?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> Maybe,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> second argument of `addBranch()` should
> >>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> a `Function<KStream,KStream>` and
> >>>>>>>>>>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>>>            whatever
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Finally, I would also suggest to update
> >>>>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>            current
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>            makes sense
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> revise the KIP and continue the
> >>>>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>>>>>>>>>            we'll
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> need
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> whether
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> could be adopted.  It would be great
> >>>>>>>>>>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>>>>>>>>>            think this
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> idea overall.  I'm not sure if that
> >>>>>>>>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>>>>>>>>>            starting a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> vote,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> or if
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> there is generally some indication of
> >>>>>>>>>>>>>>>>>>>>>> interest
> >>>>>>>>>>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> That being said, I'll continue the
> >>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>>>>>>>>>            assuming
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> forward the solution of "stream.branch()
> >>>>>>>>>>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> KBranchedStream",
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> favor
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> accomplish
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>>>>>>>>>            similar
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> just need to be sure we're not making
> >>>>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> impossible/difficult
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> just a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> little sloppy overall in terms of
> >>>>>>>>>>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> particular,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> passing in the "predicates" and
> >>>>>>>>>>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> modified
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>>>>>>>>>>>>>            KStreamLazyBranch is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> I read your code carefully and now I
> >>>>>>>>>>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>>>>>>>>>            convinced: your
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> looks better and should work. We just
> >>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> crucial
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> that KStream consumers are invoked as
> >>>>>>>>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>>>>>            And then
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> What shall we do now? I should
> >>>>>>>>>>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>>>>>>>>>            resume the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Why are you telling that your PR
> >>>>>>>>>>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> point
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> we go in this direction'? To me it
> >>>>>>>>>>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>>>>>            starting
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> as a novice in this project I might
> >>>>>>>>>>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> details.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> >>>>>>>>>>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> invoked
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> they’re
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> added, not during
> >>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>>>>>>>>>            still
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> ought
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> able to call couponIssuer.coupons()
> >>>>>>>>>>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            depend on
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> The issue I mean to point out is
> >>>>>>>>>>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>>>>>>>>>            access
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> streams in the same scope as the
> >>>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>>>>>>>>>            is, not
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> solutions.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> It
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> [Also, great to hear additional
> >>>>>>>>>>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>>>>>>>>>            excited
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> hear
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> The idea to postpone the wiring of
> >>>>>>>>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>>>>>>>>>> me at
> >>>>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> glance,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>>>>>>>>>            same
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> scope
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> other.  That is, if we wanted to merge
> >>>>>>>>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> again
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> don't see
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> You just took the words right out
> >>>>>>>>>>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>>>>>>>>>            just
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> going
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Consider the example from Bill's
> >>>>>>>>>>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>>>>>>>>>            we need
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> identify
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> customers who have bought coffee and
> >>>>>>>>>>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>>>>>>>>>            in the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> electronics
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> This is the code I usually write under
> >>>>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>>>            circumstances
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> using
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>       return
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>       /*In the real world the code
> >>>>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>>>            complex, so
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> creation of
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>>>>>>>>>            order to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> separate
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> >>>>>>>>>>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>>>>>>>>>            everything
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> later,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Does this make sense?  In order to
> >>>>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>>>            initialize the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> we need the terminal operation to be
> >>>>>>>>>>>>>>>>>>>>>> called
> >>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>>>>>>>>>            essentially
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> KIP I was going to write here. I have
> >>>>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>>>>>>>>>            based on
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> my
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> experience,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> so I will join the discussion on KIP-401
> >>>>>>>>>>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>>>>>>>>>            fluent
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> API
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>>>>>>>>>>>>>            (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> compatibility
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    there aren't any direct ones.
> >>>>>>>>>>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>>>>>>>>>            that Java
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> smart
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    distinguish between a
> >>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>>>>>>>>>            returning one
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> thing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    with no arguments returning
> >>>>>>>>>>>>>>>>>>>>>> another
> >>>>>>>>>>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> >>>>>>>>>>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>            need
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> it.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>>>>>>>>>            shares
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> >>>>>>>>>>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            branching.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    pretty in its current form, but I
> >>>>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>>>            demonstrates
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>>>>>>>>>>>>>> request should
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> final
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> even a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> starting point if we go in this
> >>>>>>>>>>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>>>>>>>>>            wanted to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> I will say though, that I'm not
> >>>>>>>>>>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>>>            solution
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> deprecated in favor of this, which
> >>>>>>>>>>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> suggested
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> was a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>            streams
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>>>>>>>>>            is, if we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> wanted
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> them back together again I don't
> >>>>>>>>>>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>>>            that.  The
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> either
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>>>>>>>>>            table.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> point.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> First, it seems that it's
> >>>>>>>>>>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            branch API
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> needs
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> There are two potential ways to
> >>>>>>>>>>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>>>>>>>>>            returns
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> its
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> >>>>>>>>>>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>>>>>>>>>> The code
> >>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>        ��   >>>>> make
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> sense
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> all the necessary ingredients are
> >>>>>>>>>>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>            instance
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> >>>>>>>>>>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> defined.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> >>>>>>>>>>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>>>>>>>>>            miss the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> fact
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that one
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> of the terminal methods should be
> >>>>>>>>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>>>>>>>>>            methods
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> I see your point when you are
> >>>>>>>>>>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> >>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>>            implemented the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> easy
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> >>>>>>>>>>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> >>>>>>>>>>>>>>>>>>>>>> method that
> >>>>>>>>>>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> >>>>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>>> the only
> >>>>>>>>>>>>>>>>>>>>> option
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> besides
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>>>>>>>>>            want to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> silently
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> >>>>>>>>>>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>>>>>>>>>            predicate. 2)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Throwing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>            looks
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> like a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> bad
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> >>>>>>>>>>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>>>>>>>>>            emit a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> >>>>>>>>>>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> `default`
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>            a clear
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> >>>>>>>>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> compiled
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> run?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> >>>>>>>>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>>>            compile if
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> used
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>>>>>>>>>            method chain
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> starting
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> >>>>>>>>>>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>>>            between
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> instantly
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> >>>>>>>>>>>>>>>>>>>>>> project
> >>>>>>>>>>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>>>>>>>>>            compilation
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>>>>>>>>>            required.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> >>>>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> defaultBranch
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>>>>>>>>>            just as
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> think it creates an
> >>>>>>>>>>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> user
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> default
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>>>>>>>            which allows
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>>>>>>>>>            to be
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> well
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>>>            a clear
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> "build
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> step"
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> allow users to do other
> >>>>>>>>>>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>>>>>>>            stream.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>> way all
> >>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> do -
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> want to process off the original
> >>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> times,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> >>>>>>>>>>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>>>>>>>>>            operations
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> >>>>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>>>>>            always need
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> don't
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> >>>>>>>>>>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>>>            so we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> can
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> more with the original branch
> >>>>>>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> >>>>>>>>>>>>>>>>>>>>>> need of
> >>>>>>>>>>>>>>>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> object
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> here
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>>>>>>>>>            flow,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> so
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> >>>>>>>>>>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>            API, but I
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> find
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>>>>>>>            contrasts the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> fluency
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
> >>>>>>>>>>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>>>>>>>            just call
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> method on
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> >>>>>>>>>>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>>>>>>>>>            cases
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> are
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> I think the
> >>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>>>>>>>>>            is very
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> nice
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> >>>>>>>>>>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>>            how we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>>>>>>>>>            terminated by
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>>>>>>>>>            incompatible with
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> >>>>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>            different
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> name,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> >>>>>>>>>>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> >>>>>>>>>>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>            KIP?  It
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>>>            while also
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> allowing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>>>>>>>>>            KBranchedStreams
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>>>>>>>>>            ks){
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>>>>>>>>>            String> ks){
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>            this::handleFirstCase)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>>>            this::handleSecondCase)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> takes a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Consumer
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>>            example in
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>>>>>>>>>> how would
> >>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> handle
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>>>>>>>>>            we had
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> 418.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> Please
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>>>>>>>>>            feedback :)
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>>>>>>>>>            >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>            >
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Attachments:
> >>>>>>>> * signature.asc
> >>>>>>
> >>>>>>
> >>>>
> >>>>
> >>>>
> >>
> >>
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
John,

---------------------

 > I'm sorry things have been dragging out a little, but I have the 
sense we're very close to the end of this discussion, which is exciting.

We are certainly moving forward!  And I'm not in a hurry at all. As I 
told you before in my projects I'm using Spring Kafka's 
KafkaStreamBrancher -- the implementation of the first, rejected version 
of this KIP. It's inferior to what we are discussing here, but it does 
its work. So it's worth to design this KIP really, really well. And by 
the way, from this discussion I'm learning the good API designing 
process. For me it has a value per se :-))

----------------------

 >>   > I'd wonder whether we need the non-static builders (like withChain).
 >>   > Do they provide any benefit over just using the right static 
factory?
 >
 > I don't have a strong feeling, either. It seems nice to offer a better
 > type inference experience than what we get with Materialized, by
 > offering the static method that takes both name and chain.
 > Given that, there doesn't seem to be a good reason to also offer the
 > non-static builder-style methods, so I guess I'd prefer to drop them.

I agree again! From a recent discussion on Twitter 
(https://twitter.com/inponomarev/status/1265220044394545153) I found out 
an interesting fact about type inference rules in Java. Funny thing is 
that although we need to explicitly set types in a chain like this

foo.branch(..., Branched.<...,...>named("foo").withChain(...));

(otherwise it won't compile), the composition of static method calls 
works just fine, all the types are being calculated correctly:

foo.branch(..., Branched.withChain(Branched.named("foo"), ...));

As I was told, for type inference there is difference between qualifiers 
and arguments, 'you go up if you are argument but stop if you are 
qualifier'. And it also seems that we should not bet on any future 
improvements in Java type inference here.

So,

1) I think we that in this KIP we should provide three static methods 
only: `as(String)`, `with(Function)`, and `with(Function, String)`, and 
drop any non-static ones.

2) If anything else will be ever needed, we can easily add anything. 
Maybe this can be done in a process of refinement of all the parameter 
classes.

----------------------------

 > we may as well hope for the best, and propose the "fully
 > covariant" definition for now.

Understood and agreed! I will edit the KIP.

----------------------------

 >> Good question, I already thought about it and rejected the idea....
 > (I cut off your quote; the rest is in the chain below)
 >  "Worst" case scenario: someone
 > else will wish the return type is something different, and we'll go
 > through a painless deprecation transition to change it later.

Of course, we cant' predict all the ways people are going to use it. 
 From my own humble experience with Kafka Streams, the worst scenario is 
unlikely. Split is split, transform is transform, too much flexibility 
is often evil.

------------------------

So it seems that we are close to the consensus. Two things to be altered 
in the current version of KIP:

* list of Branched methods, drop non-static methods.
* 'fully covariant' definition of `chained`.

Any ideas / questions / objections?

Regards,

Ivan




27.05.2020 7:03, John Roesler пишет:
> Thanks for the reply, Ivan,
> 
> I'm sorry things have been dragging out a little, but I have the sense
> we're very close to the end of this discussion, which is exciting.
> 
>>   > I'd wonder whether we need the non-static builders (like withChain).
>>   > Do they provide any benefit over just using the right static factory?
>>
>> I don't have a strong opinion here. I think it's just a matter of taste.
>> But, if we like to use Occam's razor, then yes, non-static builders can
>> be omitted, I agree!
> 
> I don't have a strong feeling, either. It seems nice to offer a better
> type inference experience than what we get with Materialized, by
> offering the static method that takes both name and chain.
> 
> Given that, there doesn't seem to be a good reason to also offer the
> non-static builder-style methods, so I guess I'd prefer to drop them.
> 
> I'll defer to Matthias, if he has a chance to consider whether it's more
> valuable to stick with the existing pattern or break the pattern to offer
> a better experience.
> 
>>   > You might as well propose the “ideal” API in the KIP, which is the
>>   > covariant typed function
>>
>> I didn't quite get it. Do I get you right that you propose the
>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>> then just see if there are any obstacles/pitfalls during implementation
>> and unit testing?
> 
> Yep! I know it's a bit sloppy, but my experience has been that we just
> won't know what works until we really try it, and try it in several different
> ways. Still, we may as well hope for the best, and propose the "fully
> covariant" definition for now.
> 
>>   > Is it necessary to restrict the result key and value types to be the
>>   > same as the inputs?
>>
>> Good question, I already thought about it and rejected the idea....
> (I cut off your quote; the rest is in the chain below)
> 
> That's fair! It's your KIP, after all. I think I might have made a different
> call here, but I think this choice is fine. "Worst" case scenario: someone
> else will wish the return type is something different, and we'll go
> through a painless deprecation transition to change it later. Thanks
> to the clean design of your API, this doesn't seem to bad. And, of
> course, you've actually been using similar functionality already, so it
> seems we should trust your intuition.
> 
> Thanks,
> -John
> 
> 
> On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
>> Hi John,
>>
>>   > I'd wonder whether we need the non-static builders (like withChain).
>> Do they provide any benefit over just using the right static factory?
>>
>> I don't have a strong opinion here. I think it's just a matter of taste.
>> But, if we like to use Occam's razor, then yes, non-static builders can
>> be omitted, I agree!
>>
>>   > You might as well propose the “ideal” API in the KIP, which is the
>> covariant typed function
>>
>> I didn't quite get it. Do I get you right that you propose the
>> following: 1) accept the 'fully covariant' definition in the KIP, 2)
>> then just see if there are any obstacles/pitfalls during implementation
>> and unit testing?
>>
>>   > Is it necessary to restrict the result key and value types to be the
>> same as the inputs?
>>
>> Good question, I already thought about it and rejected the idea.
>>
>> Look, if we want to keep `withChain`'s function optional, then we must
>> keep the result key and value types the same. Because for now, the
>> default value for the 'chain function' is Function.identity().
>>
>> Of course, we can make the 'chain function' required. But I think this
>> is not what `split` method is for. `split` is for splitting, not
>> transforming, and `chainFunction` in most of the cases should be either
>> a consumer or the identity function.
>>
>> Regards,
>>
>> Ivan
>>
>>
>> 24.05.2020 17:15, John Roesler пишет:
>>> Thanks for the reply, Ivan.
>>>
>>> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
>>>
>>> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
>>>
>>> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
>>>
>>> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
>>>
>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
>>>
>>> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
>>>
>>> Thanks,
>>> John
>>>
>>> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>>>> Hello John,
>>>>
>>>>
>>>> 1.
>>>> ---------------------------------------------
>>>>
>>>>    > Perhaps it would be better to stick with "as" for now
>>>>    > and just file a Jira to switch them all at the same time [for
>>>> compatibility with Kotlin]
>>>>
>>>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
>>>> standard workaround
>>>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>>>> So actually this should be a very low priority issue, if an issue at
>>>> all.
>>>>
>>>>    > I don't understand how your new proposed
>>>>    > methods would work any differently than the ones you already
>>>>    > had proposed in the KIP. It seems like you'd still have to provide
>>>>    > the generic type parameters on the first static factory call. Can you
>>>>    > explain how your new interface proposal differs from the existing KIP?
>>>>
>>>> In the KIP, I didn't clarify what methods should be static. Now I
>>>> propose the following methods:
>>>>
>>>> non-static: withChain(Function), withName(String).
>>>>
>>>> static: as(String), with(Function), with(Function, String).
>>>>
>>>> The overloaded `with` version that provides both Function and name can
>>>> be used without causing type inference problem!!
>>>>
>>>> 2.
>>>> ----------------------------
>>>>
>>>>    > Regarding making the K,V types covariant also, yes, that would indeed
>>>>    > be nice, but I'm not sure it will actually work.
>>>>
>>>> What I'm keeping in mind is the following example: imagine
>>>>
>>>> static KStream<String, Integer> func(KStream<String, Number> s) {
>>>>            return s.mapValues(n -> (Integer) n + 1);
>>>> }
>>>>
>>>> BranchedKStream<String, Number> b =
>>>>        s.split().branch((k, v) -> isInteger(v),
>>>>                   //Won't compile!!
>>>>                   Branched.with(Me::func));
>>>>
>>>> The simple workaround here is to change `func`'s return type from
>>>> KStream<...Integer> to KStream<...Number>.
>>>>
>>>> [On the other hand, we already agreed to remove `withJavaConsumer` from
>>>> `Branched`, so during code migration I will have to modify my functions'
>>>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>>>
>>>>    >  the map you're returning is Map<K,V>, and of course a K is not the
>>>> same as "? extends K", so it doesn't seem compatible.
>>>>
>>>> I think what you actually meant here is that KStream<? extends K, ?
>>>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
>>>> particularly is not a problem, since KStream<? extends K, ? extends V>
>>>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
>>>>
>>>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
>>>> for now it's better to just admit that API is not absolutely perfect and
>>>> accept it as is, that is
>>>>
>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>>
>>>> 21.05.2020 17:59, John Roesler пишет:
>>>>> Hello Ivan,
>>>>>
>>>>> Thanks for the refinement. Actually, I did not know that "as" would
>>>>> clash with a Kotlin operator. Maybe we should depart from convention
>>>>> and just avoid methods named "as" in the future.
>>>>>
>>>>> The convention is that "as(String name)" is used for the static factory
>>>>> method, whereas "withName(String name)" is an instance method
>>>>> inherited from NamedOperation. If you wish to propose to avoid "as"
>>>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
>>>>> although it's somewhat dubious, since all the other configuration
>>>>> classes use "as". Perhaps it would be better to stick with "as" for now
>>>>> and just file a Jira to switch them all at the same time.
>>>>>
>>>>> Re. 3:
>>>>> Regarding the type inference problem, yes, it's a blemish on all of our
>>>>> configuraion objects. The problem is that Java infers the type
>>>>> based on the _first_ method in the chain. While it does consider what
>>>>> the recipient of the method result wants, it only considers the _next_
>>>>> recipient.
>>>>>
>>>>> Thus, if you call as("foo") and immediately assign it to a
>>>>> Branched<String,String> variable, java infers the type correctly. But
>>>>> when the "next recipient" is a chained method call, like "withChain",
>>>>> then the chained method doesn't bound the type (by definition,
>>>>> withChain is defined on Branched<Object, Object>, so Java will take
>>>>> the broadest possible inferece and bind the type to
>>>>> Branched<Object, Object>, at which point, it can't be revised anymore.
>>>>>
>>>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
>>>>> that hard to recursively consider the entire context when inferring the
>>>>> generic type parameters, but this is what we have to work with.
>>>>>
>>>>> To be honest, though, I don't understand how your new proposed
>>>>> methods would work any differently than the ones you already
>>>>> had proposed in the KIP. It seems like you'd still have to provide
>>>>> the generic type parameters on the first static factory call. Can you
>>>>> explain how your new interface proposal differs from the existing KIP?
>>>>>
>>>>> Re. 4:
>>>>> Regarding making the K,V types covariant also, yes, that would indeed
>>>>> be nice, but I'm not sure it will actually work. You might want to give it a
>>>>> try. In the past, we've run into soe truly strange interactions between the
>>>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
>>>>> combination with nested covariant types.
>>>>>
>>>>> Another issue is that the value type of the map you're returning is
>>>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
>>>>> doesn't seem compatible.
>>>>>
>>>>> Thanks again,
>>>>> -John
>>>>>
>>>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>>>> Hi,
>>>>>>
>>>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>>>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>>>>>> simpler'.
>>>>>>
>>>>>> I made some quick API mocking in my IDE and tried to implement examples
>>>>>> from KIP.
>>>>>>
>>>>>> 1. Having to return something from lambda is not a very big deal.
>>>>>>
>>>>>> 2. For a moment I thouht that I won't be able to use method references
>>>>>> for already written stream consumers, but then I realized that I can
>>>>>> just change my methods from returning void to returning the input
>>>>>> parameter and use references to them. Not very convenient, but passable.
>>>>>>
>>>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>>>>>> function returns null, we don't insert it into the resulting map.
>>>>>>
>>>>>> Usually it's better to implement a non-perfect, but workable solution as
>>>>>> a first approximation. And later we can always add to `Branched`
>>>>>> anything we want.
>>>>>>
>>>>>> 3. Do we have any guidelines on how parameter classes like Branched
>>>>>> should be built? First of all, it seems that `as` now is more preferred
>>>>>> than `withName` (although as you probably know it clashes with Kotlin's
>>>>>> `as` operator).
>>>>>>
>>>>>> Then, while trying to mock the APIs, I found out that my Java cannot
>>>>>> infer types in the following construction:
>>>>>>
>>>>>> .branch((key, value) -> value == null,
>>>>>>        Branched.as("foo").withChain(s -> s.mapValues(...)))
>>>>>>
>>>>>>
>>>>>> so I have to write
>>>>>>
>>>>>> .branch((key, value) -> value == null,
>>>>>>        Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>>>>>
>>>>>>
>>>>>> This is not tolerable IMO, so this is the list of `Branched` methods
>>>>>> that I came to (will you please validate it):
>>>>>>
>>>>>> static <K, V> Branched<K, V> as(String name);
>>>>>>
>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>> extends KStream<K, V>> chain);
>>>>>>
>>>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>>>> extends KStream<K, V>> chain, String name);
>>>>>>
>>>>>> //non-static!
>>>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>>>>>> KStream<K, V>> chain);
>>>>>>
>>>>>>
>>>>>> 4. And one more. What do you think, do we need that flexibility:
>>>>>>
>>>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>>>>>
>>>>>> vs.
>>>>>>
>>>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>>>>>> extends K, ? extends V>> chain
>>>>>>
>>>>>> ??
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>>
>>>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>>>> Thanks for this thought, Matthias,
>>>>>>>
>>>>>>> Your idea has a few aspects I find attractive:
>>>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>>>>>
>>>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>>>>>
>>>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>>>>>
>>>>>>> Thanks again for sharing the idea,
>>>>>>> John
>>>>>>>
>>>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>>>> Thanks for updating the KIP!
>>>>>>>>
>>>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>>>>>> its relationship to the returned `Map`.
>>>>>>>>
>>>>>>>> Originally, we discussed two main patterns:
>>>>>>>>
>>>>>>>>      (1) split a stream and return the substreams for futher processing
>>>>>>>>      (2) split a stream and modify the substreams with in-place method chaining
>>>>>>>>
>>>>>>>> To combine both patterns we wanted to allow for
>>>>>>>>
>>>>>>>>       -> split a stream, modify the substreams, and return the _modified_
>>>>>>>> substreams for further processing
>>>>>>>>
>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>>>>>
>>>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>>>>>
>>>>>>>>      - using `withChain` I get the modified sub-stream
>>>>>>>>      - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>>>>>
>>>>>>>> This seems to be quite subtle to me.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>     From my understanding the original idea of `withJavaConsumer` was to
>>>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>>>
>>>>>>>> Branched.withChain(s -> {
>>>>>>>>       s.to();
>>>>>>>>       return null;
>>>>>>>> })
>>>>>>>>
>>>>>>>> However, I am not sure if we should even allow `withChain()` to return
>>>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>>>>>> -> null` entry in the returned Map.
>>>>>>>>
>>>>>>>> Following this train of through, and if we want to allow the "return
>>>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>>>>>> an entry to the Map.
>>>>>>>>
>>>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>>>>>> be achieved with `withChain`:
>>>>>>>>
>>>>>>>> Branched.withChain(s -> {
>>>>>>>>       s.to();
>>>>>>>>       return s;
>>>>>>>> })
>>>>>>>>
>>>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>>>>>> while for the first proposal it adds new functionality (if `return null`
>>>>>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> I guess I can be convinced either way. However, if we follow your
>>>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>>>>>> benefit seems to be small? Also, having a reduced API is usually
>>>>>>>> preferable as it's simpler to learn.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>>>> Hello, John, hello Matthias!
>>>>>>>>>
>>>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>>>
>>>>>>>>> -----------------------------------------
>>>>>>>>>
>>>>>>>>> John,
>>>>>>>>>
>>>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>>>
>>>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>>>>>> all the emails on the web.
>>>>>>>>>
>>>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>>>>>> method?
>>>>>>>>>
>>>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>>>>>
>>>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>>>>>
>>>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>>>>>
>>>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>>>>>
>>>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>>>
>>>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>>>>>> chain and the result map OR using just the sink
>>>>>>>>>
>>>>>>>>> This is discussed below.
>>>>>>>>>
>>>>>>>>> ----------------------------------------------
>>>>>>>>>
>>>>>>>>> Mathhias,
>>>>>>>>>
>>>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>>>>>
>>>>>>>>> Done.
>>>>>>>>>
>>>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>>>>>> method]
>>>>>>>>>
>>>>>>>>> Fixed.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>>>
>>>>>>>>> Done in `Proposed Changes` section.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>>>>>> be given to the `Consumer`.
>>>>>>>>>
>>>>>>>>> Yes, I agree!
>>>>>>>>>
>>>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>>>>>> implementation would be to not add the "branch" to the result map if
>>>>>>>>> `withConsumer` is used?
>>>>>>>>>
>>>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>>>>>
>>>>>>>>> In the current version of KIP it is assumed that the returned map
>>>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>>>>>> map is the user's responsibility.
>>>>>>>>>
>>>>>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>>>>>> it as it is?
>>>>>>>>>
>>>>>>>>> ----
>>>>>>>>>
>>>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>>>
>>>>>>>>> Looking forward for your feedback again!
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>>>> Thanks for updating the KIP!
>>>>>>>>>>
>>>>>>>>>> I also have some minor comment:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>>>>>
>>>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>>>>>> so we just keep them.)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> (2) Quote:
>>>>>>>>>>
>>>>>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>>>>>> parameterless alternatives.
>>>>>>>>>>
>>>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>>>>>> 4th comment:
>>>>>>>>>>
>>>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>>>
>>>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>>>>>
>>>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> (5) Reply to John's comments:
>>>>>>>>>>
>>>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>>>
>>>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>>>
>>>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>
>>>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>>>>>> but I had a few last comments.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> John
>>>>>>>>>>>
>>>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>
>>>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>>>
>>>>>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Ivan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>>>
>>>>>>>>>>>>> The exception is
>>>>>>>>>>>>>
>>>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>>>>>
>>>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>>>>>> here it is justified.
>>>>>>>>>>>>>
>>>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>>>>>> as it
>>>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>>>>>
>>>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>>>>>> build on
>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>>>>>> want to follow:
>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>>>>>> June
>>>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>>>           .branch(....)
>>>>>>>>>>>>>>>           .defaultBranch(result::set)
>>>>>>>>>>>>>>>           .onTopOf(someStream);
>>>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>         -> KBranchedStream
>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>         -> KBranchedStream
>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>>>>>> Or,
>>>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>>>>>> finally,
>>>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>            withChain(...);
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>>>                      .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>>>                      .defaultBranch(...) // creates node
>>>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>            operator(function, config_object?) OR
>>>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>          operator(function, function, string)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>           interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>>>>>            withPredicate(...);
>>>>>>>>>>>>>>>>>>>            withChain(...);
>>>>>>>>>>>>>>>>>>>            withName(...);
>>>>>>>>>>>>>>>>>>>          }
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>          -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>>>          -> KBranchedStream
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>>>          -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>>>            for the dynamic case.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>            question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>>>            in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>            seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>>>            when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>>>            for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>>>            solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>>>            to.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>            <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>>>            LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>>>            matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>>>            advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>>>            proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>            But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>>>            workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>>>            method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>>>            tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>>>            going to be clean.
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>>>            handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>>>            a handler?
>>>>>>>>>>>>>>>>>>>>>>            > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>>>>>            > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>            writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>>>            "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>>>            identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>>>            all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>>>            point?
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > ---
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>            without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>>>            the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>>>            current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>>>            me address both comments here.
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>>>            for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>>>            see example below.
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>>>            Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>>>            value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>>>            have been much messier.*/
>>>>>>>>>>>>>>>>>>>>>>            > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>>>            > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>>>            >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>>>            recordType,
>>>>>>>>>>>>>>>>>>>>>>            >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > Regards,
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > Ivan
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>            > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>            >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>>>            >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>>>            thinking
>>>>>>>>>>>>>>>>>>>>>>            >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>>>            >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>>>            >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>>>            >> An open question is the case for which no
>>>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>            >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>            `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>>>            >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>>>            >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>>>            >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>>>            `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>>>            >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>>>            >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>>>            argument" yet
>>>>>>>>>>>>>>>>>>>>>>            >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >> -Matthias
>>>>>>>>>>>>>>>>>>>>>>            >>
>>>>>>>>>>>>>>>>>>>>>>            >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>>>            that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>>>            it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>>>            The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>            (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>>>            to each other.
>>>>>>>>>>>>>>>>>>>>>>            >>>
>>>>>>>>>>>>>>>>>>>>>>            >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>>>            branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>>>>>            >>>
>>>>>>>>>>>>>>>>>>>>>>            >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>>>            <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>            <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>>>            downstream
>>>>>>>>>>>>>>>>>>>>>>            >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>            quickly become
>>>>>>>>>>>>>>>>>>>>>>            >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>>>>>            >>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>>>            <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>>            that sets a
>>>>>>>>>>>>>>>>>>>>>>            >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>>>            current
>>>>>>>>>>>>>>>>>>>>>>            >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>>>            decisions) is
>>>>>>>>>>>>>>>>>>>>>>            >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>            construct
>>>>>>>>>>>>>>>>>>>>>>            >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>            provides as much
>>>>>>>>>>>>>>>>>>>>>>            >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>            call
>>>>>>>>>>>>>>>>>>>>>>            >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>>>            The value it
>>>>>>>>>>>>>>>>>>>>>>            >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>>>            the same
>>>>>>>>>>>>>>>>>>>>>>            >>>>> scope.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>            in the sense
>>>>>>>>>>>>>>>>>>>>>>            >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>>            branches, but it is
>>>>>>>>>>>>>>>>>>>>>>            >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>>>            the "static"
>>>>>>>>>>>>>>>>>>>>>>            >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>>>            fluently declare and
>>>>>>>>>>>>>>>>>>>>>>            >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>>>            ignore a
>>>>>>>>>>>>>>>>>>>>>>            >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>>>            SortedMap
>>>>>>>>>>>>>>>>>>>>>>            >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>>>            SortedMap being
>>>>>>>>>>>>>>>>>>>>>>            >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>>>            like this:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>>>            >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>>>>>            >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>            readable than
>>>>>>>>>>>>>>>>>>>>>>            >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Cons
>>>>>>>>>>>>>>>>>>>>>>            >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>            makes it harder
>>>>>>>>>>>>>>>>>>>>>>            >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>>>            unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>            >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>>>            existing
>>>>>>>>>>>>>>>>>>>>>>            >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>>>            that's overdoing
>>>>>>>>>>>>>>>>>>>>>>            >>>>> it).
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>>>            access the
>>>>>>>>>>>>>>>>>>>>>>            >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>            possible that it
>>>>>>>>>>>>>>>>>>>>>>            >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>            left up to the
>>>>>>>>>>>>>>>>>>>>>>            >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>>>            <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>            existing
>>>>>>>>>>>>>>>>>>>>>>            >>>>> branch
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>>>            proceed by
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>            important to be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>>>            this KIP.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>            double brace
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>>>            >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>>>            <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>>>            To me,
>>>>>>>>>>>>>>>>>>>>>>            >>>>> there
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>>>            positionally coupled
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>            additional code
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>>>            constructs
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> would
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>>>            looks like
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>>>            Predicate<?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>>>            returns a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>>>            solves
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> problem
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>>>            branch you’re
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>            another
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>>>            structure, rather
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> than
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>>>            historically
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>>>            voluminously, but
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>>>            <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>            >>>>> operator
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>>>            "Split"
>>>>>>>>>>>>>>>>>>>>>>            >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>>>            >>>>> branching"
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>            >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>>>            is, `branch`
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>>>            `void`, to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>>>            definition of
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>>>            there's no
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>>>            record
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> falls
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>>>            >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>>>            should
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>>>            `branch()` does
>>>>>>>>>>>>>>>>>>>>>>            >>>>> not
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>            >>>>> hidden
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>>>            I though
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> it
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>> naming
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>>            >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>>>            avoid
>>>>>>>>>>>>>>>>>>>>>>            >>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>            >>>>> cannot
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> have
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>>>            with a
>>>>>>>>>>>>>>>>>>>>>>            >>>>> short
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>>            with all
>>>>>>>>>>>>>>>>>>>>>>            >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>>>            contained in
>>>>>>>>>>>>>>>>>>>>>>            >>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>>>            `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>>>>>            branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>>>            that users
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>>>            access them.
>>>>>>>>>>>>>>>>>>>>>>            >>>>> We
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>>>            to get
>>>>>>>>>>>>>>>>>>>>>>            >>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>>>            loss of
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>>>            without
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>            >>>>> should
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>>            >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>>>            with such
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> name
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>>>            >>>>> think
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>>>            something.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>            proposal.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> There
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>            recommend the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>>>            BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>>>            `Predicate` as
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>>>            accepted and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>>>            parameter.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>>>            different
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>>>            index)` method
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>>>            object?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>            whatever
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>            current
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>            makes sense
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>>>            we'll
>>>>>>>>>>>>>>>>>>>>>>            >>>>> need
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>>>            >>>>> whether
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>>>            think this
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>>>            starting a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>>>            assuming
>>>>>>>>>>>>>>>>>>>>>>            >>>>> we
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>            >>>>> favor
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>>>            similar
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>>>            >>>>> it's
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> modified
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>>>            KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>>>            >>>>> a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>>>            convinced: your
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>>>            And then
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>>>            resume the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>>>            >>>>> starting
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>>>            starting
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>>>            >>>>> details.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>>>            >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>>>            >>>>> invoked
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> as
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>>>            still
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> ought
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            depend on
>>>>>>>>>>>>>>>>>>>>>>            >>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>>>            access
>>>>>>>>>>>>>>>>>>>>>>            >>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>>>            is, not
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>>            >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> It
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>>>            excited
>>>>>>>>>>>>>>>>>>>>>>            >>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>>>            >>>>> glance,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>>>            same
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> scope
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>>>            >>>>> again
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>>>            just
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> going
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>>>            we need
>>>>>>>>>>>>>>>>>>>>>>            >>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>>>            in the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>>>            circumstances
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> using
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>>>            complex, so
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>>>            order to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>>>            everything
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>>>            initialize the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>>>            essentially
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>>>            based on
>>>>>>>>>>>>>>>>>>>>>>            >>>>> my
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>>>            fluent
>>>>>>>>>>>>>>>>>>>>>>            >>>>> API
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>>>            (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>>>            that Java
>>>>>>>>>>>>>>>>>>>>>>            >>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>>>            returning one
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>            need
>>>>>>>>>>>>>>>>>>>>>>            >>>>> it.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>>>            shares
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            branching.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>>>            demonstrates
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>            >>>>> final
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> or
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>>>            wanted to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> see
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>            solution
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>>>            >>>>> suggested
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>            streams
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>>>            is, if we
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>>>            that.  The
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> either
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>>>            table.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> point.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            branch API
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>>>            returns
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> its
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>        ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>            instance
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>>>            miss the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> fact
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>>>            methods
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> are
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>>            implemented the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>>>            >>>>> nothing
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> besides
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>>>            want to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> just
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>>>            predicate. 2)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>            looks
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> like a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>>>            emit a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>            >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>            a clear
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>>>            >>>>> compiled
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>>>            compile if
>>>>>>>>>>>>>>>>>>>>>>            >>>>> used
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>>>            method chain
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>>>            between
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>>>            >>>>> instantly
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>>>            compilation
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>>>            required.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>>>            just as
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>>>            >>>>> user
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> default
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>            >>>>> seems
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>>>            which allows
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> for
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>>>            to be
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> well
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>>>            >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>>>            a clear
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> error
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>            >>>>> "build
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>>>            >>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>>>            stream.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>>>            >>>>> times,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>>>            operations
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> on
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>>            always need
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>>>            >>>>> don't
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>>>            so we
>>>>>>>>>>>>>>>>>>>>>>            >>>>> can
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> do
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>>>            >>>>> object
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>>>            >>>>> here
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> we
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>>>            flow,
>>>>>>>>>>>>>>>>>>>>>>            >>>>> so
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> I
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>            API, but I
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> find
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>>>            contrasts the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>>>            just call
>>>>>>>>>>>>>>>>>>>>>>            >>>>> a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>>>            cases
>>>>>>>>>>>>>>>>>>>>>>            >>>>> are
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>>>            is very
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> nice
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>>            how we
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>>>            terminated by
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>>>            incompatible with
>>>>>>>>>>>>>>>>>>>>>>            >>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>            different
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> name,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> something
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>            KIP?  It
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> seems
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>>>            while also
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>>>            KBranchedStreams
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> if
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>>>            ks){
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>>>            String> ks){
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>            this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>>>            this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>>>            >>>>> takes a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>>            example in
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>            >>>>> handle
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>>>            >>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>>>            we had
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>>>            >>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>>>            >>>>> Please
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>>>            feedback :)
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>>>            >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>            >
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Attachments:
>>>>>>>> * signature.asc
>>>>>>
>>>>>>
>>>>
>>>>
>>>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Thanks for the reply, Ivan,

I'm sorry things have been dragging out a little, but I have the sense
we're very close to the end of this discussion, which is exciting.

>  > I'd wonder whether we need the non-static builders (like withChain). 
>  > Do they provide any benefit over just using the right static factory?
> 
> I don't have a strong opinion here. I think it's just a matter of taste. 
> But, if we like to use Occam's razor, then yes, non-static builders can 
> be omitted, I agree!

I don't have a strong feeling, either. It seems nice to offer a better
type inference experience than what we get with Materialized, by
offering the static method that takes both name and chain.

Given that, there doesn't seem to be a good reason to also offer the
non-static builder-style methods, so I guess I'd prefer to drop them.

I'll defer to Matthias, if he has a chance to consider whether it's more
valuable to stick with the existing pattern or break the pattern to offer
a better experience.

>  > You might as well propose the “ideal” API in the KIP, which is the 
>  > covariant typed function
> 
> I didn't quite get it. Do I get you right that you propose the 
> following: 1) accept the 'fully covariant' definition in the KIP, 2) 
> then just see if there are any obstacles/pitfalls during implementation 
> and unit testing?

Yep! I know it's a bit sloppy, but my experience has been that we just
won't know what works until we really try it, and try it in several different
ways. Still, we may as well hope for the best, and propose the "fully
covariant" definition for now.

>  > Is it necessary to restrict the result key and value types to be the 
>  > same as the inputs?
> 
> Good question, I already thought about it and rejected the idea....
(I cut off your quote; the rest is in the chain below)

That's fair! It's your KIP, after all. I think I might have made a different
call here, but I think this choice is fine. "Worst" case scenario: someone
else will wish the return type is something different, and we'll go 
through a painless deprecation transition to change it later. Thanks
to the clean design of your API, this doesn't seem to bad. And, of
course, you've actually been using similar functionality already, so it
seems we should trust your intuition.

Thanks,
-John


On Sun, May 24, 2020, at 19:21, Ivan Ponomarev wrote:
> Hi John,
> 
>  > I'd wonder whether we need the non-static builders (like withChain). 
> Do they provide any benefit over just using the right static factory?
> 
> I don't have a strong opinion here. I think it's just a matter of taste. 
> But, if we like to use Occam's razor, then yes, non-static builders can 
> be omitted, I agree!
> 
>  > You might as well propose the “ideal” API in the KIP, which is the 
> covariant typed function
> 
> I didn't quite get it. Do I get you right that you propose the 
> following: 1) accept the 'fully covariant' definition in the KIP, 2) 
> then just see if there are any obstacles/pitfalls during implementation 
> and unit testing?
> 
>  > Is it necessary to restrict the result key and value types to be the 
> same as the inputs?
> 
> Good question, I already thought about it and rejected the idea.
> 
> Look, if we want to keep `withChain`'s function optional, then we must 
> keep the result key and value types the same. Because for now, the 
> default value for the 'chain function' is Function.identity().
> 
> Of course, we can make the 'chain function' required. But I think this 
> is not what `split` method is for. `split` is for splitting, not 
> transforming, and `chainFunction` in most of the cases should be either 
> a consumer or the identity function.
> 
> Regards,
> 
> Ivan
> 
> 
> 24.05.2020 17:15, John Roesler пишет:
> > Thanks for the reply, Ivan.
> > 
> > 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
> > 
> > I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
> > 
> > 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
> > 
> > I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
> > 
> > Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
> > 
> > 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
> > 
> > Thanks,
> > John
> > 
> > On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
> >> Hello John,
> >>
> >>
> >> 1.
> >> ---------------------------------------------
> >>
> >>   > Perhaps it would be better to stick with "as" for now
> >>   > and just file a Jira to switch them all at the same time [for
> >> compatibility with Kotlin]
> >>
> >> Fully agree! BTW it's really not a big problem: in Kotlin they have a
> >> standard workaround
> >> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
> >> So actually this should be a very low priority issue, if an issue at
> >> all.
> >>
> >>   > I don't understand how your new proposed
> >>   > methods would work any differently than the ones you already
> >>   > had proposed in the KIP. It seems like you'd still have to provide
> >>   > the generic type parameters on the first static factory call. Can you
> >>   > explain how your new interface proposal differs from the existing KIP?
> >>
> >> In the KIP, I didn't clarify what methods should be static. Now I
> >> propose the following methods:
> >>
> >> non-static: withChain(Function), withName(String).
> >>
> >> static: as(String), with(Function), with(Function, String).
> >>
> >> The overloaded `with` version that provides both Function and name can
> >> be used without causing type inference problem!!
> >>
> >> 2.
> >> ----------------------------
> >>
> >>   > Regarding making the K,V types covariant also, yes, that would indeed
> >>   > be nice, but I'm not sure it will actually work.
> >>
> >> What I'm keeping in mind is the following example: imagine
> >>
> >> static KStream<String, Integer> func(KStream<String, Number> s) {
> >>           return s.mapValues(n -> (Integer) n + 1);
> >> }
> >>
> >> BranchedKStream<String, Number> b =
> >>       s.split().branch((k, v) -> isInteger(v),
> >>                  //Won't compile!!
> >>                  Branched.with(Me::func));
> >>
> >> The simple workaround here is to change `func`'s return type from
> >> KStream<...Integer> to KStream<...Number>.
> >>
> >> [On the other hand, we already agreed to remove `withJavaConsumer` from
> >> `Branched`, so during code migration I will have to modify my functions'
> >> return types anyway -- I mean, from `void` to `KStream`!! ]
> >>
> >>   >  the map you're returning is Map<K,V>, and of course a K is not the
> >> same as "? extends K", so it doesn't seem compatible.
> >>
> >> I think what you actually meant here is that KStream<? extends K, ?
> >> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
> >> particularly is not a problem, since KStream<? extends K, ? extends V>
> >> can be safely explicitly cast to KStream<K, V>, and be put to the map.
> >>
> >> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
> >> for now it's better to just admit that API is not absolutely perfect and
> >> accept it as is, that is
> >>
> >> Function<? super KStream<K, V>, ? extends KStream<K, V>>
> >>
> >> Regards,
> >>
> >> Ivan
> >>
> >>
> >> 21.05.2020 17:59, John Roesler пишет:
> >>> Hello Ivan,
> >>>
> >>> Thanks for the refinement. Actually, I did not know that "as" would
> >>> clash with a Kotlin operator. Maybe we should depart from convention
> >>> and just avoid methods named "as" in the future.
> >>>
> >>> The convention is that "as(String name)" is used for the static factory
> >>> method, whereas "withName(String name)" is an instance method
> >>> inherited from NamedOperation. If you wish to propose to avoid "as"
> >>> for compatibility with Kotlin, I might suggest "fromName(String name)",
> >>> although it's somewhat dubious, since all the other configuration
> >>> classes use "as". Perhaps it would be better to stick with "as" for now
> >>> and just file a Jira to switch them all at the same time.
> >>>
> >>> Re. 3:
> >>> Regarding the type inference problem, yes, it's a blemish on all of our
> >>> configuraion objects. The problem is that Java infers the type
> >>> based on the _first_ method in the chain. While it does consider what
> >>> the recipient of the method result wants, it only considers the _next_
> >>> recipient.
> >>>
> >>> Thus, if you call as("foo") and immediately assign it to a
> >>> Branched<String,String> variable, java infers the type correctly. But
> >>> when the "next recipient" is a chained method call, like "withChain",
> >>> then the chained method doesn't bound the type (by definition,
> >>> withChain is defined on Branched<Object, Object>, so Java will take
> >>> the broadest possible inferece and bind the type to
> >>> Branched<Object, Object>, at which point, it can't be revised anymore.
> >>>
> >>> As a user of Java, this is exceedingly annoying, since it doesn't seem
> >>> that hard to recursively consider the entire context when inferring the
> >>> generic type parameters, but this is what we have to work with.
> >>>
> >>> To be honest, though, I don't understand how your new proposed
> >>> methods would work any differently than the ones you already
> >>> had proposed in the KIP. It seems like you'd still have to provide
> >>> the generic type parameters on the first static factory call. Can you
> >>> explain how your new interface proposal differs from the existing KIP?
> >>>
> >>> Re. 4:
> >>> Regarding making the K,V types covariant also, yes, that would indeed
> >>> be nice, but I'm not sure it will actually work. You might want to give it a
> >>> try. In the past, we've run into soe truly strange interactions between the
> >>> Java type inferencer and lambdas (and/or anonymous inner classes) in
> >>> combination with nested covariant types.
> >>>
> >>> Another issue is that the value type of the map you're returning is
> >>> Map<K,V>, and of course a K is not the same as "? extends K", so it
> >>> doesn't seem compatible.
> >>>
> >>> Thanks again,
> >>> -John
> >>>
> >>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
> >>>> Hi,
> >>>>
> >>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
> >>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
> >>>> simpler'.
> >>>>
> >>>> I made some quick API mocking in my IDE and tried to implement examples
> >>>> from KIP.
> >>>>
> >>>> 1. Having to return something from lambda is not a very big deal.
> >>>>
> >>>> 2. For a moment I thouht that I won't be able to use method references
> >>>> for already written stream consumers, but then I realized that I can
> >>>> just change my methods from returning void to returning the input
> >>>> parameter and use references to them. Not very convenient, but passable.
> >>>>
> >>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
> >>>> function returns null, we don't insert it into the resulting map.
> >>>>
> >>>> Usually it's better to implement a non-perfect, but workable solution as
> >>>> a first approximation. And later we can always add to `Branched`
> >>>> anything we want.
> >>>>
> >>>> 3. Do we have any guidelines on how parameter classes like Branched
> >>>> should be built? First of all, it seems that `as` now is more preferred
> >>>> than `withName` (although as you probably know it clashes with Kotlin's
> >>>> `as` operator).
> >>>>
> >>>> Then, while trying to mock the APIs, I found out that my Java cannot
> >>>> infer types in the following construction:
> >>>>
> >>>> .branch((key, value) -> value == null,
> >>>>       Branched.as("foo").withChain(s -> s.mapValues(...)))
> >>>>
> >>>>
> >>>> so I have to write
> >>>>
> >>>> .branch((key, value) -> value == null,
> >>>>       Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
> >>>>
> >>>>
> >>>> This is not tolerable IMO, so this is the list of `Branched` methods
> >>>> that I came to (will you please validate it):
> >>>>
> >>>> static <K, V> Branched<K, V> as(String name);
> >>>>
> >>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>> extends KStream<K, V>> chain);
> >>>>
> >>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >>>> extends KStream<K, V>> chain, String name);
> >>>>
> >>>> //non-static!
> >>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
> >>>> KStream<K, V>> chain);
> >>>>
> >>>>
> >>>> 4. And one more. What do you think, do we need that flexibility:
> >>>>
> >>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
> >>>>
> >>>> vs.
> >>>>
> >>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
> >>>> extends K, ? extends V>> chain
> >>>>
> >>>> ??
> >>>>
> >>>> Regards,
> >>>>
> >>>> Ivan
> >>>>
> >>>>
> >>>> 21.05.2020 6:54, John Roesler пишет:
> >>>>> Thanks for this thought, Matthias,
> >>>>>
> >>>>> Your idea has a few aspects I find attractive:
> >>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
> >>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
> >>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
> >>>>>
> >>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
> >>>>>
> >>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
> >>>>>
> >>>>> Thanks again for sharing the idea,
> >>>>> John
> >>>>>
> >>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> >>>>>> Thanks for updating the KIP!
> >>>>>>
> >>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
> >>>>>> its relationship to the returned `Map`.
> >>>>>>
> >>>>>> Originally, we discussed two main patterns:
> >>>>>>
> >>>>>>     (1) split a stream and return the substreams for futher processing
> >>>>>>     (2) split a stream and modify the substreams with in-place method chaining
> >>>>>>
> >>>>>> To combine both patterns we wanted to allow for
> >>>>>>
> >>>>>>      -> split a stream, modify the substreams, and return the _modified_
> >>>>>> substreams for further processing
> >>>>>>
> >>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> >>>>>>
> >>>>>> That is of course possible. However, it introduces some "hidded" semantics:
> >>>>>>
> >>>>>>     - using `withChain` I get the modified sub-stream
> >>>>>>     - using `withJavaConsumer` I get the unmodifed sub-stream
> >>>>>>
> >>>>>> This seems to be quite subtle to me.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>    From my understanding the original idea of `withJavaConsumer` was to
> >>>>>> model a terminal operation, ie, it should be similar to:
> >>>>>>
> >>>>>> Branched.withChain(s -> {
> >>>>>>      s.to();
> >>>>>>      return null;
> >>>>>> })
> >>>>>>
> >>>>>> However, I am not sure if we should even allow `withChain()` to return
> >>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
> >>>>>> -> null` entry in the returned Map.
> >>>>>>
> >>>>>> Following this train of through, and if we want to allow the "return
> >>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
> >>>>>> an entry to the Map.
> >>>>>>
> >>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
> >>>>>> be achieved with `withChain`:
> >>>>>>
> >>>>>> Branched.withChain(s -> {
> >>>>>>      s.to();
> >>>>>>      return s;
> >>>>>> })
> >>>>>>
> >>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> >>>>>> while for the first proposal it adds new functionality (if `return null`
> >>>>>> is not allowed, using `withChain()` is not possible to "hide a
> >>>>>> sub-stream in the result). Furthermore, we might need to allow `return
> >>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> I guess I can be convinced either way. However, if we follow your
> >>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
> >>>>>> benefit seems to be small? Also, having a reduced API is usually
> >>>>>> preferable as it's simpler to learn.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> >>>>>>> Hello, John, hello Matthias!
> >>>>>>>
> >>>>>>> Thank you very much for your detailed feedback!
> >>>>>>>
> >>>>>>> -----------------------------------------
> >>>>>>>
> >>>>>>> John,
> >>>>>>>
> >>>>>>>> It looks like you missed my reply on Apr 23rd.
> >>>>>>>
> >>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
> >>>>>>> all the emails on the web.
> >>>>>>>
> >>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> >>>>>>> method?
> >>>>>>>
> >>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
> >>>>>>>
> >>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
> >>>>>>> off of the parent KStream for the needs of dynamic branching]
> >>>>>>>
> >>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
> >>>>>>>
> >>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> >>>>>>>
> >>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
> >>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
> >>>>>>>
> >>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
> >>>>>>> chain and the result map OR using just the sink
> >>>>>>>
> >>>>>>> This is discussed below.
> >>>>>>>
> >>>>>>> ----------------------------------------------
> >>>>>>>
> >>>>>>> Mathhias,
> >>>>>>>
> >>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> >>>>>>>
> >>>>>>> Done.
> >>>>>>>
> >>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> >>>>>>> method]
> >>>>>>>
> >>>>>>> Fixed.
> >>>>>>>
> >>>>>>>
> >>>>>>>> 3. Overview of newly added methods/interfaces
> >>>>>>>
> >>>>>>> Done in `Proposed Changes` section.
> >>>>>>>
> >>>>>>>
> >>>>>>>> 4. [Concerning John's note] > I don't think that using both
> >>>>>>> `withChain()` and `withConsumer()` is the
> >>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
> >>>>>>> be given to the `Consumer`.
> >>>>>>>
> >>>>>>> Yes, I agree!
> >>>>>>>
> >>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
> >>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> >>>>>>> implementation would be to not add the "branch" to the result map if
> >>>>>>> `withConsumer` is used?
> >>>>>>>
> >>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
> >>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
> >>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> >>>>>>> stream in the Map, one simply does not extract it from there :-)
> >>>>>>>
> >>>>>>> In the current version of KIP it is assumed that the returned map
> >>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
> >>>>>>> programmer, or with some default auto-generated ids. Dealing with this
> >>>>>>> map is the user's responsibility.
> >>>>>>>
> >>>>>>> What seems to me to be an issue is introducing exclusions to this
> >>>>>>> general rule, like 'swallowing' some streams by provided
> >>>>>>> [Java]Consumers. This can make things complicated. What if a user
> >>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
> >>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
> >>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
> >>>>>>> it as it is?
> >>>>>>>
> >>>>>>> ----
> >>>>>>>
> >>>>>>> I rewrote the KIP and also fixed a couple of typos.
> >>>>>>>
> >>>>>>> Looking forward for your feedback again!
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>>
> >>>>>>> Ivan.
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
> >>>>>>>> Thanks for updating the KIP!
> >>>>>>>>
> >>>>>>>> I also have some minor comment:
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> >>>>>>>>
> >>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> >>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> >>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
> >>>>>>>> so we just keep them.)
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> (2) Quote:
> >>>>>>>>
> >>>>>>>>> Both branch and defaultBranch operations also have overloaded
> >>>>>>>>> parameterless alternatives.
> >>>>>>>>
> >>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
> >>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> >>>>>>>> as `branch()` would not be "parameterless".
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
> >>>>>>>> newly added and deprecated methods/classes (cf.
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> >>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
> >>>>>>>> 4th comment:
> >>>>>>>>
> >>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
> >>>>>>>>> chain and the result map OR using just the sink.
> >>>>>>>>
> >>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
> >>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
> >>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
> >>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> >>>>>>>>
> >>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
> >>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
> >>>>>>>> it in the JavaDocs, this might be fine?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> (5) Reply to John's comments:
> >>>>>>>>
> >>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> >>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
> >>>>>>>>> sense, of course). I get that you were referring to the java Consumer
> >>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
> >>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
> >>>>>>>>
> >>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
> >>>>>>>> node", ie., writing the KStream to a topic.
> >>>>>>>>
> >>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
> >>>>>>>>> Hi Ivan,
> >>>>>>>>>
> >>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> >>>>>>>>> but I had a few last comments.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> John
> >>>>>>>>>
> >>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> >>>>>>>>>> Hello everyone,
> >>>>>>>>>>
> >>>>>>>>>> will someone please take a look at the reworked KIP?
> >>>>>>>>>>
> >>>>>>>>>> I believe that now it follows design principles and takes into account
> >>>>>>>>>> all the arguments discussed here.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>>
> >>>>>>>>>> Ivan
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> I have read the John's "DSL design principles" and have completely
> >>>>>>>>>>> rewritten the KIP, see
> >>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> This version includes all the previous discussion results and follows
> >>>>>>>>>>> the design principles, with one exception.
> >>>>>>>>>>>
> >>>>>>>>>>> The exception is
> >>>>>>>>>>>
> >>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> >>>>>>>>>>>
> >>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
> >>>>>>>>>>> here it is justified.
> >>>>>>>>>>>
> >>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
> >>>>>>>>>>> for the default branch. Thus for both operations we may use a single
> >>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
> >>>>>>>>>>>
> >>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
> >>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> >>>>>>>>>>> as it
> >>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
> >>>>>>>>>>>
> >>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>>
> >>>>>>>>>>> Ivan
> >>>>>>>>>>>
> >>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>
> >>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
> >>>>>>>>>>>>
> >>>>>>>>>>>> The DSL improved further in the meantime and we already have a
> >>>>>>>>>>>> `Named`
> >>>>>>>>>>>> config object to name operators. It seems reasonable to me to
> >>>>>>>>>>>> build on
> >>>>>>>>>>>> this.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> >>>>>>>>>>>> want to follow:
> >>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> -- might be worth to checkout.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>> Hi everyone!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
> >>>>>>>>>>>>> June
> >>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
> >>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
> >>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
> >>>>>>>>>>>>> propose other things before this one is finalized.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
> >>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
> >>>>>>>>>>>>> branches, I worked around it this way:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>>>>>>>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>>>>>>>          .branch(....)
> >>>>>>>>>>>>>          .defaultBranch(result::set)
> >>>>>>>>>>>>>          .onTopOf(someStream);
> >>>>>>>>>>>>> result.get()...
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>        -> KBranchedStream
> >>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>> //
> >>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>>>>>>>        -> KBranchedStream
> >>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
> >>>>>>>>>>>>> a good
> >>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
> >>>>>>>>>>>>> you
> >>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
> >>>>>>>>>>>>> Or,
> >>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
> >>>>>>>>>>>>> finally,
> >>>>>>>>>>>>> you might use the returned Map to have the named branches in the
> >>>>>>>>>>>>> original scope.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> >>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
> >>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
> >>>>>>>>>>>>> familiar with Streams API design principles than me.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>> at any point.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> >>>>>>>>>>>>>> do so.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
> >>>>>>>>>>>>>>> to mix
> >>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> >>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> >>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> >>>>>>>>>>>>>>>> patterns
> >>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
> >>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>> totally sense.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> >>>>>>>>>>>>>>>> `Map` only
> >>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
> >>>>>>>>>>>>>>>> all of
> >>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>>>>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> >>>>>>>>>>>>>>>> `Named` is
> >>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> >>>>>>>>>>>>>>>> counter
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
> >>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
> >>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>>>>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
> >>>>>>>>>>>>>>>> misses to
> >>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> >>>>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> >>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>>>>>>>>>>>> specifying a
> >>>>>>>>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
> >>>>>>>>>>>>>>>> hence
> >>>>>>>>>>>>>>>> would not include it in a configuration object.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           withChain(...);
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> >>>>>>>>>>>>>>>> does not
> >>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> >>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
> >>>>>>>>>>>>>>>> make sense
> >>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> >>>>>>>>>>>>>>>> check
> >>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
> >>>>>>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
> >>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> >>>>>>>>>>>>>>>> object
> >>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
> >>>>>>>>>>>>>>>> others,
> >>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
> >>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
> >>>>>>>>>>>>>>>> methods that
> >>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
> >>>>>>>>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> >>>>>>>>>>>>>>>> two main
> >>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> >>>>>>>>>>>>>>>>> satisfy
> >>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
> >>>>>>>>>>>>>>>>> solves
> >>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> >>>>>>>>>>>>>>>>> to add
> >>>>>>>>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
> >>>>>>>>>>>>>>>>> Otherwise,
> >>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> >>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
> >>>>>>>>>>>>>>>>> names for
> >>>>>>>>>>>>>>>>> operations are not required to define stream processing
> >>>>>>>>>>>>>>>>> logic, it
> >>>>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
> >>>>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> >>>>>>>>>>>>>>>>> processing onto
> >>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> >>>>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
> >>>>>>>>>>>>>>>>> proposal, we
> >>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
> >>>>>>>>>>>>>>>>> name,
> >>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          stream.split(Named.withName("mysplit")) //creates node
> >>>>>>>>>>>>>>>>> "mysplit"
> >>>>>>>>>>>>>>>>>                     .branch(..., ..., "abranch") // creates node
> >>>>>>>>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>>>>>>>                     .defaultBranch(...) // creates node
> >>>>>>>>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> >>>>>>>>>>>>>>>>> debate
> >>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
> >>>>>>>>>>>>>>>>> general,
> >>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>           operator(function, config_object?) OR
> >>>>>>>>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
> >>>>>>>>>>>>>>>>> variant.
> >>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>         operator(function, function, string)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> where the string is the name.
> >>>>>>>>>>>>>>>>> My first question is whether the name should instead be
> >>>>>>>>>>>>>>>>> specified
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
> >>>>>>>>>>>>>>>>> arguments
> >>>>>>>>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>          interface BranchConfig extends NamedOperation {
> >>>>>>>>>>>>>>>>>           withPredicate(...);
> >>>>>>>>>>>>>>>>>           withChain(...);
> >>>>>>>>>>>>>>>>>           withName(...);
> >>>>>>>>>>>>>>>>>         }
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> >>>>>>>>>>>>>>>>> more like
> >>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
> >>>>>>>>>>>>>>>>> makes us
> >>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> >>>>>>>>>>>>>>>>> purely
> >>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
> >>>>>>>>>>>>>>>>> overloads
> >>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> WDYT?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> >>>>>>>>>>>>>>>>>> view.
> >>>>>>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> >>>>>>>>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>>>>>         -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> >>>>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>>>         -> KBranchedStream
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> >>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
> >>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> >>>>>>>>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
> >>>>>>>>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> >>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> >>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
> >>>>>>>>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> >>>>>>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> >>>>>>>>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> >>>>>>>>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>>>>>>>> language
> >>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> >>>>>>>>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
> >>>>>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> >>>>>>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> >>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> >>>>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> >>>>>>>>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           Ivan, I’ll definitely forfeit my point on the
> >>>>>>>>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           branch(predicate, consumer) solution, I don’t see
> >>>>>>>>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>>>>>>>           for the dynamic case.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           IMO the one trade off to consider at this point is the
> >>>>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>>>           question. I don’t know if I totally agree that “we
> >>>>>>>>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>>>>>           in the same scope” since merging the branches back
> >>>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>           seems like a perfectly plausible use case that can
> >>>>>>>>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>>>>>>>           when the branched streams are in the same scope.
> >>>>>>>>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>>>           for the reasons Ivan listed, I think it is overall the
> >>>>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>>>           solution - working around the scope thing is easy
> >>>>>>>>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>>>>>>>           to.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>           <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > Hello everyone, thank you all for joining the
> >>>>>>>>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > Well, I don't think the idea of named branches,
> >>>>>>>>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>>>>>>>           LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>>>>>           matters) or `branch` method  taking name and Consumer
> >>>>>>>>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>>>>>>>           advantages than drawbacks.
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > In my opinion, the only real positive outcome from
> >>>>>>>>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>>>>>>>           proposal is that all the returned branches are in
> >>>>>>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>>>           But 1) we rarely need them in the same scope 2)
> >>>>>>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>>>>>           workaround for the scope problem, described in the
> >>>>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > 'Inlining the complex logic' is not a problem,
> >>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>>>>>>>           method references instead of lambdas. In real world
> >>>>>>>>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>>>>>>>           tend to split the complex logic to methods anyway,
> >>>>>>>>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>>>>>>>           going to be clean.
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > The drawbacks are strong. The cohesion between
> >>>>>>>>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           handlers is lost. We have to define predicates in one
> >>>>>>>>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>>>>>>>           handlers in another. This opens the door for bugs:
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > - what if we forget to define a handler for a
> >>>>>>>>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>>>>>>>           a handler?
> >>>>>>>>>>>>>>>>>>>>           > - what if we misspell a name?
> >>>>>>>>>>>>>>>>>>>>           > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > What Michael propose would have been totally OK
> >>>>>>>>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>           writing the API in Lua, Ruby or Python. In those
> >>>>>>>>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>>>>>>>           "dynamic naming" approach would have looked most
> >>>>>>>>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           beautiful. But in Java we expect all the problems
> >>>>>>>>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>>>>>>>           identifiers to be eliminated in compile time.
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > And if we do, what advantage are we supposed to get
> >>>>>>>>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>>>>>>>           all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>>>>>>>           point?
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > ---
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > Earlier in this discussion John Roesler also
> >>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>           without "start branching" operator, and later Paul
> >>>>>>>>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>>>>>>>           the case when we have to add a dynamic number of
> >>>>>>>>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>>>>>>>           current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>>>>>>>           me address both comments here.
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > 1) "Start branching" operator (I think that
> >>>>>>>>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>>>>>>>           for it indeed) is critical when we need to do a
> >>>>>>>>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>>>>>>>           see example below.
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > 2) No, dynamic branching in current KIP is not
> >>>>>>>>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>>>           Imagine a real-world scenario when you need one
> >>>>>>>>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>>>>>>>           value (say, RecordType). You can have something
> >>>>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > /*John:if we had to start with stream.branch(...)
> >>>>>>>>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>>>>>>>           have been much messier.*/
> >>>>>>>>>>>>>>>>>>>>           > KBranchedStream branched = stream.split();
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > /*Not clumsy at all :-)*/
> >>>>>>>>>>>>>>>>>>>>           > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>>>>>>>>>>>           >             branched = branched.branch((k, v) ->
> >>>>>>>>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>>>>>>>           recordType,
> >>>>>>>>>>>>>>>>>>>>           >                     recordType::processRecords);
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > Regards,
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > Ivan
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>           > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>           >> I also agree with Michael's observation about
> >>>>>>>>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>>>>>>>           >> current `branch()` implementation.
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>>>>>>>           thinking
> >>>>>>>>>>>>>>>>>>>>           >> was more aligned with Paul's proposal to just
> >>>>>>>>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>>>>>>>           >> `branch()` statement and return a
> >>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >> It makes the code easier to read, and also make the
> >>>>>>>>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>>>>>>>           >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >>>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>>>>>>>           >> An open question is the case for which no
> >>>>>>>>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>           >> specified. Atm, `split()` and `branch()` would
> >>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>           `BranchedKStream`
> >>>>>>>>>>>>>>>>>>>>           >> and the call to `defaultBranch()` that returns the
> >>>>>>>>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>>>>>>>           >> (what is not the case atm). Or is this actually
> >>>>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>> real
> >>>>>>>>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>>>>>>>           >> because users can just ignore the branch
> >>>>>>>>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>>>>>>>           `defaultBranch()`
> >>>>>>>>>>>>>>>>>>>>           >> in the result `Map` ?
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >> About "inlining": So far, it seems to be a
> >>>>>>>>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>>>>>>>           >> preference. I can see arguments for both, but no
> >>>>>>>>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>>>>>>>           argument" yet
> >>>>>>>>>>>>>>>>>>>>           >> that clearly make the case for one or the other.
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >> -Matthias
> >>>>>>>>>>>>>>>>>>>>           >>
> >>>>>>>>>>>>>>>>>>>>           >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>           >>> Perhaps inlining is the wrong terminology. It
> >>>>>>>>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>           that a lambda with the full downstream topology be
> >>>>>>>>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>>>>>>>           it can be a method reference as with Ivan’s original
> >>>>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>>>           The advantage of putting the predicate and its
> >>>>>>>>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>>           (Consumer) together in branch() is that they are
> >>>>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>>>>>>>           to each other.
> >>>>>>>>>>>>>>>>>>>>           >>>
> >>>>>>>>>>>>>>>>>>>>           >>> Ultimately the downstream code has to live
> >>>>>>>>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>>>>>>>           branch trees will be hard to read regardless.
> >>>>>>>>>>>>>>>>>>>>           >>>
> >>>>>>>>>>>>>>>>>>>>           >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>>>>>>>           <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>           <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>
> >>>>>>>>>>>>>>>>>>>>           >>>> I'm less enthusiastic about inlining the
> >>>>>>>>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>>>>>>>           downstream
> >>>>>>>>>>>>>>>>>>>>           >>>> functionality. Programs that have deep branch
> >>>>>>>>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>           quickly become
> >>>>>>>>>>>>>>>>>>>>           >>>> harder to read as a single unit.
> >>>>>>>>>>>>>>>>>>>>           >>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>>>           <pgwhalen@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Also +1 on the issues/goals as Michael
> >>>>>>>>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>>           that sets a
> >>>>>>>>>>>>>>>>>>>>           >>>>> great framework for the discussion.
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Regarding the SortedMap solution, my
> >>>>>>>>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>>>>>           current
> >>>>>>>>>>>>>>>>>>>>           >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>>>>>>>           decisions) is
> >>>>>>>>>>>>>>>>>>>>           >>>>> roughly this:
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>           >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>           >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>>>           >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Obviously some ordering is necessary, since
> >>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>           construct
> >>>>>>>>>>>>>>>>>>>>           >>>>> doesn't work without it, but this solution seems
> >>>>>>>>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>           provides as much
> >>>>>>>>>>>>>>>>>>>>           >>>>> associativity as the SortedMap solution,
> >>>>>>>>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>           call
> >>>>>>>>>>>>>>>>>>>>           >>>>> directly associates the "conditional" with
> >>>>>>>>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>>>>>>>           The value it
> >>>>>>>>>>>>>>>>>>>>           >>>>> provides over the KIP solution is the
> >>>>>>>>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>>>>>           the same
> >>>>>>>>>>>>>>>>>>>>           >>>>> scope.
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> The KIP solution is less "dynamic" than the
> >>>>>>>>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>           in the sense
> >>>>>>>>>>>>>>>>>>>>           >>>>> that it is slightly clumsier to add a dynamic
> >>>>>>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>>>           branches, but it is
> >>>>>>>>>>>>>>>>>>>>           >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>>>>>>>           the "static"
> >>>>>>>>>>>>>>>>>>>>           >>>>> case anyway, and should make it simple and
> >>>>>>>>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>>>>>>>           fluently declare and
> >>>>>>>>>>>>>>>>>>>>           >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>>>>>>>           ignore a
> >>>>>>>>>>>>>>>>>>>>           >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>>>>>>>           SortedMap
> >>>>>>>>>>>>>>>>>>>>           >>>>> solution on top of it.
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> I could also see a middle ground where
> >>>>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>>>>>>>           SortedMap being
> >>>>>>>>>>>>>>>>>>>>           >>>>> taken in, branch() takes a name and not a
> >>>>>>>>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>>>>>>>           like this:
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>           >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>           >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>>>           >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Pros for that solution:
> >>>>>>>>>>>>>>>>>>>>           >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>>>>>>>>>>>           >>>>> - no double brace initialization, hopefully
> >>>>>>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>           readable than
> >>>>>>>>>>>>>>>>>>>>           >>>>> SortedMap
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Cons
> >>>>>>>>>>>>>>>>>>>>           >>>>> - downstream branch logic cannot be specified
> >>>>>>>>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>           makes it harder
> >>>>>>>>>>>>>>>>>>>>           >>>>> to read top to bottom (like existing API and
> >>>>>>>>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>>>>>>>           unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>           >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>>>>>>>           existing
> >>>>>>>>>>>>>>>>>>>>           >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> (KBranchedStreams could even work *both* ways
> >>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>>>>>>>           that's overdoing
> >>>>>>>>>>>>>>>>>>>>           >>>>> it).
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Overall I'm curious how important it is to be
> >>>>>>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>>>>>           access the
> >>>>>>>>>>>>>>>>>>>>           >>>>> branched KStream in the same scope as the
> >>>>>>>>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>           possible that it
> >>>>>>>>>>>>>>>>>>>>           >>>>> doesn't need to be handled directly by the
> >>>>>>>>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>           left up to the
> >>>>>>>>>>>>>>>>>>>>           >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> Paul
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> >>>>>>>>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>>>>>>>           <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>> I'd like to +1 what Michael said about the
> >>>>>>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>           existing
> >>>>>>>>>>>>>>>>>>>>           >>>>> branch
> >>>>>>>>>>>>>>>>>>>>           >>>>>> method, I agree with what he's outlined and
> >>>>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>>>>>>>           proceed by
> >>>>>>>>>>>>>>>>>>>>           >>>>>> trying to alleviate these problems.
> >>>>>>>>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>           important to be
> >>>>>>>>>>>>>>>>>>>>           >>>>>> able to cleanly access the individual
> >>>>>>>>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>>>>>>>           >>>>>> name->stream), which I thought was the original
> >>>>>>>>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>>>>>>>           this KIP.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>> That said, I don't think we should so easily
> >>>>>>>>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>           double brace
> >>>>>>>>>>>>>>>>>>>>           >>>>>> anti-pattern or force ours users into it if
> >>>>>>>>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>>>>>           >>>>> avoid...just
> >>>>>>>>>>>>>>>>>>>>           >>>>>> my two cents.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>>           >>>>>> Sophie
> >>>>>>>>>>>>>>>>>>>>           >>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> >>>>>>>>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>>>>>>>           >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>>>           <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> I’d like to propose a different way of
> >>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>>>>>>>           To me,
> >>>>>>>>>>>>>>>>>>>>           >>>>> there
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> are three problems with the existing branch
> >>>>>>>>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>>>           >>>>>> warnings.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> 2. The way in which you use the stream
> >>>>>>>>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>>>>>>>           positionally coupled
> >>>>>>>>>>>>>>>>>>>>           >>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> 3. It is brittle to extend existing branch
> >>>>>>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>           additional code
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> paths.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> Using associative constructs instead of
> >>>>>>>>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>>>>>>>           constructs
> >>>>>>>>>>>>>>>>>>>>           >>>>>> would
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> be a stronger approach. Consider a
> >>>>>>>>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>>>           looks like
> >>>>>>>>>>>>>>>>>>>>           >>>>>> this:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>>>>>>>           Predicate<?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> super K,? super V>>);
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> Branches are given names in a map, and as a
> >>>>>>>>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>>>>>>>           returns a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> mapping of names to streams. The ordering
> >>>>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>>>>>>>           >>>>>> maintained
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> because it’s a sorted map. Insert order
> >>>>>>>>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>>>>>>>> the order
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>           >>>>>> evaluation.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> This solves problem 1 because there are no
> >>>>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>>>>>>>           solves
> >>>>>>>>>>>>>>>>>>>>           >>>>>> problem
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> 2 because you no longer lean on ordering to
> >>>>>>>>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>>>>>>>           branch you’re
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> interested in. It solves problem 3 because
> >>>>>>>>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>>>>>           another
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> conditional by simply attaching another
> >>>>>>>>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>>>>>>>           structure, rather
> >>>>>>>>>>>>>>>>>>>>           >>>>>> than
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> messing with the existing indices.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>>>>>>>           historically
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> awkward in Java. I know it’s an
> >>>>>>>>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>>>>>>>           voluminously, but
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> double brace initialization would clean up the
> >>>>>>>>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>>>>>>>           <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>>>           >>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> Thanks for the update.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> FWIW, I agree with Matthias that the current
> >>>>>>>>>>>>>>>>>>>> "start
> >>>>>>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>>>>>           >>>>> operator
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> confusing when named the same way as the
> >>>>>>>>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>>>>>>>           "Split"
> >>>>>>>>>>>>>>>>>>>>           >>>>> seems
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> like a good name. Alternatively, we can do
> >>>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>>>>>>>           >>>>> branching"
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> operator at all, and just do:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> Tentatively, I think that this branching
> >>>>>>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>           >>>>> terminal.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> That
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> way, we don't create ambiguity about how
> >>>>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>>>>>>>           is, `branch`
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>>>>>>>           `void`, to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> enforce that it comes last, and that there
> >>>>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>>>           definition of
> >>>>>>>>>>>>>>>>>>>>           >>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>>>>>>>           there's no
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> default,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>>>>>>>           record
> >>>>>>>>>>>>>>>>>>>>           >>>>>> falls
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> though with no default.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>>>>>>>           >>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Thanks for updating the KIP and your
> >>>>>>>>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> this is to make the name similar to
> >>>>>>>>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> The intend was to avoid name duplication.
> >>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>>>>>>>           should
> >>>>>>>>>>>>>>>>>>>>           >>>>>> _not_
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> be an array.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> The current proposal is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> IMHO, this reads a little odd, because
> >>>>>>>>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>>>>>>>           `branch()` does
> >>>>>>>>>>>>>>>>>>>>           >>>>> not
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> take any parameters and has different
> >>>>>>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>>>> than the
> >>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>           >>>>> hidden
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that the first call is `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>>>> the others
> >>>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Because I suggested to rename
> >>>>>>>>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>>>>>>>           I though
> >>>>>>>>>>>>>>>>>>>>           >>>>>> it
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> might be better to also rename
> >>>>>>>>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>> naming
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> overlap that seems to be confusing. The
> >>>>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>>           >>>>> cleaner
> >>>>>>>>>>>>>>>>>>>>           >>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> me:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Maybe there is a better alternative to
> >>>>>>>>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>>>>>>>           avoid
> >>>>>>>>>>>>>>>>>>>>           >>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> naming overlap.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>           >>>>> cannot
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> have
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> a method with such name :-)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Bummer. Didn't consider this. Maybe we
> >>>>>>>>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>>>>>>>           with a
> >>>>>>>>>>>>>>>>>>>>           >>>>> short
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> name?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Can you add the interface
> >>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>>           with all
> >>>>>>>>>>>>>>>>>>>>           >>>>> it's
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> methods? It will be part of public API and
> >>>>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>>>           contained in
> >>>>>>>>>>>>>>>>>>>>           >>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>>>>>>>>>>>           `KBranchedStream#get(int
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> index)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>>>>>>>>>>>           branched-KStreams. Would
> >>>>>>>>>>>>>>>>>>>>           >>>>>> be
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> nice to get your feedback about it. It
> >>>>>>>>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>>>           that users
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> would need to write custom utility code
> >>>>>>>>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>>>>>>>           access them.
> >>>>>>>>>>>>>>>>>>>>           >>>>> We
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>>>>>>>           to get
> >>>>>>>>>>>>>>>>>>>>           >>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> I have updated the KIP-418 according to
> >>>>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> I can see your point: this is to make
> >>>>>>>>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> String#split
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> that also returns an array, right? But
> >>>>>>>>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>>>>>>>           loss of
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> compatibility? We can have overloaded
> >>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>>>>>>>           without
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> affecting
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> the existing code. Maybe the old
> >>>>>>>>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>           >>>>> should
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> be
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> deprecated, but this is a subject for
> >>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>>           >>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Totally agree with 'addBranch->branch'
> >>>>>>>>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> however, a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> reserved word, so unfortunately we
> >>>>>>>>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>>>           with such
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> name
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> :-)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> defaultBranch() does take an
> >>>>>>>>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>>           >>>>> think
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> is not required?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Absolutely! I think that was just
> >>>>>>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>>>>>>>           something.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> please revise the new version of the KIP
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Thanks for driving the discussion of
> >>>>>>>>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> everybody
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> agrees that the current branch() method
> >>>>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>           >>>>>> optimal.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> I had a quick look into the PR and I
> >>>>>>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>>>           proposal.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> There
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> are some minor things we need to
> >>>>>>>>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>           recommend the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> following renaming:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>>>           BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> It's just a suggestion to get slightly
> >>>>>>>>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> In the current PR, defaultBranch() does
> >>>>>>>>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>>>>>>>           `Predicate` as
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>>>>>>>           accepted and
> >>>>>>>>>>>>>>>>>>>>           >>>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Ie, we should add overloads that
> >>>>>>>>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>>>           parameter.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> For the issue that the created
> >>>>>>>>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>>>>>>>           different
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> scopes:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> could we extend `KBranchedStream` with a
> >>>>>>>>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>>>>>>>           index)` method
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> returns the corresponding "branched"
> >>>>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>>>>>>>           object?
> >>>>>>>>>>>>>>>>>>>>           >>>>>> Maybe,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> second argument of `addBranch()` should
> >>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>>>>>>>           >>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> a `Function<KStream,KStream>` and
> >>>>>>>>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>>           whatever
> >>>>>>>>>>>>>>>>>>>>           >>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Finally, I would also suggest to update
> >>>>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>           current
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>           makes sense
> >>>>>>>>>>>>>>>>>>>>           >>>>>> for
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> revise the KIP and continue the
> >>>>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>>>>>>>           we'll
> >>>>>>>>>>>>>>>>>>>>           >>>>> need
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>>>>>>>           >>>>> whether
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> could be adopted.  It would be great
> >>>>>>>>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>>>>>>>           think this
> >>>>>>>>>>>>>>>>>>>>           >>>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> idea overall.  I'm not sure if that
> >>>>>>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>>>>>>>           starting a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> vote,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> or if
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> there is generally some indication of
> >>>>>>>>>>>>>>>>>>>> interest
> >>>>>>>>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> That being said, I'll continue the
> >>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>>>>>>>           assuming
> >>>>>>>>>>>>>>>>>>>>           >>>>> we
> >>>>>>>>>>>>>>>>>>>>           >>>>>> do
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> move
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> forward the solution of "stream.branch()
> >>>>>>>>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>>>>>>>           >>>>>> KBranchedStream",
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> do
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>           >>>>> favor
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> accomplish
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>>>>>>>           similar
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> just need to be sure we're not making
> >>>>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> impossible/difficult
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>>>           >>>>> it's
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> just a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> little sloppy overall in terms of
> >>>>>>>>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> particular,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> passing in the "predicates" and
> >>>>>>>>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>           >>>>>> modified
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>>>>>>>>>>>           KStreamLazyBranch is
> >>>>>>>>>>>>>>>>>>>>           >>>>> a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>           >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> I read your code carefully and now I
> >>>>>>>>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>>>>>>>           convinced: your
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> looks better and should work. We just
> >>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>> crucial
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> fact
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> that KStream consumers are invoked as
> >>>>>>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>>>           And then
> >>>>>>>>>>>>>>>>>>>>           >>>>>> it's
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> all
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> What shall we do now? I should
> >>>>>>>>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>>>>>>>           resume the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Why are you telling that your PR
> >>>>>>>>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>>>>>>>           >>>>> starting
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> point
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> we go in this direction'? To me it
> >>>>>>>>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>>>           starting
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> point.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> as a novice in this project I might
> >>>>>>>>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>>>>>>>           >>>>> details.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> >>>>>>>>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>>>>>>>           >>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>>>>>>>           >>>>> invoked
> >>>>>>>>>>>>>>>>>>>>           >>>>>> as
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> they’re
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> added, not during
> >>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>>>>>>>           still
> >>>>>>>>>>>>>>>>>>>>           >>>>>> ought
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> able to call couponIssuer.coupons()
> >>>>>>>>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           depend on
> >>>>>>>>>>>>>>>>>>>>           >>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> The issue I mean to point out is
> >>>>>>>>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>>>>>>>           access
> >>>>>>>>>>>>>>>>>>>>           >>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> streams in the same scope as the
> >>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>>>>>>>           is, not
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> inside
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>>>           >>>>> solutions.
> >>>>>>>>>>>>>>>>>>>>           >>>>>> It
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> [Also, great to hear additional
> >>>>>>>>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>>>>>>>           excited
> >>>>>>>>>>>>>>>>>>>>           >>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> hear
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>           >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> The idea to postpone the wiring of
> >>>>>>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>>>>>>>> me at
> >>>>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>>>           >>>>> glance,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> but
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>>>>>>>           same
> >>>>>>>>>>>>>>>>>>>>           >>>>>> scope
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> as
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> other.  That is, if we wanted to merge
> >>>>>>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>>>           >>>>> again
> >>>>>>>>>>>>>>>>>>>>           >>>>>> I
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> don't see
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> You just took the words right out
> >>>>>>>>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>>>>>>>           just
> >>>>>>>>>>>>>>>>>>>>           >>>>>> going
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Consider the example from Bill's
> >>>>>>>>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>>>>>>>           we need
> >>>>>>>>>>>>>>>>>>>>           >>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> identify
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> customers who have bought coffee and
> >>>>>>>>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>>>>>>>           in the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> electronics
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> This is the code I usually write under
> >>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>>           circumstances
> >>>>>>>>>>>>>>>>>>>>           >>>>>> using
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>       return
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>       /*In the real world the code
> >>>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>           complex, so
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> creation of
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>>>>>>>           order to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> separate
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>>>>>>>>>>>           >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> >>>>>>>>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>>>>>>>           everything
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> later,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Does this make sense?  In order to
> >>>>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>>>           initialize the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> we need the terminal operation to be
> >>>>>>>>>>>>>>>>>>>> called
> >>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>>>>>>>           essentially
> >>>>>>>>>>>>>>>>>>>>           >>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> next
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> KIP I was going to write here. I have
> >>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>>>>>>>           based on
> >>>>>>>>>>>>>>>>>>>>           >>>>> my
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> experience,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> so I will join the discussion on KIP-401
> >>>>>>>>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>>>>>>>           fluent
> >>>>>>>>>>>>>>>>>>>>           >>>>> API
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>>>>>>>>>>>           (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>>>>>>>>>>>           >>>>>> and
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> I
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> compatibility
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    there aren't any direct ones.
> >>>>>>>>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>>>>>>>           that Java
> >>>>>>>>>>>>>>>>>>>>           >>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> smart
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    distinguish between a
> >>>>>>>>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>>>>>>>           returning one
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> thing
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    with no arguments returning
> >>>>>>>>>>>>>>>>>>>> another
> >>>>>>>>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> >>>>>>>>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>           need
> >>>>>>>>>>>>>>>>>>>>           >>>>> it.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> We
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>>>>>>>           shares
> >>>>>>>>>>>>>>>>>>>>           >>>>>> its
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> >>>>>>>>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           branching.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> It's
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    pretty in its current form, but I
> >>>>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>>>           demonstrates
> >>>>>>>>>>>>>>>>>>>>           >>>>>> its
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>>>>>>>>>>>> request should
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>           >>>>> final
> >>>>>>>>>>>>>>>>>>>>           >>>>>> or
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> even a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> starting point if we go in this
> >>>>>>>>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>>>>>>>           wanted to
> >>>>>>>>>>>>>>>>>>>>           >>>>>> see
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> I will say though, that I'm not
> >>>>>>>>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>           solution
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> could
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> deprecated in favor of this, which
> >>>>>>>>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>>>>>           >>>>> suggested
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> was a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>           streams
> >>>>>>>>>>>>>>>>>>>>           >>>>>> are
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>>>>>>>           is, if we
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> wanted
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> them back together again I don't
> >>>>>>>>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>>>           that.  The
> >>>>>>>>>>>>>>>>>>>>           >>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>           >>>>>> either
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>>>>>>>           table.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>           >>>>>> point.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> First, it seems that it's
> >>>>>>>>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           branch API
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> needs
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> There are two potential ways to
> >>>>>>>>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>>>>>>>           returns
> >>>>>>>>>>>>>>>>>>>>           >>>>>> its
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> >>>>>>>>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>>>>>>>> The code
> >>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>       ��   >>>>> make
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> sense
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> all the necessary ingredients are
> >>>>>>>>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>           instance
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> >>>>>>>>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>> defined.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> >>>>>>>>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>>>>>>>           miss the
> >>>>>>>>>>>>>>>>>>>>           >>>>>> fact
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that one
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> of the terminal methods should be
> >>>>>>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>>>>>>>           methods
> >>>>>>>>>>>>>>>>>>>>           >>>>>> are
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> I see your point when you are
> >>>>>>>>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> >>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>>           implemented the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> easy
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> >>>>>>>>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> >>>>>>>>>>>>>>>>>>>> method that
> >>>>>>>>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>>>>>>>           >>>>> nothing
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> >>>>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>>> the only
> >>>>>>>>>>>>>>>>>>> option
> >>>>>>>>>>>>>>>>>>>>           >>>>>> besides
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>>>>>>>           want to
> >>>>>>>>>>>>>>>>>>>>           >>>>>> just
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> silently
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> >>>>>>>>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>>>>>>>           predicate. 2)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> Throwing
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>           looks
> >>>>>>>>>>>>>>>>>>>>           >>>>>> like a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> bad
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> >>>>>>>>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>>>>>>>           emit a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> >>>>>>>>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> `default`
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>           >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>           a clear
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> >>>>>>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>>>>>>>           >>>>> compiled
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> run?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> >>>>>>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>>>           compile if
> >>>>>>>>>>>>>>>>>>>>           >>>>> used
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>>>>>>>           method chain
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> starting
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> >>>>>>>>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>>>           between
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>>>>>>>           >>>>> instantly
> >>>>>>>>>>>>>>>>>>>>           >>>>>> on
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> >>>>>>>>>>>>>>>>>>>> project
> >>>>>>>>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>>>>>>>           compilation
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>>>>>>>           required.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> But
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> >>>>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>>>>>>>           >>>>>> defaultBranch
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>>>>>>>           just as
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> think it creates an
> >>>>>>>>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>>>>>>>           >>>>> user
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>> default
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>           >>>>> seems
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> like
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>>>>>           which allows
> >>>>>>>>>>>>>>>>>>>>           >>>>>> for
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>>>>>>>           to be
> >>>>>>>>>>>>>>>>>>>>           >>>>>> well
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>>>           >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>>>           a clear
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> error
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>           >>>>> "build
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> step"
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>           >>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>>>>>           >>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> allow users to do other
> >>>>>>>>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>>>>>           stream.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> With
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>> way all
> >>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> operations
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> do -
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> want to process off the original
> >>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>>>           >>>>> times,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> you
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> >>>>>>>>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>>>>>>>           operations
> >>>>>>>>>>>>>>>>>>>>           >>>>>> on
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> it
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> >>>>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>>>           always need
> >>>>>>>>>>>>>>>>>>>>           >>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>>>>>>>           >>>>> don't
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> >>>>>>>>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>>>           so we
> >>>>>>>>>>>>>>>>>>>>           >>>>> can
> >>>>>>>>>>>>>>>>>>>>           >>>>>> do
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> more with the original branch
> >>>>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> >>>>>>>>>>>>>>>>>>>> need of
> >>>>>>>>>>>>>>>>>>> special
> >>>>>>>>>>>>>>>>>>>>           >>>>> object
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>>           >>>>> here
> >>>>>>>>>>>>>>>>>>>>           >>>>>> we
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>>>>>>>           flow,
> >>>>>>>>>>>>>>>>>>>>           >>>>> so
> >>>>>>>>>>>>>>>>>>>>           >>>>>> I
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> >>>>>>>>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>           API, but I
> >>>>>>>>>>>>>>>>>>>>           >>>>>> find
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>>>>>           contrasts the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> fluency
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
> >>>>>>>>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>>>>>           just call
> >>>>>>>>>>>>>>>>>>>>           >>>>> a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> method on
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> >>>>>>>>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>>>>>>>           cases
> >>>>>>>>>>>>>>>>>>>>           >>>>> are
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> I think the
> >>>>>>>>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>>>>>>>           is very
> >>>>>>>>>>>>>>>>>>>>           >>>>>> nice
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> >>>>>>>>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>           how we
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> specify
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>>>>>>>           terminated by
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>>>>>>>           incompatible with
> >>>>>>>>>>>>>>>>>>>>           >>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> >>>>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>           different
> >>>>>>>>>>>>>>>>>>>>           >>>>>> name,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> >>>>>>>>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>>>>>>>           >>>>>> something
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> >>>>>>>>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>           KIP?  It
> >>>>>>>>>>>>>>>>>>>>           >>>>>> seems
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>>>           while also
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> allowing
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>>>>>>>           KBranchedStreams
> >>>>>>>>>>>>>>>>>>>>           >>>>>> if
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>>>>>>>           ks){
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>>>>>>>           String> ks){
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>>>           this::handleFirstCase)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>>>           this::handleSecondCase)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>>>           >>>>> takes a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Consumer
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>           example in
> >>>>>>>>>>>>>>>>>>>>           >>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>>>>>>>> how would
> >>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>           >>>>> handle
> >>>>>>>>>>>>>>>>>>>>           >>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>>>           >>>>>> stream
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>>>>>>>           we had
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>>>>>>>           >>>>> 418.
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>>>>>>>           >>>>> Please
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>> take
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>>>>>>>           feedback :)
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>>>>>>>           >>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>>>>>>>           >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>           >
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>> Attachments:
> >>>>>> * signature.asc
> >>>>
> >>>>
> >>
> >>
> >>
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi John,

 > I'd wonder whether we need the non-static builders (like withChain). 
Do they provide any benefit over just using the right static factory?

I don't have a strong opinion here. I think it's just a matter of taste. 
But, if we like to use Occam's razor, then yes, non-static builders can 
be omitted, I agree!

 > You might as well propose the “ideal” API in the KIP, which is the 
covariant typed function

I didn't quite get it. Do I get you right that you propose the 
following: 1) accept the 'fully covariant' definition in the KIP, 2) 
then just see if there are any obstacles/pitfalls during implementation 
and unit testing?

 > Is it necessary to restrict the result key and value types to be the 
same as the inputs?

Good question, I already thought about it and rejected the idea.

Look, if we want to keep `withChain`'s function optional, then we must 
keep the result key and value types the same. Because for now, the 
default value for the 'chain function' is Function.identity().

Of course, we can make the 'chain function' required. But I think this 
is not what `split` method is for. `split` is for splitting, not 
transforming, and `chainFunction` in most of the cases should be either 
a consumer or the identity function.

Regards,

Ivan


24.05.2020 17:15, John Roesler пишет:
> Thanks for the reply, Ivan.
> 
> 1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function).
> 
> I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?
> 
> 2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to.
> 
> I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:
> 
> Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>
> 
> 3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?
> 
> Thanks,
> John
> 
> On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
>> Hello John,
>>
>>
>> 1.
>> ---------------------------------------------
>>
>>   > Perhaps it would be better to stick with "as" for now
>>   > and just file a Jira to switch them all at the same time [for
>> compatibility with Kotlin]
>>
>> Fully agree! BTW it's really not a big problem: in Kotlin they have a
>> standard workaround
>> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin).
>> So actually this should be a very low priority issue, if an issue at
>> all.
>>
>>   > I don't understand how your new proposed
>>   > methods would work any differently than the ones you already
>>   > had proposed in the KIP. It seems like you'd still have to provide
>>   > the generic type parameters on the first static factory call. Can you
>>   > explain how your new interface proposal differs from the existing KIP?
>>
>> In the KIP, I didn't clarify what methods should be static. Now I
>> propose the following methods:
>>
>> non-static: withChain(Function), withName(String).
>>
>> static: as(String), with(Function), with(Function, String).
>>
>> The overloaded `with` version that provides both Function and name can
>> be used without causing type inference problem!!
>>
>> 2.
>> ----------------------------
>>
>>   > Regarding making the K,V types covariant also, yes, that would indeed
>>   > be nice, but I'm not sure it will actually work.
>>
>> What I'm keeping in mind is the following example: imagine
>>
>> static KStream<String, Integer> func(KStream<String, Number> s) {
>>           return s.mapValues(n -> (Integer) n + 1);
>> }
>>
>> BranchedKStream<String, Number> b =
>>       s.split().branch((k, v) -> isInteger(v),
>>                  //Won't compile!!
>>                  Branched.with(Me::func));
>>
>> The simple workaround here is to change `func`'s return type from
>> KStream<...Integer> to KStream<...Number>.
>>
>> [On the other hand, we already agreed to remove `withJavaConsumer` from
>> `Branched`, so during code migration I will have to modify my functions'
>> return types anyway -- I mean, from `void` to `KStream`!! ]
>>
>>   >  the map you're returning is Map<K,V>, and of course a K is not the
>> same as "? extends K", so it doesn't seem compatible.
>>
>> I think what you actually meant here is that KStream<? extends K, ?
>> extends V> is not fit as a value for Map<String, KStream<K, V>>. This
>> particularly is not a problem, since KStream<? extends K, ? extends V>
>> can be safely explicitly cast to KStream<K, V>, and be put to the map.
>>
>> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe
>> for now it's better to just admit that API is not absolutely perfect and
>> accept it as is, that is
>>
>> Function<? super KStream<K, V>, ? extends KStream<K, V>>
>>
>> Regards,
>>
>> Ivan
>>
>>
>> 21.05.2020 17:59, John Roesler пишет:
>>> Hello Ivan,
>>>
>>> Thanks for the refinement. Actually, I did not know that "as" would
>>> clash with a Kotlin operator. Maybe we should depart from convention
>>> and just avoid methods named "as" in the future.
>>>
>>> The convention is that "as(String name)" is used for the static factory
>>> method, whereas "withName(String name)" is an instance method
>>> inherited from NamedOperation. If you wish to propose to avoid "as"
>>> for compatibility with Kotlin, I might suggest "fromName(String name)",
>>> although it's somewhat dubious, since all the other configuration
>>> classes use "as". Perhaps it would be better to stick with "as" for now
>>> and just file a Jira to switch them all at the same time.
>>>
>>> Re. 3:
>>> Regarding the type inference problem, yes, it's a blemish on all of our
>>> configuraion objects. The problem is that Java infers the type
>>> based on the _first_ method in the chain. While it does consider what
>>> the recipient of the method result wants, it only considers the _next_
>>> recipient.
>>>
>>> Thus, if you call as("foo") and immediately assign it to a
>>> Branched<String,String> variable, java infers the type correctly. But
>>> when the "next recipient" is a chained method call, like "withChain",
>>> then the chained method doesn't bound the type (by definition,
>>> withChain is defined on Branched<Object, Object>, so Java will take
>>> the broadest possible inferece and bind the type to
>>> Branched<Object, Object>, at which point, it can't be revised anymore.
>>>
>>> As a user of Java, this is exceedingly annoying, since it doesn't seem
>>> that hard to recursively consider the entire context when inferring the
>>> generic type parameters, but this is what we have to work with.
>>>
>>> To be honest, though, I don't understand how your new proposed
>>> methods would work any differently than the ones you already
>>> had proposed in the KIP. It seems like you'd still have to provide
>>> the generic type parameters on the first static factory call. Can you
>>> explain how your new interface proposal differs from the existing KIP?
>>>
>>> Re. 4:
>>> Regarding making the K,V types covariant also, yes, that would indeed
>>> be nice, but I'm not sure it will actually work. You might want to give it a
>>> try. In the past, we've run into soe truly strange interactions between the
>>> Java type inferencer and lambdas (and/or anonymous inner classes) in
>>> combination with nested covariant types.
>>>
>>> Another issue is that the value type of the map you're returning is
>>> Map<K,V>, and of course a K is not the same as "? extends K", so it
>>> doesn't seem compatible.
>>>
>>> Thanks again,
>>> -John
>>>
>>> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>>>> Hi,
>>>>
>>>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>>>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>>>> simpler'.
>>>>
>>>> I made some quick API mocking in my IDE and tried to implement examples
>>>> from KIP.
>>>>
>>>> 1. Having to return something from lambda is not a very big deal.
>>>>
>>>> 2. For a moment I thouht that I won't be able to use method references
>>>> for already written stream consumers, but then I realized that I can
>>>> just change my methods from returning void to returning the input
>>>> parameter and use references to them. Not very convenient, but passable.
>>>>
>>>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>>>> function returns null, we don't insert it into the resulting map.
>>>>
>>>> Usually it's better to implement a non-perfect, but workable solution as
>>>> a first approximation. And later we can always add to `Branched`
>>>> anything we want.
>>>>
>>>> 3. Do we have any guidelines on how parameter classes like Branched
>>>> should be built? First of all, it seems that `as` now is more preferred
>>>> than `withName` (although as you probably know it clashes with Kotlin's
>>>> `as` operator).
>>>>
>>>> Then, while trying to mock the APIs, I found out that my Java cannot
>>>> infer types in the following construction:
>>>>
>>>> .branch((key, value) -> value == null,
>>>>       Branched.as("foo").withChain(s -> s.mapValues(...)))
>>>>
>>>>
>>>> so I have to write
>>>>
>>>> .branch((key, value) -> value == null,
>>>>       Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>>>
>>>>
>>>> This is not tolerable IMO, so this is the list of `Branched` methods
>>>> that I came to (will you please validate it):
>>>>
>>>> static <K, V> Branched<K, V> as(String name);
>>>>
>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>> extends KStream<K, V>> chain);
>>>>
>>>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>>>> extends KStream<K, V>> chain, String name);
>>>>
>>>> //non-static!
>>>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>>>> KStream<K, V>> chain);
>>>>
>>>>
>>>> 4. And one more. What do you think, do we need that flexibility:
>>>>
>>>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>>>
>>>> vs.
>>>>
>>>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>>>> extends K, ? extends V>> chain
>>>>
>>>> ??
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>>
>>>> 21.05.2020 6:54, John Roesler пишет:
>>>>> Thanks for this thought, Matthias,
>>>>>
>>>>> Your idea has a few aspects I find attractive:
>>>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>>>
>>>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>>>
>>>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>>>
>>>>> Thanks again for sharing the idea,
>>>>> John
>>>>>
>>>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>>>> Thanks for updating the KIP!
>>>>>>
>>>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>>>> its relationship to the returned `Map`.
>>>>>>
>>>>>> Originally, we discussed two main patterns:
>>>>>>
>>>>>>     (1) split a stream and return the substreams for futher processing
>>>>>>     (2) split a stream and modify the substreams with in-place method chaining
>>>>>>
>>>>>> To combine both patterns we wanted to allow for
>>>>>>
>>>>>>      -> split a stream, modify the substreams, and return the _modified_
>>>>>> substreams for further processing
>>>>>>
>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>>>
>>>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>>>
>>>>>>     - using `withChain` I get the modified sub-stream
>>>>>>     - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>>>
>>>>>> This seems to be quite subtle to me.
>>>>>>
>>>>>>
>>>>>>
>>>>>>    From my understanding the original idea of `withJavaConsumer` was to
>>>>>> model a terminal operation, ie, it should be similar to:
>>>>>>
>>>>>> Branched.withChain(s -> {
>>>>>>      s.to();
>>>>>>      return null;
>>>>>> })
>>>>>>
>>>>>> However, I am not sure if we should even allow `withChain()` to return
>>>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>>>> -> null` entry in the returned Map.
>>>>>>
>>>>>> Following this train of through, and if we want to allow the "return
>>>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>>>> an entry to the Map.
>>>>>>
>>>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>>>> be achieved with `withChain`:
>>>>>>
>>>>>> Branched.withChain(s -> {
>>>>>>      s.to();
>>>>>>      return s;
>>>>>> })
>>>>>>
>>>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>>>> while for the first proposal it adds new functionality (if `return null`
>>>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>>>
>>>>>>
>>>>>>
>>>>>> I guess I can be convinced either way. However, if we follow your
>>>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>>>> benefit seems to be small? Also, having a reduced API is usually
>>>>>> preferable as it's simpler to learn.
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>>>> Hello, John, hello Matthias!
>>>>>>>
>>>>>>> Thank you very much for your detailed feedback!
>>>>>>>
>>>>>>> -----------------------------------------
>>>>>>>
>>>>>>> John,
>>>>>>>
>>>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>>>
>>>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>>>> all the emails on the web.
>>>>>>>
>>>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>>>> method?
>>>>>>>
>>>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>>>
>>>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>>>
>>>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>>>
>>>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>>>
>>>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>>>
>>>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>>>> chain and the result map OR using just the sink
>>>>>>>
>>>>>>> This is discussed below.
>>>>>>>
>>>>>>> ----------------------------------------------
>>>>>>>
>>>>>>> Mathhias,
>>>>>>>
>>>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>>>
>>>>>>> Done.
>>>>>>>
>>>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>>>> method]
>>>>>>>
>>>>>>> Fixed.
>>>>>>>
>>>>>>>
>>>>>>>> 3. Overview of newly added methods/interfaces
>>>>>>>
>>>>>>> Done in `Proposed Changes` section.
>>>>>>>
>>>>>>>
>>>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>>>> `withChain()` and `withConsumer()` is the
>>>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>>>> be given to the `Consumer`.
>>>>>>>
>>>>>>> Yes, I agree!
>>>>>>>
>>>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>>>> implementation would be to not add the "branch" to the result map if
>>>>>>> `withConsumer` is used?
>>>>>>>
>>>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>>>
>>>>>>> In the current version of KIP it is assumed that the returned map
>>>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>>>> map is the user's responsibility.
>>>>>>>
>>>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>>>> general rule, like 'swallowing' some streams by provided
>>>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>>>> it as it is?
>>>>>>>
>>>>>>> ----
>>>>>>>
>>>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>>>
>>>>>>> Looking forward for your feedback again!
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>>>> Thanks for updating the KIP!
>>>>>>>>
>>>>>>>> I also have some minor comment:
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>>>
>>>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>>>> so we just keep them.)
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> (2) Quote:
>>>>>>>>
>>>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>>>> parameterless alternatives.
>>>>>>>>
>>>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>>>> as `branch()` would not be "parameterless".
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>>>> newly added and deprecated methods/classes (cf.
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>>>> 4th comment:
>>>>>>>>
>>>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>>>> chain and the result map OR using just the sink.
>>>>>>>>
>>>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>>>
>>>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>>>> it in the JavaDocs, this might be fine?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> (5) Reply to John's comments:
>>>>>>>>
>>>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>>>
>>>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>>>> node", ie., writing the KStream to a topic.
>>>>>>>>
>>>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>>>> Hi Ivan,
>>>>>>>>>
>>>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>>>> but I had a few last comments.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> John
>>>>>>>>>
>>>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>>>> Hello everyone,
>>>>>>>>>>
>>>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>>>
>>>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>>>> all the arguments discussed here.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>>
>>>>>>>>>> Ivan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>>>> rewritten the KIP, see
>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>>>> the design principles, with one exception.
>>>>>>>>>>>
>>>>>>>>>>> The exception is
>>>>>>>>>>>
>>>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>>>
>>>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>>>> here it is justified.
>>>>>>>>>>>
>>>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>>>
>>>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>>>> as it
>>>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>>>
>>>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>
>>>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>>>
>>>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>>>> `Named`
>>>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>>>> build on
>>>>>>>>>>>> this.
>>>>>>>>>>>>
>>>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>>>> want to follow:
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>>>
>>>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>>>> June
>>>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>>>
>>>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>>>> this
>>>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>>>> the
>>>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>>>
>>>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>>>          .branch(....)
>>>>>>>>>>>>>          .defaultBranch(result::set)
>>>>>>>>>>>>>          .onTopOf(someStream);
>>>>>>>>>>>>> result.get()...
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>>>> from
>>>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>>>
>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>        -> KBranchedStream
>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>> //
>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>>>        -> KBranchedStream
>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>>
>>>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>>>> a good
>>>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>>>> you
>>>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>>>> Or,
>>>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>>>> finally,
>>>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>>>> original scope.
>>>>>>>>>>>>>
>>>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>> at any point.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>>>> do so.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>>>> to mix
>>>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           withChain(...);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>>>> check
>>>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>>>> object
>>>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>>>                     .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>>>                     .defaultBranch(...) // creates node
>>>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>           operator(function, config_object?) OR
>>>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         operator(function, function, string)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>          interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>>>           withPredicate(...);
>>>>>>>>>>>>>>>>>           withChain(...);
>>>>>>>>>>>>>>>>>           withName(...);
>>>>>>>>>>>>>>>>>         }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>>>         -> KBranchedStream
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>>>         -> KBranchedStream
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>>>         -> Map<String,KStream>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>>>           for the dynamic case.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>>>           question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>>>           in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>           seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>>>           when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>>           for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>>>           solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>>>           to.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>           <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>>>           LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>>>           matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>>>           advantages than drawbacks.
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>>>           proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>>>           But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>>>           workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>>>           method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>>>           tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>>>           going to be clean.
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>>>           handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>>>           a handler?
>>>>>>>>>>>>>>>>>>>>           > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>>>           > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>           writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>>>           "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>>>           identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>>>           all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>>>           point?
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > ---
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>           without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>>>           the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>>>           current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>>>           me address both comments here.
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>>>           for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>>>           see example below.
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>>>           Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>>>           value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>>>           have been much messier.*/
>>>>>>>>>>>>>>>>>>>>           > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>>>           > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>>>           >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>>>           recordType,
>>>>>>>>>>>>>>>>>>>>           >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > Regards,
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > Ivan
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>           > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>           >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>>>           >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>>>           thinking
>>>>>>>>>>>>>>>>>>>>           >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>>>           >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>>>           >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>           >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>           >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>           >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>>>           >> An open question is the case for which no
>>>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>           >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>           `BranchedKStream`
>>>>>>>>>>>>>>>>>>>>           >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>>>           >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>>>           >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>>>           `defaultBranch()`
>>>>>>>>>>>>>>>>>>>>           >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>>>           >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>>>           argument" yet
>>>>>>>>>>>>>>>>>>>>           >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >> -Matthias
>>>>>>>>>>>>>>>>>>>>           >>
>>>>>>>>>>>>>>>>>>>>           >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>           >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>           that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>>>           it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>>>           The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>           (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>>>           to each other.
>>>>>>>>>>>>>>>>>>>>           >>>
>>>>>>>>>>>>>>>>>>>>           >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>>>           branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>>>           >>>
>>>>>>>>>>>>>>>>>>>>           >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>>>           <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>           <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>
>>>>>>>>>>>>>>>>>>>>           >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>>>           downstream
>>>>>>>>>>>>>>>>>>>>           >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>           quickly become
>>>>>>>>>>>>>>>>>>>>           >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>>>           >>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>>>           <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>>           that sets a
>>>>>>>>>>>>>>>>>>>>           >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>>>           current
>>>>>>>>>>>>>>>>>>>>           >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>>>           decisions) is
>>>>>>>>>>>>>>>>>>>>           >>>>> roughly this:
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>           >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>           >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>>>           >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>           construct
>>>>>>>>>>>>>>>>>>>>           >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>           provides as much
>>>>>>>>>>>>>>>>>>>>           >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>           call
>>>>>>>>>>>>>>>>>>>>           >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>>>           The value it
>>>>>>>>>>>>>>>>>>>>           >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>>>           the same
>>>>>>>>>>>>>>>>>>>>           >>>>> scope.
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>           in the sense
>>>>>>>>>>>>>>>>>>>>           >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>           branches, but it is
>>>>>>>>>>>>>>>>>>>>           >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>>>           the "static"
>>>>>>>>>>>>>>>>>>>>           >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>>>           fluently declare and
>>>>>>>>>>>>>>>>>>>>           >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>>>           ignore a
>>>>>>>>>>>>>>>>>>>>           >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>>>           SortedMap
>>>>>>>>>>>>>>>>>>>>           >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>>>           SortedMap being
>>>>>>>>>>>>>>>>>>>>           >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>>>           like this:
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>           >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>           >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>>>           >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>>>           >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>>>           >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>           readable than
>>>>>>>>>>>>>>>>>>>>           >>>>> SortedMap
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Cons
>>>>>>>>>>>>>>>>>>>>           >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>           makes it harder
>>>>>>>>>>>>>>>>>>>>           >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>>>           unlike the KIP)
>>>>>>>>>>>>>>>>>>>>           >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>>>           existing
>>>>>>>>>>>>>>>>>>>>           >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>>>           that's overdoing
>>>>>>>>>>>>>>>>>>>>           >>>>> it).
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>>>           access the
>>>>>>>>>>>>>>>>>>>>           >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>>           possible that it
>>>>>>>>>>>>>>>>>>>>           >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>           left up to the
>>>>>>>>>>>>>>>>>>>>           >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> Paul
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>>>           <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>           >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>           existing
>>>>>>>>>>>>>>>>>>>>           >>>>> branch
>>>>>>>>>>>>>>>>>>>>           >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>>>           proceed by
>>>>>>>>>>>>>>>>>>>>           >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>           important to be
>>>>>>>>>>>>>>>>>>>>           >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>>>           >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>>>           this KIP.
>>>>>>>>>>>>>>>>>>>>           >>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>           double brace
>>>>>>>>>>>>>>>>>>>>           >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>>>           >>>>> avoid...just
>>>>>>>>>>>>>>>>>>>>           >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>>>           >>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>           >>>>>> Sophie
>>>>>>>>>>>>>>>>>>>>           >>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>>>           >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>>>           <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>>>           To me,
>>>>>>>>>>>>>>>>>>>>           >>>>> there
>>>>>>>>>>>>>>>>>>>>           >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>           >>>>>> warnings.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>>>           positionally coupled
>>>>>>>>>>>>>>>>>>>>           >>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>           additional code
>>>>>>>>>>>>>>>>>>>>           >>>>>>> paths.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>>>           constructs
>>>>>>>>>>>>>>>>>>>>           >>>>>> would
>>>>>>>>>>>>>>>>>>>>           >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>>>           looks like
>>>>>>>>>>>>>>>>>>>>           >>>>>> this:
>>>>>>>>>>>>>>>>>>>>           >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>>>           Predicate<?
>>>>>>>>>>>>>>>>>>>>           >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>>>           >>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>>>           returns a
>>>>>>>>>>>>>>>>>>>>           >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>>>           >>>>>> maintained
>>>>>>>>>>>>>>>>>>>>           >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>           >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>>>           solves
>>>>>>>>>>>>>>>>>>>>           >>>>>> problem
>>>>>>>>>>>>>>>>>>>>           >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>>>           branch you’re
>>>>>>>>>>>>>>>>>>>>           >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>           another
>>>>>>>>>>>>>>>>>>>>           >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>>>           structure, rather
>>>>>>>>>>>>>>>>>>>>           >>>>>> than
>>>>>>>>>>>>>>>>>>>>           >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>>>           historically
>>>>>>>>>>>>>>>>>>>>           >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>>>           voluminously, but
>>>>>>>>>>>>>>>>>>>>           >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>>>           <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>>>           >>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>>>           >>>>> operator
>>>>>>>>>>>>>>>>>>>>           >>>>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>>>           "Split"
>>>>>>>>>>>>>>>>>>>>           >>>>> seems
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>>>           >>>>> branching"
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>           >>>>> terminal.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> That
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>>>           is, `branch`
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>>>           `void`, to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>>>           definition of
>>>>>>>>>>>>>>>>>>>>           >>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>>>           there's no
>>>>>>>>>>>>>>>>>>>>           >>>>>>> default,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>>>           record
>>>>>>>>>>>>>>>>>>>>           >>>>>> falls
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>>>           >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>>>           should
>>>>>>>>>>>>>>>>>>>>           >>>>>> _not_
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>>>           `branch()` does
>>>>>>>>>>>>>>>>>>>>           >>>>> not
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>           >>>>> hidden
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>>>           I though
>>>>>>>>>>>>>>>>>>>>           >>>>>> it
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>> naming
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>           >>>>> cleaner
>>>>>>>>>>>>>>>>>>>>           >>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> me:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>>>           avoid
>>>>>>>>>>>>>>>>>>>>           >>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>           >>>>> cannot
>>>>>>>>>>>>>>>>>>>>           >>>>>>> have
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>>>           with a
>>>>>>>>>>>>>>>>>>>>           >>>>> short
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> name?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>>           with all
>>>>>>>>>>>>>>>>>>>>           >>>>> it's
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>>>           contained in
>>>>>>>>>>>>>>>>>>>>           >>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>>>           `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>>>           >>>>>>> index)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>>>           branched-KStreams. Would
>>>>>>>>>>>>>>>>>>>>           >>>>>> be
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>>>           that users
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>>>           access them.
>>>>>>>>>>>>>>>>>>>>           >>>>> We
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>>>           to get
>>>>>>>>>>>>>>>>>>>>           >>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>>>           >>>>>>> String#split
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>>>           loss of
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>>>           without
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>           >>>>> should
>>>>>>>>>>>>>>>>>>>>           >>>>>>> be
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>>           >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>>           >>>>>>> however, a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>>>           with such
>>>>>>>>>>>>>>>>>>>>           >>>>>>> name
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>           >>>>> think
>>>>>>>>>>>>>>>>>>>>           >>>>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>>>           something.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>> everybody
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>           >>>>>> optimal.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>           proposal.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> There
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>           recommend the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>>>           BranchingKStream#default()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>>>           `Predicate` as
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>>>           accepted and
>>>>>>>>>>>>>>>>>>>>           >>>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>>>           parameter.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>>>           different
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>>>           index)` method
>>>>>>>>>>>>>>>>>>>>           >>>>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>>>           object?
>>>>>>>>>>>>>>>>>>>>           >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>>>           >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>           whatever
>>>>>>>>>>>>>>>>>>>>           >>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>           current
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>           makes sense
>>>>>>>>>>>>>>>>>>>>           >>>>>> for
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> you
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>>>           we'll
>>>>>>>>>>>>>>>>>>>>           >>>>> need
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> some
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>>>           >>>>> whether
>>>>>>>>>>>>>>>>>>>>           >>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>>>           think this
>>>>>>>>>>>>>>>>>>>>           >>>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>>> a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>>>           starting a
>>>>>>>>>>>>>>>>>>>>           >>>>>>> vote,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>>>           assuming
>>>>>>>>>>>>>>>>>>>>           >>>>> we
>>>>>>>>>>>>>>>>>>>>           >>>>>> do
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> move
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>>>           >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>>>           >>>>>>> do
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>           >>>>> favor
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>>>           similar
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>           >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>>>           >>>>> it's
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> just a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>>>           >>>>>>> particular,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>           >>>>>> modified
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> in
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>>>           KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>>>           >>>>> a
>>>>>>>>>>>>>>>>>>>>           >>>>>>> bit
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>           >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>>>           convinced: your
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>> crucial
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>>>           And then
>>>>>>>>>>>>>>>>>>>>           >>>>>> it's
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> all
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>>>           resume the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>>>           >>>>> starting
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> point
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>>>           starting
>>>>>>>>>>>>>>>>>>>>           >>>>>>> point.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>           >>>>> details.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>>>           >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>>>           >>>>> invoked
>>>>>>>>>>>>>>>>>>>>           >>>>>> as
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>>>           still
>>>>>>>>>>>>>>>>>>>>           >>>>>> ought
>>>>>>>>>>>>>>>>>>>>           >>>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           depend on
>>>>>>>>>>>>>>>>>>>>           >>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>>>           access
>>>>>>>>>>>>>>>>>>>>           >>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>>>           is, not
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> inside
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>           >>>>> solutions.
>>>>>>>>>>>>>>>>>>>>           >>>>>> It
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>>>           excited
>>>>>>>>>>>>>>>>>>>>           >>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> hear
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>           >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>>>           >>>>> glance,
>>>>>>>>>>>>>>>>>>>>           >>>>>>> but
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>>>           same
>>>>>>>>>>>>>>>>>>>>           >>>>>> scope
>>>>>>>>>>>>>>>>>>>>           >>>>>>> as
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>>>           >>>>> again
>>>>>>>>>>>>>>>>>>>>           >>>>>> I
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>>>           just
>>>>>>>>>>>>>>>>>>>>           >>>>>> going
>>>>>>>>>>>>>>>>>>>>           >>>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>>>           we need
>>>>>>>>>>>>>>>>>>>>           >>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>>>           in the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>>           circumstances
>>>>>>>>>>>>>>>>>>>>           >>>>>> using
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> my
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>           complex, so
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>>>           order to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> separate
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>>>           >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>>>           everything
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> later,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>>>           initialize the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>>>           essentially
>>>>>>>>>>>>>>>>>>>>           >>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>>>           based on
>>>>>>>>>>>>>>>>>>>>           >>>>> my
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>>>           fluent
>>>>>>>>>>>>>>>>>>>>           >>>>> API
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> based
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>>>           (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>>>           >>>>>> and
>>>>>>>>>>>>>>>>>>>>           >>>>>>> I
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>>>           >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>>>           that Java
>>>>>>>>>>>>>>>>>>>>           >>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> smart
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>>>           returning one
>>>>>>>>>>>>>>>>>>>>           >>>>>>> thing
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>           need
>>>>>>>>>>>>>>>>>>>>           >>>>> it.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> We
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>>>           shares
>>>>>>>>>>>>>>>>>>>>           >>>>>> its
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           branching.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> It's
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>>>           demonstrates
>>>>>>>>>>>>>>>>>>>>           >>>>>> its
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>           >>>>> final
>>>>>>>>>>>>>>>>>>>>           >>>>>> or
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>>>           wanted to
>>>>>>>>>>>>>>>>>>>>           >>>>>> see
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> how
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>           solution
>>>>>>>>>>>>>>>>>>>>           >>>>>>> could
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> be
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>>>           >>>>> suggested
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> was a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>           streams
>>>>>>>>>>>>>>>>>>>>           >>>>>> are
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>>>           is, if we
>>>>>>>>>>>>>>>>>>>>           >>>>>>> wanted
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>>>           that.  The
>>>>>>>>>>>>>>>>>>>>           >>>>>> KIP
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>           >>>>>> either
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>>>           table.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>           >>>>>> point.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>           branch API
>>>>>>>>>>>>>>>>>>>>           >>>>>>> needs
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>>>           returns
>>>>>>>>>>>>>>>>>>>>           >>>>>> its
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>       ��   >>>>> make
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> sense
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>           instance
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>> defined.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>>>           miss the
>>>>>>>>>>>>>>>>>>>>           >>>>>> fact
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>>>           methods
>>>>>>>>>>>>>>>>>>>>           >>>>>> are
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> not
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>>           implemented the
>>>>>>>>>>>>>>>>>>>>           >>>>>>> easy
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>>>           >>>>> nothing
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> will
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>>>           >>>>>> besides
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>>>           want to
>>>>>>>>>>>>>>>>>>>>           >>>>>> just
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>>>           predicate. 2)
>>>>>>>>>>>>>>>>>>>>           >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>           looks
>>>>>>>>>>>>>>>>>>>>           >>>>>> like a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> bad
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>>>           emit a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> special
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>           >>>>>>> `default`
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>           >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>           >>>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>           a clear
>>>>>>>>>>>>>>>>>>>>           >>>>>>> error
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>>>           >>>>> compiled
>>>>>>>>>>>>>>>>>>>>           >>>>>>> and
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>>>           compile if
>>>>>>>>>>>>>>>>>>>>           >>>>> used
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>>>           method chain
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> starting
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>>>           between
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>>>           >>>>> instantly
>>>>>>>>>>>>>>>>>>>>           >>>>>> on
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>>>           compilation
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>>>           required.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> But
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>>>           >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>>>           just as
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>>>           >>>>> user
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> could
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>> default
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>           >>>>> seems
>>>>>>>>>>>>>>>>>>>>           >>>>>>> like
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>           which allows
>>>>>>>>>>>>>>>>>>>>           >>>>>> for
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>>>           to be
>>>>>>>>>>>>>>>>>>>>           >>>>>> well
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>>>           >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>>>           >>>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> track
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>>>           a clear
>>>>>>>>>>>>>>>>>>>>           >>>>>>> error
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>           >>>>> "build
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> step"
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>           >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>>>           >>>>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>>>           >>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>> it's
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>>>           stream.
>>>>>>>>>>>>>>>>>>>>           >>>>>>> With
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>           >>>>>>> operations
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>>>           >>>>> times,
>>>>>>>>>>>>>>>>>>>>           >>>>>>> you
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>>>           operations
>>>>>>>>>>>>>>>>>>>>           >>>>>> on
>>>>>>>>>>>>>>>>>>>>           >>>>>>> it
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>           always need
>>>>>>>>>>>>>>>>>>>>           >>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>>>           >>>>> don't
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> know
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>>>           so we
>>>>>>>>>>>>>>>>>>>>           >>>>> can
>>>>>>>>>>>>>>>>>>>>           >>>>>> do
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>>>           >>>>> object
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>>           >>>>> here
>>>>>>>>>>>>>>>>>>>>           >>>>>> we
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>>>           flow,
>>>>>>>>>>>>>>>>>>>>           >>>>> so
>>>>>>>>>>>>>>>>>>>>           >>>>>> I
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>           API, but I
>>>>>>>>>>>>>>>>>>>>           >>>>>> find
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>>>           contrasts the
>>>>>>>>>>>>>>>>>>>>           >>>>>>> fluency
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>>>           just call
>>>>>>>>>>>>>>>>>>>>           >>>>> a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>>>           cases
>>>>>>>>>>>>>>>>>>>>           >>>>> are
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>>>           is very
>>>>>>>>>>>>>>>>>>>>           >>>>>> nice
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>           how we
>>>>>>>>>>>>>>>>>>>>           >>>>>>> specify
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>>>           terminated by
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>>>           incompatible with
>>>>>>>>>>>>>>>>>>>>           >>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>           different
>>>>>>>>>>>>>>>>>>>>           >>>>>> name,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> but
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>>>           >>>>>> something
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> like
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>           KIP?  It
>>>>>>>>>>>>>>>>>>>>           >>>>>> seems
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>>>           while also
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>>>           KBranchedStreams
>>>>>>>>>>>>>>>>>>>>           >>>>>> if
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>>>           ks){
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>>>           String> ks){
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>>>           this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>>>           this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>>>           >>>>> takes a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>           example in
>>>>>>>>>>>>>>>>>>>>           >>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>           >>>>> handle
>>>>>>>>>>>>>>>>>>>>           >>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>>>           >>>>>> stream
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>>>           we had
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> something
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>>>           >>>>> 418.
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>>>           >>>>> Please
>>>>>>>>>>>>>>>>>>>>           >>>>>>>> take
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>>>           feedback :)
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>>>           >>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>>>           >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >>>>>>>>>
>>>>>>>>>>>>>>>>>>>>           >
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>> Attachments:
>>>>>> * signature.asc
>>>>
>>>>
>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Thanks for the reply, Ivan.

1. Ah, now I get it. Yes, AFAICT, the only way to get past that generic type inference problem is to offer a static factory method that takes all the options (both name and function). 

I’m happy with adding that option, although if we have all possible combinations available as static methods, then I’d wonder whether we need the non-static builders (like withChain). Do they provide any benefit over just using the right static factory?

2. Sorry for my ambiguity. Yes, the covariant function type bounds works algebraically, but I’m not sure whether the limitations of Java type inference will actually let you pass in all the different functions you would want to. 

I recall Bruno and I wrestling with this when he added Transform operators. In retrospect, this is a problem we can iron out when we write tests in the PR. You might as well propose the “ideal” API in the KIP, which is the covariant typed function:

Function<? super KStream<? super K, ? super V>, ? extends KStream<? extends K, ? extends V>>

3. Actually, I have a new question about the types: Is it necessary to restrict the result key and value types to be the same as the inputs? I.e., shouldn’t the result types be K1,V1?

Thanks,
John

On Fri, May 22, 2020, at 21:09, Ivan Ponomarev wrote:
> Hello John,
> 
> 
> 1.
> ---------------------------------------------
> 
>  > Perhaps it would be better to stick with "as" for now
>  > and just file a Jira to switch them all at the same time [for 
> compatibility with Kotlin]
> 
> Fully agree! BTW it's really not a big problem: in Kotlin they have a 
> standard workaround 
> (https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin). 
> So actually this should be a very low priority issue, if an issue at 
> all.
> 
>  > I don't understand how your new proposed
>  > methods would work any differently than the ones you already
>  > had proposed in the KIP. It seems like you'd still have to provide
>  > the generic type parameters on the first static factory call. Can you
>  > explain how your new interface proposal differs from the existing KIP?
> 
> In the KIP, I didn't clarify what methods should be static. Now I 
> propose the following methods:
> 
> non-static: withChain(Function), withName(String).
> 
> static: as(String), with(Function), with(Function, String).
> 
> The overloaded `with` version that provides both Function and name can 
> be used without causing type inference problem!!
> 
> 2.
> ----------------------------
> 
>  > Regarding making the K,V types covariant also, yes, that would indeed
>  > be nice, but I'm not sure it will actually work.
> 
> What I'm keeping in mind is the following example: imagine
> 
> static KStream<String, Integer> func(KStream<String, Number> s) {
>          return s.mapValues(n -> (Integer) n + 1);
> }
> 
> BranchedKStream<String, Number> b =
>      s.split().branch((k, v) -> isInteger(v),
>                 //Won't compile!!
>                 Branched.with(Me::func));
> 
> The simple workaround here is to change `func`'s return type from 
> KStream<...Integer> to KStream<...Number>.
> 
> [On the other hand, we already agreed to remove `withJavaConsumer` from 
> `Branched`, so during code migration I will have to modify my functions' 
> return types anyway -- I mean, from `void` to `KStream`!! ]
> 
>  >  the map you're returning is Map<K,V>, and of course a K is not the 
> same as "? extends K", so it doesn't seem compatible.
> 
> I think what you actually meant here is that KStream<? extends K, ? 
> extends V> is not fit as a value for Map<String, KStream<K, V>>. This 
> particularly is not a problem, since KStream<? extends K, ? extends V> 
> can be safely explicitly cast to KStream<K, V>, and be put to the map.
> 
> BUT, I do really afraid of pitfalls of nested wildcard types. So maybe 
> for now it's better to just admit that API is not absolutely perfect and 
> accept it as is, that is
> 
> Function<? super KStream<K, V>, ? extends KStream<K, V>>
> 
> Regards,
> 
> Ivan
> 
> 
> 21.05.2020 17:59, John Roesler пишет:
> > Hello Ivan,
> > 
> > Thanks for the refinement. Actually, I did not know that "as" would
> > clash with a Kotlin operator. Maybe we should depart from convention
> > and just avoid methods named "as" in the future.
> > 
> > The convention is that "as(String name)" is used for the static factory
> > method, whereas "withName(String name)" is an instance method
> > inherited from NamedOperation. If you wish to propose to avoid "as"
> > for compatibility with Kotlin, I might suggest "fromName(String name)",
> > although it's somewhat dubious, since all the other configuration
> > classes use "as". Perhaps it would be better to stick with "as" for now
> > and just file a Jira to switch them all at the same time.
> > 
> > Re. 3:
> > Regarding the type inference problem, yes, it's a blemish on all of our
> > configuraion objects. The problem is that Java infers the type
> > based on the _first_ method in the chain. While it does consider what
> > the recipient of the method result wants, it only considers the _next_
> > recipient.
> > 
> > Thus, if you call as("foo") and immediately assign it to a
> > Branched<String,String> variable, java infers the type correctly. But
> > when the "next recipient" is a chained method call, like "withChain",
> > then the chained method doesn't bound the type (by definition,
> > withChain is defined on Branched<Object, Object>, so Java will take
> > the broadest possible inferece and bind the type to
> > Branched<Object, Object>, at which point, it can't be revised anymore.
> > 
> > As a user of Java, this is exceedingly annoying, since it doesn't seem
> > that hard to recursively consider the entire context when inferring the
> > generic type parameters, but this is what we have to work with.
> > 
> > To be honest, though, I don't understand how your new proposed
> > methods would work any differently than the ones you already
> > had proposed in the KIP. It seems like you'd still have to provide
> > the generic type parameters on the first static factory call. Can you
> > explain how your new interface proposal differs from the existing KIP?
> > 
> > Re. 4:
> > Regarding making the K,V types covariant also, yes, that would indeed
> > be nice, but I'm not sure it will actually work. You might want to give it a
> > try. In the past, we've run into soe truly strange interactions between the
> > Java type inferencer and lambdas (and/or anonymous inner classes) in
> > combination with nested covariant types.
> > 
> > Another issue is that the value type of the map you're returning is
> > Map<K,V>, and of course a K is not the same as "? extends K", so it
> > doesn't seem compatible.
> > 
> > Thanks again,
> > -John
> > 
> > On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
> >> Hi,
> >>
> >> Thanks Matthias for your suggestion: yes, I agree that getting rid of
> >> `with[Java]Consumer` makes this thing 'as simple as possible, but not
> >> simpler'.
> >>
> >> I made some quick API mocking in my IDE and tried to implement examples
> >> from KIP.
> >>
> >> 1. Having to return something from lambda is not a very big deal.
> >>
> >> 2. For a moment I thouht that I won't be able to use method references
> >> for already written stream consumers, but then I realized that I can
> >> just change my methods from returning void to returning the input
> >> parameter and use references to them. Not very convenient, but passable.
> >>
> >> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
> >> function returns null, we don't insert it into the resulting map.
> >>
> >> Usually it's better to implement a non-perfect, but workable solution as
> >> a first approximation. And later we can always add to `Branched`
> >> anything we want.
> >>
> >> 3. Do we have any guidelines on how parameter classes like Branched
> >> should be built? First of all, it seems that `as` now is more preferred
> >> than `withName` (although as you probably know it clashes with Kotlin's
> >> `as` operator).
> >>
> >> Then, while trying to mock the APIs, I found out that my Java cannot
> >> infer types in the following construction:
> >>
> >> .branch((key, value) -> value == null,
> >>      Branched.as("foo").withChain(s -> s.mapValues(...)))
> >>
> >>
> >> so I have to write
> >>
> >> .branch((key, value) -> value == null,
> >>      Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
> >>
> >>
> >> This is not tolerable IMO, so this is the list of `Branched` methods
> >> that I came to (will you please validate it):
> >>
> >> static <K, V> Branched<K, V> as(String name);
> >>
> >> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >> extends KStream<K, V>> chain);
> >>
> >> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
> >> extends KStream<K, V>> chain, String name);
> >>
> >> //non-static!
> >> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
> >> KStream<K, V>> chain);
> >>
> >>
> >> 4. And one more. What do you think, do we need that flexibility:
> >>
> >> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
> >>
> >> vs.
> >>
> >> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
> >> extends K, ? extends V>> chain
> >>
> >> ??
> >>
> >> Regards,
> >>
> >> Ivan
> >>
> >>
> >> 21.05.2020 6:54, John Roesler пишет:
> >>> Thanks for this thought, Matthias,
> >>>
> >>> Your idea has a few aspects I find attractive:
> >>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
> >>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
> >>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
> >>>
> >>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
> >>>
> >>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
> >>>
> >>> Thanks again for sharing the idea,
> >>> John
> >>>
> >>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> >>>> Thanks for updating the KIP!
> >>>>
> >>>> I guess the only open question is about `Branched.withJavaConsumer` and
> >>>> its relationship to the returned `Map`.
> >>>>
> >>>> Originally, we discussed two main patterns:
> >>>>
> >>>>    (1) split a stream and return the substreams for futher processing
> >>>>    (2) split a stream and modify the substreams with in-place method chaining
> >>>>
> >>>> To combine both patterns we wanted to allow for
> >>>>
> >>>>     -> split a stream, modify the substreams, and return the _modified_
> >>>> substreams for further processing
> >>>>
> >>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> >>>>
> >>>> That is of course possible. However, it introduces some "hidded" semantics:
> >>>>
> >>>>    - using `withChain` I get the modified sub-stream
> >>>>    - using `withJavaConsumer` I get the unmodifed sub-stream
> >>>>
> >>>> This seems to be quite subtle to me.
> >>>>
> >>>>
> >>>>
> >>>>   From my understanding the original idea of `withJavaConsumer` was to
> >>>> model a terminal operation, ie, it should be similar to:
> >>>>
> >>>> Branched.withChain(s -> {
> >>>>     s.to();
> >>>>     return null;
> >>>> })
> >>>>
> >>>> However, I am not sure if we should even allow `withChain()` to return
> >>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
> >>>> -> null` entry in the returned Map.
> >>>>
> >>>> Following this train of through, and if we want to allow the "return
> >>>> null" pattern in general, we need `withJavaConsumer` that does not add
> >>>> an entry to the Map.
> >>>>
> >>>> Following your proposal, the semantics of `withJavaConsumer` could also
> >>>> be achieved with `withChain`:
> >>>>
> >>>> Branched.withChain(s -> {
> >>>>     s.to();
> >>>>     return s;
> >>>> })
> >>>>
> >>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> >>>> while for the first proposal it adds new functionality (if `return null`
> >>>> is not allowed, using `withChain()` is not possible to "hide a
> >>>> sub-stream in the result). Furthermore, we might need to allow `return
> >>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> >>>>
> >>>>
> >>>>
> >>>> I guess I can be convinced either way. However, if we follow your
> >>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
> >>>> benefit seems to be small? Also, having a reduced API is usually
> >>>> preferable as it's simpler to learn.
> >>>>
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> >>>>> Hello, John, hello Matthias!
> >>>>>
> >>>>> Thank you very much for your detailed feedback!
> >>>>>
> >>>>> -----------------------------------------
> >>>>>
> >>>>> John,
> >>>>>
> >>>>>> It looks like you missed my reply on Apr 23rd.
> >>>>>
> >>>>> For some unknown reason it didn't reach my inbox, fortunately we have
> >>>>> all the emails on the web.
> >>>>>
> >>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> >>>>> method?
> >>>>>
> >>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
> >>>>>
> >>>>>> 2. [Explain why 'branch' operator is superior to branching directly
> >>>>> off of the parent KStream for the needs of dynamic branching]
> >>>>>
> >>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
> >>>>>
> >>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> >>>>>
> >>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
> >>>>> method to `withJavaConsumer` per Matthias' suggestion.
> >>>>>
> >>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
> >>>>> chain and the result map OR using just the sink
> >>>>>
> >>>>> This is discussed below.
> >>>>>
> >>>>> ----------------------------------------------
> >>>>>
> >>>>> Mathhias,
> >>>>>
> >>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> >>>>>
> >>>>> Done.
> >>>>>
> >>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> >>>>> method]
> >>>>>
> >>>>> Fixed.
> >>>>>
> >>>>>
> >>>>>> 3. Overview of newly added methods/interfaces
> >>>>>
> >>>>> Done in `Proposed Changes` section.
> >>>>>
> >>>>>
> >>>>>> 4. [Concerning John's note] > I don't think that using both
> >>>>> `withChain()` and `withConsumer()` is the
> >>>>> issue, as the KIP clearly states that the result of `withChain()` will
> >>>>> be given to the `Consumer`.
> >>>>>
> >>>>> Yes, I agree!
> >>>>>
> >>>>>> The issue is really with the `Consumer` and the returned `Map` of
> >>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> >>>>> implementation would be to not add the "branch" to the result map if
> >>>>> `withConsumer` is used?
> >>>>>
> >>>>> But is it also an issue? With Kafka Streams, we can split the topology
> >>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
> >>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> >>>>> stream in the Map, one simply does not extract it from there :-)
> >>>>>
> >>>>> In the current version of KIP it is assumed that the returned map
> >>>>> contains ALL the branches, either tagged with IDs explicitly set by the
> >>>>> programmer, or with some default auto-generated ids. Dealing with this
> >>>>> map is the user's responsibility.
> >>>>>
> >>>>> What seems to me to be an issue is introducing exclusions to this
> >>>>> general rule, like 'swallowing' some streams by provided
> >>>>> [Java]Consumers. This can make things complicated. What if a user
> >>>>> provides both the name of the branch and a [Java]Consumer? What do they
> >>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
> >>>>> There's no point in 'saving the space' in this map, so maybe just leave
> >>>>> it as it is?
> >>>>>
> >>>>> ----
> >>>>>
> >>>>> I rewrote the KIP and also fixed a couple of typos.
> >>>>>
> >>>>> Looking forward for your feedback again!
> >>>>>
> >>>>> Regards,
> >>>>>
> >>>>> Ivan.
> >>>>>
> >>>>>
> >>>>>
> >>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
> >>>>>> Thanks for updating the KIP!
> >>>>>>
> >>>>>> I also have some minor comment:
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> >>>>>>
> >>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> >>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> >>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
> >>>>>> so we just keep them.)
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> (2) Quote:
> >>>>>>
> >>>>>>> Both branch and defaultBranch operations also have overloaded
> >>>>>>> parameterless alternatives.
> >>>>>>
> >>>>>> I think `branch()` always needs to take a `Predicate` and assume you
> >>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> >>>>>> as `branch()` would not be "parameterless".
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
> >>>>>> newly added and deprecated methods/classes (cf.
> >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> >>>>>> and the finally returned `Map<String, KStream>`. This related to John's
> >>>>>> 4th comment:
> >>>>>>
> >>>>>>> It seems like there are really two disjoint use cases: EITHER using
> >>>>>>> chain and the result map OR using just the sink.
> >>>>>>
> >>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
> >>>>>> issue though, as the KIP clearly states that the result of `withChain()`
> >>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
> >>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> >>>>>>
> >>>>>> Maybe a reasonable implementation would be to not add the "branch" to
> >>>>>> the result map if `withConsumer` is used? As long as we clearly document
> >>>>>> it in the JavaDocs, this might be fine?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> (5) Reply to John's comments:
> >>>>>>
> >>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> >>>>>>> were talking about the kafka Consumer interface (which doesn’t make
> >>>>>>> sense, of course). I get that you were referring to the java Consumer
> >>>>>>> interface, but we should still probably to to avoid the ambiguity.
> >>>>>>> Just throwing out a suggestion, how about ‘withSink’?
> >>>>>>
> >>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
> >>>>>> node", ie., writing the KStream to a topic.
> >>>>>>
> >>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
> >>>>>>> Hi Ivan,
> >>>>>>>
> >>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> >>>>>>> but I had a few last comments.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> John
> >>>>>>>
> >>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> >>>>>>>> Hello everyone,
> >>>>>>>>
> >>>>>>>> will someone please take a look at the reworked KIP?
> >>>>>>>>
> >>>>>>>> I believe that now it follows design principles and takes into account
> >>>>>>>> all the arguments discussed here.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>>
> >>>>>>>> Ivan
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>>>>>> Hi,
> >>>>>>>>>
> >>>>>>>>> I have read the John's "DSL design principles" and have completely
> >>>>>>>>> rewritten the KIP, see
> >>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> This version includes all the previous discussion results and follows
> >>>>>>>>> the design principles, with one exception.
> >>>>>>>>>
> >>>>>>>>> The exception is
> >>>>>>>>>
> >>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> >>>>>>>>>
> >>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
> >>>>>>>>> here it is justified.
> >>>>>>>>>
> >>>>>>>>> We must provide a predicate for a branch and don't need to provide one
> >>>>>>>>> for the default branch. Thus for both operations we may use a single
> >>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
> >>>>>>>>>
> >>>>>>>>> Since predicate is a natural, necessary part of a branch, no
> >>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> >>>>>>>>> as it
> >>>>>>>>> is said in the rationale for the 'single parameter rule'.
> >>>>>>>>>
> >>>>>>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>>
> >>>>>>>>> Ivan
> >>>>>>>>>
> >>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>>>>>> Ivan,
> >>>>>>>>>>
> >>>>>>>>>> no worries about getting side tracked. Glad to have you back!
> >>>>>>>>>>
> >>>>>>>>>> The DSL improved further in the meantime and we already have a
> >>>>>>>>>> `Named`
> >>>>>>>>>> config object to name operators. It seems reasonable to me to
> >>>>>>>>>> build on
> >>>>>>>>>> this.
> >>>>>>>>>>
> >>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> >>>>>>>>>> want to follow:
> >>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -- might be worth to checkout.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>>>>>> Hi everyone!
> >>>>>>>>>>>
> >>>>>>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>>>>>
> >>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
> >>>>>>>>>>> June
> >>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
> >>>>>>>>>>> spare time. But I think I must finish this, because we invested
> >>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
> >>>>>>>>>>> propose other things before this one is finalized.
> >>>>>>>>>>>
> >>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
> >>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> >>>>>>>>>>> this
> >>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> >>>>>>>>>>> the
> >>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
> >>>>>>>>>>> branches, I worked around it this way:
> >>>>>>>>>>>
> >>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>>>>>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>>>>>         .branch(....)
> >>>>>>>>>>>         .defaultBranch(result::set)
> >>>>>>>>>>>         .onTopOf(someStream);
> >>>>>>>>>>> result.get()...
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
> >>>>>>>>>>>
> >>>>>>>>>>> I think that Matthias came up with a bright solution in his post
> >>>>>>>>>>> from
> >>>>>>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>>>>>
> >>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>       -> KBranchedStream
> >>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>> //
> >>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>>>>>       -> KBranchedStream
> >>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>>
> >>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
> >>>>>>>>>>> a good
> >>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
> >>>>>>>>>>> you
> >>>>>>>>>>> just don't use names and you don't risk making your code brittle.
> >>>>>>>>>>> Or,
> >>>>>>>>>>> you might want to add names just for debugging purposes. Or,
> >>>>>>>>>>> finally,
> >>>>>>>>>>> you might use the returned Map to have the named branches in the
> >>>>>>>>>>> original scope.
> >>>>>>>>>>>
> >>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> >>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
> >>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
> >>>>>>>>>>> familiar with Streams API design principles than me.
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>>
> >>>>>>>>>>> Ivan
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> >>>>>>>>>>>> the
> >>>>>>>>>>>> KIP
> >>>>>>>>>>>> at any point.
> >>>>>>>>>>>>
> >>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> >>>>>>>>>>>> do so.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
> >>>>>>>>>>>>> to mix
> >>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> >>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> >>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> >>>>>>>>>>>>>> patterns
> >>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
> >>>>>>>>>>>>>> make
> >>>>>>>>>>>>>> totally sense.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> >>>>>>>>>>>>>> `Map` only
> >>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
> >>>>>>>>>>>>>> all of
> >>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> >>>>>>>>>>>>>> `Named` is
> >>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> >>>>>>>>>>>>>> counter
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
> >>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
> >>>>>>>>>>>>>> names
> >>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
> >>>>>>>>>>>>>> misses to
> >>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> >>>>>>>>>>>>>> keep
> >>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> >>>>>>>>>>>>>> required
> >>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>>>>>>>>>> specifying a
> >>>>>>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> About
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
> >>>>>>>>>>>>>> hence
> >>>>>>>>>>>>>> would not include it in a configuration object.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          withChain(...);
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> >>>>>>>>>>>>>> does not
> >>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> >>>>>>>>>>>>>> call
> >>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
> >>>>>>>>>>>>>> make sense
> >>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> >>>>>>>>>>>>>> check
> >>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
> >>>>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
> >>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> >>>>>>>>>>>>>> object
> >>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
> >>>>>>>>>>>>>> others,
> >>>>>>>>>>>>>> but this seems not to be the case here. If we add new
> >>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>> later, we can also just move forward by deprecating the
> >>>>>>>>>>>>>> methods that
> >>>>>>>>>>>>>> accept `Named` and add new methods that accepted
> >>>>>>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> >>>>>>>>>>>>>> two main
> >>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> >>>>>>>>>>>>>>> satisfy
> >>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
> >>>>>>>>>>>>>>> solves
> >>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> >>>>>>>>>>>>>>> to add
> >>>>>>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
> >>>>>>>>>>>>>>> Otherwise,
> >>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> >>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
> >>>>>>>>>>>>>>> names for
> >>>>>>>>>>>>>>> operations are not required to define stream processing
> >>>>>>>>>>>>>>> logic, it
> >>>>>>>>>>>>>>> does
> >>>>>>>>>>>>>>> significantly improve the operational experience when you can
> >>>>>>>>>>>>>>> map
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> >>>>>>>>>>>>>>> processing onto
> >>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> >>>>>>>>>>>>>>> names
> >>>>>>>>>>>>>>> built from the branch operator name. I guess under this
> >>>>>>>>>>>>>>> proposal, we
> >>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
> >>>>>>>>>>>>>>> name,
> >>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>         stream.split(Named.withName("mysplit")) //creates node
> >>>>>>>>>>>>>>> "mysplit"
> >>>>>>>>>>>>>>>                    .branch(..., ..., "abranch") // creates node
> >>>>>>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>>>>>                    .defaultBranch(...) // creates node
> >>>>>>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> >>>>>>>>>>>>>>> debate
> >>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
> >>>>>>>>>>>>>>> general,
> >>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>          operator(function, config_object?) OR
> >>>>>>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> where config_object is often just Named in the "function"
> >>>>>>>>>>>>>>> variant.
> >>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>        operator(function, function, string)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> where the string is the name.
> >>>>>>>>>>>>>>> My first question is whether the name should instead be
> >>>>>>>>>>>>>>> specified
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> My second question is whether we should just roll all these
> >>>>>>>>>>>>>>> arguments
> >>>>>>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>         KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>         interface BranchConfig extends NamedOperation {
> >>>>>>>>>>>>>>>          withPredicate(...);
> >>>>>>>>>>>>>>>          withChain(...);
> >>>>>>>>>>>>>>>          withName(...);
> >>>>>>>>>>>>>>>        }
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> >>>>>>>>>>>>>>> more like
> >>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
> >>>>>>>>>>>>>>> makes us
> >>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> >>>>>>>>>>>>>>> purely
> >>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
> >>>>>>>>>>>>>>> overloads
> >>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> WDYT?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> >>>>>>>>>>>>>>>> view.
> >>>>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> >>>>>>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>>>        -> KBranchedStream
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> >>>>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>>>        -> KBranchedStream
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>>>        -> Map<String,KStream>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> >>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
> >>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> >>>>>>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
> >>>>>>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> >>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> >>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
> >>>>>>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> >>>>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> >>>>>>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> >>>>>>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>>>>>> language
> >>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> >>>>>>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
> >>>>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> >>>>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> >>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> >>>>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> >>>>>>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> >>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          Ivan, I’ll definitely forfeit my point on the
> >>>>>>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          branch(predicate, consumer) solution, I don’t see
> >>>>>>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>>>>>          for the dynamic case.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          IMO the one trade off to consider at this point is the
> >>>>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>>>          question. I don’t know if I totally agree that “we
> >>>>>>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>>>          in the same scope” since merging the branches back
> >>>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>          seems like a perfectly plausible use case that can
> >>>>>>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>>>>>          when the branched streams are in the same scope.
> >>>>>>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>          for the reasons Ivan listed, I think it is overall the
> >>>>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>>>          solution - working around the scope thing is easy
> >>>>>>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>>>>>          to.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>          <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > Hello everyone, thank you all for joining the
> >>>>>>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > Well, I don't think the idea of named branches,
> >>>>>>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>>>>>          LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>>>          matters) or `branch` method  taking name and Consumer
> >>>>>>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>>>>>          advantages than drawbacks.
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > In my opinion, the only real positive outcome from
> >>>>>>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>>>>>          proposal is that all the returned branches are in
> >>>>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>>>          But 1) we rarely need them in the same scope 2)
> >>>>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>>>          workaround for the scope problem, described in the
> >>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > 'Inlining the complex logic' is not a problem,
> >>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>>>>>          method references instead of lambdas. In real world
> >>>>>>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>>>>>          tend to split the complex logic to methods anyway,
> >>>>>>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>>>>>          going to be clean.
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > The drawbacks are strong. The cohesion between
> >>>>>>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          handlers is lost. We have to define predicates in one
> >>>>>>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>>>>>          handlers in another. This opens the door for bugs:
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > - what if we forget to define a handler for a
> >>>>>>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>>>>>          a handler?
> >>>>>>>>>>>>>>>>>>          > - what if we misspell a name?
> >>>>>>>>>>>>>>>>>>          > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > What Michael propose would have been totally OK
> >>>>>>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>          writing the API in Lua, Ruby or Python. In those
> >>>>>>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>>>>>          "dynamic naming" approach would have looked most
> >>>>>>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          beautiful. But in Java we expect all the problems
> >>>>>>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>>>>>          identifiers to be eliminated in compile time.
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > And if we do, what advantage are we supposed to get
> >>>>>>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>>>>>          all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>>>>>          point?
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > ---
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > Earlier in this discussion John Roesler also
> >>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>          without "start branching" operator, and later Paul
> >>>>>>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>>>>>          the case when we have to add a dynamic number of
> >>>>>>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>>>>>          current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>>>>>          me address both comments here.
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > 1) "Start branching" operator (I think that
> >>>>>>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>>>>>          for it indeed) is critical when we need to do a
> >>>>>>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>>>>>          see example below.
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > 2) No, dynamic branching in current KIP is not
> >>>>>>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>>>          Imagine a real-world scenario when you need one
> >>>>>>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>>>>>          value (say, RecordType). You can have something
> >>>>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > /*John:if we had to start with stream.branch(...)
> >>>>>>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>>>>>          have been much messier.*/
> >>>>>>>>>>>>>>>>>>          > KBranchedStream branched = stream.split();
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > /*Not clumsy at all :-)*/
> >>>>>>>>>>>>>>>>>>          > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>>>>>>>>>          >             branched = branched.branch((k, v) ->
> >>>>>>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>>>>>          recordType,
> >>>>>>>>>>>>>>>>>>          >                     recordType::processRecords);
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > Regards,
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > Ivan
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>          > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>          >> I also agree with Michael's observation about
> >>>>>>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>>>>>          >> current `branch()` implementation.
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>>>>>          thinking
> >>>>>>>>>>>>>>>>>>          >> was more aligned with Paul's proposal to just
> >>>>>>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>>>>>          >> `branch()` statement and return a
> >>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >> It makes the code easier to read, and also make the
> >>>>>>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>>>>>          >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >>>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>          >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>          >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>          >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>>>>>          >> An open question is the case for which no
> >>>>>>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>          >> specified. Atm, `split()` and `branch()` would
> >>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>          `BranchedKStream`
> >>>>>>>>>>>>>>>>>>          >> and the call to `defaultBranch()` that returns the
> >>>>>>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>>>>>          >> (what is not the case atm). Or is this actually
> >>>>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>> real
> >>>>>>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>>>>>          >> because users can just ignore the branch
> >>>>>>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>>>>>          `defaultBranch()`
> >>>>>>>>>>>>>>>>>>          >> in the result `Map` ?
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >> About "inlining": So far, it seems to be a
> >>>>>>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>>>>>          >> preference. I can see arguments for both, but no
> >>>>>>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>>>>>          argument" yet
> >>>>>>>>>>>>>>>>>>          >> that clearly make the case for one or the other.
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >> -Matthias
> >>>>>>>>>>>>>>>>>>          >>
> >>>>>>>>>>>>>>>>>>          >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>          >>> Perhaps inlining is the wrong terminology. It
> >>>>>>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>          that a lambda with the full downstream topology be
> >>>>>>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>>>>>          it can be a method reference as with Ivan’s original
> >>>>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>>>          The advantage of putting the predicate and its
> >>>>>>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>          (Consumer) together in branch() is that they are
> >>>>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>>>>>          to each other.
> >>>>>>>>>>>>>>>>>>          >>>
> >>>>>>>>>>>>>>>>>>          >>> Ultimately the downstream code has to live
> >>>>>>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>>>>>          branch trees will be hard to read regardless.
> >>>>>>>>>>>>>>>>>>          >>>
> >>>>>>>>>>>>>>>>>>          >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>>>>>          <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>          <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>
> >>>>>>>>>>>>>>>>>>          >>>> I'm less enthusiastic about inlining the
> >>>>>>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>>>>>          downstream
> >>>>>>>>>>>>>>>>>>          >>>> functionality. Programs that have deep branch
> >>>>>>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>          quickly become
> >>>>>>>>>>>>>>>>>>          >>>> harder to read as a single unit.
> >>>>>>>>>>>>>>>>>>          >>>>
> >>>>>>>>>>>>>>>>>>          >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>>>>>          <pgwhalen@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Also +1 on the issues/goals as Michael
> >>>>>>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>>          that sets a
> >>>>>>>>>>>>>>>>>>          >>>>> great framework for the discussion.
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Regarding the SortedMap solution, my
> >>>>>>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>>>          current
> >>>>>>>>>>>>>>>>>>          >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>>>>>          decisions) is
> >>>>>>>>>>>>>>>>>>          >>>>> roughly this:
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> stream.split()
> >>>>>>>>>>>>>>>>>>          >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>          >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>>>          >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Obviously some ordering is necessary, since
> >>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>          construct
> >>>>>>>>>>>>>>>>>>          >>>>> doesn't work without it, but this solution seems
> >>>>>>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>          provides as much
> >>>>>>>>>>>>>>>>>>          >>>>> associativity as the SortedMap solution,
> >>>>>>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>          call
> >>>>>>>>>>>>>>>>>>          >>>>> directly associates the "conditional" with
> >>>>>>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>>>>>          The value it
> >>>>>>>>>>>>>>>>>>          >>>>> provides over the KIP solution is the
> >>>>>>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>>>          the same
> >>>>>>>>>>>>>>>>>>          >>>>> scope.
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> The KIP solution is less "dynamic" than the
> >>>>>>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>          in the sense
> >>>>>>>>>>>>>>>>>>          >>>>> that it is slightly clumsier to add a dynamic
> >>>>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>          branches, but it is
> >>>>>>>>>>>>>>>>>>          >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>>>>>          the "static"
> >>>>>>>>>>>>>>>>>>          >>>>> case anyway, and should make it simple and
> >>>>>>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>>>>>          fluently declare and
> >>>>>>>>>>>>>>>>>>          >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>>>>>          ignore a
> >>>>>>>>>>>>>>>>>>          >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>>>>>          SortedMap
> >>>>>>>>>>>>>>>>>>          >>>>> solution on top of it.
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> I could also see a middle ground where
> >>>>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>>>>>          SortedMap being
> >>>>>>>>>>>>>>>>>>          >>>>> taken in, branch() takes a name and not a
> >>>>>>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>>>>>          like this:
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>          >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>          >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>>>          >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Pros for that solution:
> >>>>>>>>>>>>>>>>>>          >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>>>>>>>>>          >>>>> - no double brace initialization, hopefully
> >>>>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>          readable than
> >>>>>>>>>>>>>>>>>>          >>>>> SortedMap
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Cons
> >>>>>>>>>>>>>>>>>>          >>>>> - downstream branch logic cannot be specified
> >>>>>>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>          makes it harder
> >>>>>>>>>>>>>>>>>>          >>>>> to read top to bottom (like existing API and
> >>>>>>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>>>>>          unlike the KIP)
> >>>>>>>>>>>>>>>>>>          >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>>>>>          existing
> >>>>>>>>>>>>>>>>>>          >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> (KBranchedStreams could even work *both* ways
> >>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>>>>>          that's overdoing
> >>>>>>>>>>>>>>>>>>          >>>>> it).
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Overall I'm curious how important it is to be
> >>>>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>>>          access the
> >>>>>>>>>>>>>>>>>>          >>>>> branched KStream in the same scope as the
> >>>>>>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>          possible that it
> >>>>>>>>>>>>>>>>>>          >>>>> doesn't need to be handled directly by the
> >>>>>>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>          left up to the
> >>>>>>>>>>>>>>>>>>          >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> Paul
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> >>>>>>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>>>>>          <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>          >>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>> I'd like to +1 what Michael said about the
> >>>>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>          existing
> >>>>>>>>>>>>>>>>>>          >>>>> branch
> >>>>>>>>>>>>>>>>>>          >>>>>> method, I agree with what he's outlined and
> >>>>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>>>>>          proceed by
> >>>>>>>>>>>>>>>>>>          >>>>>> trying to alleviate these problems.
> >>>>>>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>          important to be
> >>>>>>>>>>>>>>>>>>          >>>>>> able to cleanly access the individual
> >>>>>>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>>>>>          >>>>>> name->stream), which I thought was the original
> >>>>>>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>>>>>          this KIP.
> >>>>>>>>>>>>>>>>>>          >>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>> That said, I don't think we should so easily
> >>>>>>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>          double brace
> >>>>>>>>>>>>>>>>>>          >>>>>> anti-pattern or force ours users into it if
> >>>>>>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>>>          >>>>> avoid...just
> >>>>>>>>>>>>>>>>>>          >>>>>> my two cents.
> >>>>>>>>>>>>>>>>>>          >>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>          >>>>>> Sophie
> >>>>>>>>>>>>>>>>>>          >>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> >>>>>>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>>>>>          >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>>>          <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>> I’d like to propose a different way of
> >>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>>>>>          To me,
> >>>>>>>>>>>>>>>>>>          >>>>> there
> >>>>>>>>>>>>>>>>>>          >>>>>>> are three problems with the existing branch
> >>>>>>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>>>>>          >>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>>>          >>>>>> warnings.
> >>>>>>>>>>>>>>>>>>          >>>>>>> 2. The way in which you use the stream
> >>>>>>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>>>>>          positionally coupled
> >>>>>>>>>>>>>>>>>>          >>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>>>>>>>>>          >>>>>>> 3. It is brittle to extend existing branch
> >>>>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>          additional code
> >>>>>>>>>>>>>>>>>>          >>>>>>> paths.
> >>>>>>>>>>>>>>>>>>          >>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>> Using associative constructs instead of
> >>>>>>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>>>>>          constructs
> >>>>>>>>>>>>>>>>>>          >>>>>> would
> >>>>>>>>>>>>>>>>>>          >>>>>>> be a stronger approach. Consider a
> >>>>>>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>>>          looks like
> >>>>>>>>>>>>>>>>>>          >>>>>> this:
> >>>>>>>>>>>>>>>>>>          >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>>>>>          Predicate<?
> >>>>>>>>>>>>>>>>>>          >>>>>>> super K,? super V>>);
> >>>>>>>>>>>>>>>>>>          >>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>> Branches are given names in a map, and as a
> >>>>>>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>>>>>          returns a
> >>>>>>>>>>>>>>>>>>          >>>>>>> mapping of names to streams. The ordering
> >>>>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>>>>>          >>>>>> maintained
> >>>>>>>>>>>>>>>>>>          >>>>>>> because it’s a sorted map. Insert order
> >>>>>>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>>>>>> the order
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>          >>>>>> evaluation.
> >>>>>>>>>>>>>>>>>>          >>>>>>> This solves problem 1 because there are no
> >>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>>>>>          solves
> >>>>>>>>>>>>>>>>>>          >>>>>> problem
> >>>>>>>>>>>>>>>>>>          >>>>>>> 2 because you no longer lean on ordering to
> >>>>>>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>>>>>          branch you’re
> >>>>>>>>>>>>>>>>>>          >>>>>>> interested in. It solves problem 3 because
> >>>>>>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>>>          another
> >>>>>>>>>>>>>>>>>>          >>>>>>> conditional by simply attaching another
> >>>>>>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>>>>>          structure, rather
> >>>>>>>>>>>>>>>>>>          >>>>>> than
> >>>>>>>>>>>>>>>>>>          >>>>>>> messing with the existing indices.
> >>>>>>>>>>>>>>>>>>          >>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>>>>>          historically
> >>>>>>>>>>>>>>>>>>          >>>>>>> awkward in Java. I know it’s an
> >>>>>>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>>>>>          voluminously, but
> >>>>>>>>>>>>>>>>>>          >>>>>>> double brace initialization would clean up the
> >>>>>>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>>>>>          >>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>>>>>          <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>>>          >>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> Thanks for the update.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> FWIW, I agree with Matthias that the current
> >>>>>>>>>>>>>>>>>> "start
> >>>>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>>>          >>>>> operator
> >>>>>>>>>>>>>>>>>>          >>>>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>>>> confusing when named the same way as the
> >>>>>>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>>>>>          "Split"
> >>>>>>>>>>>>>>>>>>          >>>>> seems
> >>>>>>>>>>>>>>>>>>          >>>>>>>> like a good name. Alternatively, we can do
> >>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>>>>>          >>>>> branching"
> >>>>>>>>>>>>>>>>>>          >>>>>>>> operator at all, and just do:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> stream
> >>>>>>>>>>>>>>>>>>          >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> Tentatively, I think that this branching
> >>>>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>          >>>>> terminal.
> >>>>>>>>>>>>>>>>>>          >>>>>>> That
> >>>>>>>>>>>>>>>>>>          >>>>>>>> way, we don't create ambiguity about how
> >>>>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>>>>>          is, `branch`
> >>>>>>>>>>>>>>>>>>          >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>>>>>          `void`, to
> >>>>>>>>>>>>>>>>>>          >>>>>>>> enforce that it comes last, and that there
> >>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>>>          definition of
> >>>>>>>>>>>>>>>>>>          >>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>>>>>          there's no
> >>>>>>>>>>>>>>>>>>          >>>>>>> default,
> >>>>>>>>>>>>>>>>>>          >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>>>>>          record
> >>>>>>>>>>>>>>>>>>          >>>>>> falls
> >>>>>>>>>>>>>>>>>>          >>>>>>>> though with no default.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>          >>>>>>>> -John
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>>>>>          >>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>>>          >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> Thanks for updating the KIP and your
> >>>>>>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> this is to make the name similar to
> >>>>>>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> The intend was to avoid name duplication.
> >>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>>>>>          should
> >>>>>>>>>>>>>>>>>>          >>>>>> _not_
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> be an array.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> The current proposal is
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> IMHO, this reads a little odd, because
> >>>>>>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>>>>>          `branch()` does
> >>>>>>>>>>>>>>>>>>          >>>>> not
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> take any parameters and has different
> >>>>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>>>> than the
> >>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>          >>>>> hidden
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> that the first call is `KStream#branch()`
> >>>>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>>>> the others
> >>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> Because I suggested to rename
> >>>>>>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>>>>>          I though
> >>>>>>>>>>>>>>>>>>          >>>>>> it
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> might be better to also rename
> >>>>>>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>> naming
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> overlap that seems to be confusing. The
> >>>>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>          >>>>> cleaner
> >>>>>>>>>>>>>>>>>>          >>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>> me:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> Maybe there is a better alternative to
> >>>>>>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>>>>>          avoid
> >>>>>>>>>>>>>>>>>>          >>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> naming overlap.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>          >>>>> cannot
> >>>>>>>>>>>>>>>>>>          >>>>>>> have
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> a method with such name :-)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> Bummer. Didn't consider this. Maybe we
> >>>>>>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>>>>>          with a
> >>>>>>>>>>>>>>>>>>          >>>>> short
> >>>>>>>>>>>>>>>>>>          >>>>>>>> name?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> Can you add the interface
> >>>>>>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>>          with all
> >>>>>>>>>>>>>>>>>>          >>>>> it's
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> methods? It will be part of public API and
> >>>>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>>>          contained in
> >>>>>>>>>>>>>>>>>>          >>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>>>>>>>>>          `KBranchedStream#get(int
> >>>>>>>>>>>>>>>>>>          >>>>>>> index)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>>>>>>>>>          branched-KStreams. Would
> >>>>>>>>>>>>>>>>>>          >>>>>> be
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> nice to get your feedback about it. It
> >>>>>>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>>>          that users
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> would need to write custom utility code
> >>>>>>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>>>>>          access them.
> >>>>>>>>>>>>>>>>>>          >>>>> We
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>>>>>          to get
> >>>>>>>>>>>>>>>>>>          >>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> I have updated the KIP-418 according to
> >>>>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> I can see your point: this is to make
> >>>>>>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>>>          >>>>>>> String#split
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> that also returns an array, right? But
> >>>>>>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>>>>>          loss of
> >>>>>>>>>>>>>>>>>>          >>>>>>>> backwards
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> compatibility? We can have overloaded
> >>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>>>>>          without
> >>>>>>>>>>>>>>>>>>          >>>>>>>> affecting
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> the existing code. Maybe the old
> >>>>>>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>          >>>>> should
> >>>>>>>>>>>>>>>>>>          >>>>>>> be
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> deprecated, but this is a subject for
> >>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>>          >>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Totally agree with 'addBranch->branch'
> >>>>>>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>          >>>>>>> however, a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> reserved word, so unfortunately we
> >>>>>>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>>>          with such
> >>>>>>>>>>>>>>>>>>          >>>>>>> name
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> :-)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> defaultBranch() does take an
> >>>>>>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>          >>>>> think
> >>>>>>>>>>>>>>>>>>          >>>>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> is not required?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Absolutely! I think that was just
> >>>>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>>>>>          something.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> please revise the new version of the KIP
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Thanks for driving the discussion of
> >>>>>>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>> everybody
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> agrees that the current branch() method
> >>>>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>          >>>>>> optimal.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> I had a quick look into the PR and I
> >>>>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>          proposal.
> >>>>>>>>>>>>>>>>>>          >>>>>>> There
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> are some minor things we need to
> >>>>>>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>          recommend the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> following renaming:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>>>          BranchingKStream#default()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> It's just a suggestion to get slightly
> >>>>>>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> In the current PR, defaultBranch() does
> >>>>>>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>>>>>          `Predicate` as
> >>>>>>>>>>>>>>>>>>          >>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>>>>>          accepted and
> >>>>>>>>>>>>>>>>>>          >>>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Ie, we should add overloads that
> >>>>>>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>>>          parameter.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> For the issue that the created
> >>>>>>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>>>>>          different
> >>>>>>>>>>>>>>>>>>          >>>>>>>> scopes:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> could we extend `KBranchedStream` with a
> >>>>>>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>>>>>          index)` method
> >>>>>>>>>>>>>>>>>>          >>>>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> returns the corresponding "branched"
> >>>>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>>>>>          object?
> >>>>>>>>>>>>>>>>>>          >>>>>> Maybe,
> >>>>>>>>>>>>>>>>>>          >>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> second argument of `addBranch()` should
> >>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>>>>>          >>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>>>>>          >>>>>>>> but
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> a `Function<KStream,KStream>` and
> >>>>>>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>          whatever
> >>>>>>>>>>>>>>>>>>          >>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Finally, I would also suggest to update
> >>>>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>          current
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>          makes sense
> >>>>>>>>>>>>>>>>>>          >>>>>> for
> >>>>>>>>>>>>>>>>>>          >>>>>>>> you
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> revise the KIP and continue the
> >>>>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>>>>>          we'll
> >>>>>>>>>>>>>>>>>>          >>>>> need
> >>>>>>>>>>>>>>>>>>          >>>>>>>> some
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>>>>>          >>>>> whether
> >>>>>>>>>>>>>>>>>>          >>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> could be adopted.  It would be great
> >>>>>>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>>>>>          think this
> >>>>>>>>>>>>>>>>>>          >>>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>>> a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> good
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> idea overall.  I'm not sure if that
> >>>>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>>>>>          starting a
> >>>>>>>>>>>>>>>>>>          >>>>>>> vote,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> or if
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> there is generally some indication of
> >>>>>>>>>>>>>>>>>> interest
> >>>>>>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> That being said, I'll continue the
> >>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>>>>>          assuming
> >>>>>>>>>>>>>>>>>>          >>>>> we
> >>>>>>>>>>>>>>>>>>          >>>>>> do
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> move
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> forward the solution of "stream.branch()
> >>>>>>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>>>>>          >>>>>> KBranchedStream",
> >>>>>>>>>>>>>>>>>>          >>>>>>> do
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> we
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>          >>>>> favor
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>> accomplish
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>>>>>          similar
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> just need to be sure we're not making
> >>>>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>          >>>>>>> impossible/difficult
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>>>          >>>>> it's
> >>>>>>>>>>>>>>>>>>          >>>>>>>> just a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> little sloppy overall in terms of
> >>>>>>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>>>>>          >>>>>>> particular,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> passing in the "predicates" and
> >>>>>>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>          >>>>>> modified
> >>>>>>>>>>>>>>>>>>          >>>>>>>> in
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>>>>>>>>>          KStreamLazyBranch is
> >>>>>>>>>>>>>>>>>>          >>>>> a
> >>>>>>>>>>>>>>>>>>          >>>>>>> bit
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>          >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> I read your code carefully and now I
> >>>>>>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>>>>>          convinced: your
> >>>>>>>>>>>>>>>>>>          >>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> looks better and should work. We just
> >>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>> crucial
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> fact
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> that KStream consumers are invoked as
> >>>>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>>>          And then
> >>>>>>>>>>>>>>>>>>          >>>>>> it's
> >>>>>>>>>>>>>>>>>>          >>>>>>>> all
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> What shall we do now? I should
> >>>>>>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>>>>>          resume the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Why are you telling that your PR
> >>>>>>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>>>>>          >>>>> starting
> >>>>>>>>>>>>>>>>>>          >>>>>>>> point
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> if
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> we go in this direction'? To me it
> >>>>>>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>>>          starting
> >>>>>>>>>>>>>>>>>>          >>>>>>> point.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> But
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> as a novice in this project I might
> >>>>>>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>>>>>          >>>>> details.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> >>>>>>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>>>>>          >>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> solution
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>>>>>          >>>>> invoked
> >>>>>>>>>>>>>>>>>>          >>>>>> as
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> they’re
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> added, not during
> >>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>>>>>          still
> >>>>>>>>>>>>>>>>>>          >>>>>> ought
> >>>>>>>>>>>>>>>>>>          >>>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> be
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> able to call couponIssuer.coupons()
> >>>>>>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          depend on
> >>>>>>>>>>>>>>>>>>          >>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> The issue I mean to point out is
> >>>>>>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>>>>>          access
> >>>>>>>>>>>>>>>>>>          >>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> streams in the same scope as the
> >>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>>>>>          is, not
> >>>>>>>>>>>>>>>>>>          >>>>>>>> inside
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>>>          >>>>> solutions.
> >>>>>>>>>>>>>>>>>>          >>>>>> It
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> [Also, great to hear additional
> >>>>>>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>>>>>          excited
> >>>>>>>>>>>>>>>>>>          >>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>> hear
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>          >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> The idea to postpone the wiring of
> >>>>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>>>>>> me at
> >>>>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>>>          >>>>> glance,
> >>>>>>>>>>>>>>>>>>          >>>>>>> but
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> ---
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>>>>>          same
> >>>>>>>>>>>>>>>>>>          >>>>>> scope
> >>>>>>>>>>>>>>>>>>          >>>>>>> as
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> each
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> other.  That is, if we wanted to merge
> >>>>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>>>          >>>>> again
> >>>>>>>>>>>>>>>>>>          >>>>>> I
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> don't see
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> You just took the words right out
> >>>>>>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>>>>>          just
> >>>>>>>>>>>>>>>>>>          >>>>>> going
> >>>>>>>>>>>>>>>>>>          >>>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Consider the example from Bill's
> >>>>>>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>>>>>          we need
> >>>>>>>>>>>>>>>>>>          >>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> identify
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> customers who have bought coffee and
> >>>>>>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>>>>>          in the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> electronics
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> This is the code I usually write under
> >>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>          circumstances
> >>>>>>>>>>>>>>>>>>          >>>>>> using
> >>>>>>>>>>>>>>>>>>          >>>>>>>> my
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>       return
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>       /*In the real world the code
> >>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>          complex, so
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> creation of
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>>>>>          order to
> >>>>>>>>>>>>>>>>>>          >>>>>>>> separate
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>>>>>>>>>          >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> >>>>>>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>>>>>          everything
> >>>>>>>>>>>>>>>>>>          >>>>>>>> later,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Does this make sense?  In order to
> >>>>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>>>          initialize the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> we need the terminal operation to be
> >>>>>>>>>>>>>>>>>> called
> >>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>>>>>          essentially
> >>>>>>>>>>>>>>>>>>          >>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> next
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> KIP I was going to write here. I have
> >>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>>>>>          based on
> >>>>>>>>>>>>>>>>>>          >>>>> my
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> experience,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> so I will join the discussion on KIP-401
> >>>>>>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>>>>>          fluent
> >>>>>>>>>>>>>>>>>>          >>>>> API
> >>>>>>>>>>>>>>>>>>          >>>>>>>> based
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>>>>>>>>>          (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>>>>>>>>>          >>>>>> and
> >>>>>>>>>>>>>>>>>>          >>>>>>> I
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>>>>>          >>>>>>> compatibility
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    there aren't any direct ones.
> >>>>>>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>>>>>          that Java
> >>>>>>>>>>>>>>>>>>          >>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>>>> smart
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    distinguish between a
> >>>>>>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>>>>>          returning one
> >>>>>>>>>>>>>>>>>>          >>>>>>> thing
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    with no arguments returning
> >>>>>>>>>>>>>>>>>> another
> >>>>>>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> >>>>>>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>          need
> >>>>>>>>>>>>>>>>>>          >>>>> it.
> >>>>>>>>>>>>>>>>>>          >>>>>>> We
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>>>>>          shares
> >>>>>>>>>>>>>>>>>>          >>>>>> its
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> state
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> >>>>>>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          branching.
> >>>>>>>>>>>>>>>>>>          >>>>>>> It's
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> not
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    pretty in its current form, but I
> >>>>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>>>          demonstrates
> >>>>>>>>>>>>>>>>>>          >>>>>> its
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>>>>>>>>>> request should
> >>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>          >>>>> final
> >>>>>>>>>>>>>>>>>>          >>>>>> or
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> even a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> starting point if we go in this
> >>>>>>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>>>>>          wanted to
> >>>>>>>>>>>>>>>>>>          >>>>>> see
> >>>>>>>>>>>>>>>>>>          >>>>>>>> how
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> I will say though, that I'm not
> >>>>>>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>          solution
> >>>>>>>>>>>>>>>>>>          >>>>>>> could
> >>>>>>>>>>>>>>>>>>          >>>>>>>> be
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> deprecated in favor of this, which
> >>>>>>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>>>          >>>>> suggested
> >>>>>>>>>>>>>>>>>>          >>>>>>>> was a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>          streams
> >>>>>>>>>>>>>>>>>>          >>>>>> are
> >>>>>>>>>>>>>>>>>>          >>>>>>>> not
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>>>>>          is, if we
> >>>>>>>>>>>>>>>>>>          >>>>>>> wanted
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> them back together again I don't
> >>>>>>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>>>          that.  The
> >>>>>>>>>>>>>>>>>>          >>>>>> KIP
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>          >>>>>> either
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>>>>>          table.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>          >>>>>> point.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> First, it seems that it's
> >>>>>>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>          branch API
> >>>>>>>>>>>>>>>>>>          >>>>>>> needs
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> There are two potential ways to
> >>>>>>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>>>>>          returns
> >>>>>>>>>>>>>>>>>>          >>>>>> its
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> argument
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> >>>>>>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>>>>>> The code
> >>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>      ��   >>>>> make
> >>>>>>>>>>>>>>>>>>          >>>>>>>> sense
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> all the necessary ingredients are
> >>>>>>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>          instance
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> >>>>>>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>> defined.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> >>>>>>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>>>>>          >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>>>>>          miss the
> >>>>>>>>>>>>>>>>>>          >>>>>> fact
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> that one
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> of the terminal methods should be
> >>>>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>>>>>          methods
> >>>>>>>>>>>>>>>>>>          >>>>>> are
> >>>>>>>>>>>>>>>>>>          >>>>>>>> not
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> I see your point when you are
> >>>>>>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> >>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>>          implemented the
> >>>>>>>>>>>>>>>>>>          >>>>>>> easy
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> >>>>>>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> >>>>>>>>>>>>>>>>>> method that
> >>>>>>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>>>>>          >>>>> nothing
> >>>>>>>>>>>>>>>>>>          >>>>>>>> will
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> >>>>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>>> the only
> >>>>>>>>>>>>>>>>> option
> >>>>>>>>>>>>>>>>>>          >>>>>> besides
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>>>>>          want to
> >>>>>>>>>>>>>>>>>>          >>>>>> just
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> silently
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> >>>>>>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>>>>>          predicate. 2)
> >>>>>>>>>>>>>>>>>>          >>>>>>> Throwing
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>          looks
> >>>>>>>>>>>>>>>>>>          >>>>>> like a
> >>>>>>>>>>>>>>>>>>          >>>>>>>> bad
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> >>>>>>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>>>>>          emit a
> >>>>>>>>>>>>>>>>>>          >>>>>>>> special
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> >>>>>>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>          >>>>>>> `default`
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> can
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>          >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>          >>>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>          a clear
> >>>>>>>>>>>>>>>>>>          >>>>>>> error
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> >>>>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>>>>>          >>>>> compiled
> >>>>>>>>>>>>>>>>>>          >>>>>>> and
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> run?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> >>>>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>>>          compile if
> >>>>>>>>>>>>>>>>>>          >>>>> used
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>>>>>          method chain
> >>>>>>>>>>>>>>>>>>          >>>>>>>> starting
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> >>>>>>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>>>          between
> >>>>>>>>>>>>>>>>>>          >>>>>>>> runtime
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>>>>>          >>>>> instantly
> >>>>>>>>>>>>>>>>>>          >>>>>> on
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> >>>>>>>>>>>>>>>>>> project
> >>>>>>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>>>>>          compilation
> >>>>>>>>>>>>>>>>>>          >>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>>>>>          required.
> >>>>>>>>>>>>>>>>>>          >>>>>>> But
> >>>>>>>>>>>>>>>>>>          >>>>>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> >>>>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>>>>>          >>>>>> defaultBranch
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> they
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>>>>>          just as
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> think it creates an
> >>>>>>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>>>>>          >>>>> user
> >>>>>>>>>>>>>>>>>>          >>>>>>>> could
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>> default
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>          >>>>> seems
> >>>>>>>>>>>>>>>>>>          >>>>>>> like
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>>>          which allows
> >>>>>>>>>>>>>>>>>>          >>>>>> for
> >>>>>>>>>>>>>>>>>>          >>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>>>>>          to be
> >>>>>>>>>>>>>>>>>>          >>>>>> well
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>>>          >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>>>          >>>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>>>          a clear
> >>>>>>>>>>>>>>>>>>          >>>>>>> error
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>>>>>> there is
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>          >>>>> "build
> >>>>>>>>>>>>>>>>>>          >>>>>>>> step"
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>          >>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>>>>>          >>>>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>>>          >>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>> it's
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> allow users to do other
> >>>>>>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>>>          stream.
> >>>>>>>>>>>>>>>>>>          >>>>>>> With
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>> way all
> >>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>          >>>>>>> operations
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> do -
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> want to process off the original
> >>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>>>          >>>>> times,
> >>>>>>>>>>>>>>>>>>          >>>>>>> you
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> just
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> >>>>>>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>>>>>          operations
> >>>>>>>>>>>>>>>>>>          >>>>>> on
> >>>>>>>>>>>>>>>>>>          >>>>>>> it
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> as
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> >>>>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>          always need
> >>>>>>>>>>>>>>>>>>          >>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>>>>>          >>>>> don't
> >>>>>>>>>>>>>>>>>>          >>>>>>>> know
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> >>>>>>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>>>          so we
> >>>>>>>>>>>>>>>>>>          >>>>> can
> >>>>>>>>>>>>>>>>>>          >>>>>> do
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> more with the original branch
> >>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> >>>>>>>>>>>>>>>>>> need of
> >>>>>>>>>>>>>>>>> special
> >>>>>>>>>>>>>>>>>>          >>>>> object
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>>>          >>>>> here
> >>>>>>>>>>>>>>>>>>          >>>>>> we
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>>>>>          flow,
> >>>>>>>>>>>>>>>>>>          >>>>> so
> >>>>>>>>>>>>>>>>>>          >>>>>> I
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> think
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> >>>>>>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>          API, but I
> >>>>>>>>>>>>>>>>>>          >>>>>> find
> >>>>>>>>>>>>>>>>>>          >>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>>>          contrasts the
> >>>>>>>>>>>>>>>>>>          >>>>>>> fluency
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> of
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
> >>>>>>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>>>          just call
> >>>>>>>>>>>>>>>>>>          >>>>> a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> method on
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> >>>>>>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>>>>>          cases
> >>>>>>>>>>>>>>>>>>          >>>>> are
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> I think the
> >>>>>>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>>>>>          is very
> >>>>>>>>>>>>>>>>>>          >>>>>> nice
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> >>>>>>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>          how we
> >>>>>>>>>>>>>>>>>>          >>>>>>> specify
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>>>>>          >>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>>>>>          terminated by
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>>>>>          incompatible with
> >>>>>>>>>>>>>>>>>>          >>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> current
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> >>>>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>          different
> >>>>>>>>>>>>>>>>>>          >>>>>> name,
> >>>>>>>>>>>>>>>>>>          >>>>>>>> but
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> >>>>>>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>>>>>          >>>>>> something
> >>>>>>>>>>>>>>>>>>          >>>>>>>> like
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> >>>>>>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>          KIP?  It
> >>>>>>>>>>>>>>>>>>          >>>>>> seems
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>>>          while also
> >>>>>>>>>>>>>>>>>>          >>>>>>>> allowing
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> you
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>>>>>          KBranchedStreams
> >>>>>>>>>>>>>>>>>>          >>>>>> if
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>>>>>          ks){
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>>>>>          String> ks){
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>>>>>>>>>          this::handleFirstCase)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>>>>>>>>>          this::handleSecondCase)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>>>          >>>>> takes a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> Consumer
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>          example in
> >>>>>>>>>>>>>>>>>>          >>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>>>>>> how would
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>          >>>>> handle
> >>>>>>>>>>>>>>>>>>          >>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> case
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>          >>>>>>>> processing
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>>>          >>>>>> stream
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>>>>>          we had
> >>>>>>>>>>>>>>>>>>          >>>>>>>> something
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> like
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>>>>>          >>>>> 418.
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>>>>>          >>>>> Please
> >>>>>>>>>>>>>>>>>>          >>>>>>>> take
> >>>>>>>>>>>>>>>>>>          >>>>>>>>> a
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>>>>>          feedback :)
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>>>>>          >>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>>>>>          >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >>>>>>>>>
> >>>>>>>>>>>>>>>>>>          >
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>> Attachments:
> >>>> * signature.asc
> >>
> >>
> 
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hello John,


1.
---------------------------------------------

 > Perhaps it would be better to stick with "as" for now
 > and just file a Jira to switch them all at the same time [for 
compatibility with Kotlin]

Fully agree! BTW it's really not a big problem: in Kotlin they have a 
standard workaround 
(https://kotlinlang.org/docs/reference/java-interop.html#escaping-for-java-identifiers-that-are-keywords-in-kotlin). 
So actually this should be a very low priority issue, if an issue at all.

 > I don't understand how your new proposed
 > methods would work any differently than the ones you already
 > had proposed in the KIP. It seems like you'd still have to provide
 > the generic type parameters on the first static factory call. Can you
 > explain how your new interface proposal differs from the existing KIP?

In the KIP, I didn't clarify what methods should be static. Now I 
propose the following methods:

non-static: withChain(Function), withName(String).

static: as(String), with(Function), with(Function, String).

The overloaded `with` version that provides both Function and name can 
be used without causing type inference problem!!

2.
----------------------------

 > Regarding making the K,V types covariant also, yes, that would indeed
 > be nice, but I'm not sure it will actually work.

What I'm keeping in mind is the following example: imagine

static KStream<String, Integer> func(KStream<String, Number> s) {
         return s.mapValues(n -> (Integer) n + 1);
}

BranchedKStream<String, Number> b =
     s.split().branch((k, v) -> isInteger(v),
                //Won't compile!!
                Branched.with(Me::func));

The simple workaround here is to change `func`'s return type from 
KStream<...Integer> to KStream<...Number>.

[On the other hand, we already agreed to remove `withJavaConsumer` from 
`Branched`, so during code migration I will have to modify my functions' 
return types anyway -- I mean, from `void` to `KStream`!! ]

 >  the map you're returning is Map<K,V>, and of course a K is not the 
same as "? extends K", so it doesn't seem compatible.

I think what you actually meant here is that KStream<? extends K, ? 
extends V> is not fit as a value for Map<String, KStream<K, V>>. This 
particularly is not a problem, since KStream<? extends K, ? extends V> 
can be safely explicitly cast to KStream<K, V>, and be put to the map.

BUT, I do really afraid of pitfalls of nested wildcard types. So maybe 
for now it's better to just admit that API is not absolutely perfect and 
accept it as is, that is

Function<? super KStream<K, V>, ? extends KStream<K, V>>

Regards,

Ivan


21.05.2020 17:59, John Roesler пишет:
> Hello Ivan,
> 
> Thanks for the refinement. Actually, I did not know that "as" would
> clash with a Kotlin operator. Maybe we should depart from convention
> and just avoid methods named "as" in the future.
> 
> The convention is that "as(String name)" is used for the static factory
> method, whereas "withName(String name)" is an instance method
> inherited from NamedOperation. If you wish to propose to avoid "as"
> for compatibility with Kotlin, I might suggest "fromName(String name)",
> although it's somewhat dubious, since all the other configuration
> classes use "as". Perhaps it would be better to stick with "as" for now
> and just file a Jira to switch them all at the same time.
> 
> Re. 3:
> Regarding the type inference problem, yes, it's a blemish on all of our
> configuraion objects. The problem is that Java infers the type
> based on the _first_ method in the chain. While it does consider what
> the recipient of the method result wants, it only considers the _next_
> recipient.
> 
> Thus, if you call as("foo") and immediately assign it to a
> Branched<String,String> variable, java infers the type correctly. But
> when the "next recipient" is a chained method call, like "withChain",
> then the chained method doesn't bound the type (by definition,
> withChain is defined on Branched<Object, Object>, so Java will take
> the broadest possible inferece and bind the type to
> Branched<Object, Object>, at which point, it can't be revised anymore.
> 
> As a user of Java, this is exceedingly annoying, since it doesn't seem
> that hard to recursively consider the entire context when inferring the
> generic type parameters, but this is what we have to work with.
> 
> To be honest, though, I don't understand how your new proposed
> methods would work any differently than the ones you already
> had proposed in the KIP. It seems like you'd still have to provide
> the generic type parameters on the first static factory call. Can you
> explain how your new interface proposal differs from the existing KIP?
> 
> Re. 4:
> Regarding making the K,V types covariant also, yes, that would indeed
> be nice, but I'm not sure it will actually work. You might want to give it a
> try. In the past, we've run into soe truly strange interactions between the
> Java type inferencer and lambdas (and/or anonymous inner classes) in
> combination with nested covariant types.
> 
> Another issue is that the value type of the map you're returning is
> Map<K,V>, and of course a K is not the same as "? extends K", so it
> doesn't seem compatible.
> 
> Thanks again,
> -John
> 
> On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
>> Hi,
>>
>> Thanks Matthias for your suggestion: yes, I agree that getting rid of
>> `with[Java]Consumer` makes this thing 'as simple as possible, but not
>> simpler'.
>>
>> I made some quick API mocking in my IDE and tried to implement examples
>> from KIP.
>>
>> 1. Having to return something from lambda is not a very big deal.
>>
>> 2. For a moment I thouht that I won't be able to use method references
>> for already written stream consumers, but then I realized that I can
>> just change my methods from returning void to returning the input
>> parameter and use references to them. Not very convenient, but passable.
>>
>> So, I'm ready to agree: 1) we use only functions, no consumer 2) when
>> function returns null, we don't insert it into the resulting map.
>>
>> Usually it's better to implement a non-perfect, but workable solution as
>> a first approximation. And later we can always add to `Branched`
>> anything we want.
>>
>> 3. Do we have any guidelines on how parameter classes like Branched
>> should be built? First of all, it seems that `as` now is more preferred
>> than `withName` (although as you probably know it clashes with Kotlin's
>> `as` operator).
>>
>> Then, while trying to mock the APIs, I found out that my Java cannot
>> infer types in the following construction:
>>
>> .branch((key, value) -> value == null,
>>      Branched.as("foo").withChain(s -> s.mapValues(...)))
>>
>>
>> so I have to write
>>
>> .branch((key, value) -> value == null,
>>      Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
>>
>>
>> This is not tolerable IMO, so this is the list of `Branched` methods
>> that I came to (will you please validate it):
>>
>> static <K, V> Branched<K, V> as(String name);
>>
>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>> extends KStream<K, V>> chain);
>>
>> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ?
>> extends KStream<K, V>> chain, String name);
>>
>> //non-static!
>> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends
>> KStream<K, V>> chain);
>>
>>
>> 4. And one more. What do you think, do we need that flexibility:
>>
>> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
>>
>> vs.
>>
>> Function<? super KStream<? super K, ? super V>, ? extends KStream<?
>> extends K, ? extends V>> chain
>>
>> ??
>>
>> Regards,
>>
>> Ivan
>>
>>
>> 21.05.2020 6:54, John Roesler пишет:
>>> Thanks for this thought, Matthias,
>>>
>>> Your idea has a few aspects I find attractive:
>>> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
>>> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
>>> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
>>>
>>> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
>>>
>>> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
>>>
>>> Thanks again for sharing the idea,
>>> John
>>>
>>> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>>>> Thanks for updating the KIP!
>>>>
>>>> I guess the only open question is about `Branched.withJavaConsumer` and
>>>> its relationship to the returned `Map`.
>>>>
>>>> Originally, we discussed two main patterns:
>>>>
>>>>    (1) split a stream and return the substreams for futher processing
>>>>    (2) split a stream and modify the substreams with in-place method chaining
>>>>
>>>> To combine both patterns we wanted to allow for
>>>>
>>>>     -> split a stream, modify the substreams, and return the _modified_
>>>> substreams for further processing
>>>>
>>>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>>>
>>>> That is of course possible. However, it introduces some "hidded" semantics:
>>>>
>>>>    - using `withChain` I get the modified sub-stream
>>>>    - using `withJavaConsumer` I get the unmodifed sub-stream
>>>>
>>>> This seems to be quite subtle to me.
>>>>
>>>>
>>>>
>>>>   From my understanding the original idea of `withJavaConsumer` was to
>>>> model a terminal operation, ie, it should be similar to:
>>>>
>>>> Branched.withChain(s -> {
>>>>     s.to();
>>>>     return null;
>>>> })
>>>>
>>>> However, I am not sure if we should even allow `withChain()` to return
>>>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>>>> -> null` entry in the returned Map.
>>>>
>>>> Following this train of through, and if we want to allow the "return
>>>> null" pattern in general, we need `withJavaConsumer` that does not add
>>>> an entry to the Map.
>>>>
>>>> Following your proposal, the semantics of `withJavaConsumer` could also
>>>> be achieved with `withChain`:
>>>>
>>>> Branched.withChain(s -> {
>>>>     s.to();
>>>>     return s;
>>>> })
>>>>
>>>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>>>> while for the first proposal it adds new functionality (if `return null`
>>>> is not allowed, using `withChain()` is not possible to "hide a
>>>> sub-stream in the result). Furthermore, we might need to allow `return
>>>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>>>
>>>>
>>>>
>>>> I guess I can be convinced either way. However, if we follow your
>>>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>>>> benefit seems to be small? Also, having a reduced API is usually
>>>> preferable as it's simpler to learn.
>>>>
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>>
>>>>
>>>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>>>> Hello, John, hello Matthias!
>>>>>
>>>>> Thank you very much for your detailed feedback!
>>>>>
>>>>> -----------------------------------------
>>>>>
>>>>> John,
>>>>>
>>>>>> It looks like you missed my reply on Apr 23rd.
>>>>>
>>>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>>>> all the emails on the web.
>>>>>
>>>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>>>> method?
>>>>>
>>>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>>>
>>>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>>>> off of the parent KStream for the needs of dynamic branching]
>>>>>
>>>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>>>
>>>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>>>
>>>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>>>
>>>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>>>> chain and the result map OR using just the sink
>>>>>
>>>>> This is discussed below.
>>>>>
>>>>> ----------------------------------------------
>>>>>
>>>>> Mathhias,
>>>>>
>>>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>>>
>>>>> Done.
>>>>>
>>>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>>>> method]
>>>>>
>>>>> Fixed.
>>>>>
>>>>>
>>>>>> 3. Overview of newly added methods/interfaces
>>>>>
>>>>> Done in `Proposed Changes` section.
>>>>>
>>>>>
>>>>>> 4. [Concerning John's note] > I don't think that using both
>>>>> `withChain()` and `withConsumer()` is the
>>>>> issue, as the KIP clearly states that the result of `withChain()` will
>>>>> be given to the `Consumer`.
>>>>>
>>>>> Yes, I agree!
>>>>>
>>>>>> The issue is really with the `Consumer` and the returned `Map` of
>>>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>>>> implementation would be to not add the "branch" to the result map if
>>>>> `withConsumer` is used?
>>>>>
>>>>> But is it also an issue? With Kafka Streams, we can split the topology
>>>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>>>> stream in the Map, one simply does not extract it from there :-)
>>>>>
>>>>> In the current version of KIP it is assumed that the returned map
>>>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>>>> programmer, or with some default auto-generated ids. Dealing with this
>>>>> map is the user's responsibility.
>>>>>
>>>>> What seems to me to be an issue is introducing exclusions to this
>>>>> general rule, like 'swallowing' some streams by provided
>>>>> [Java]Consumers. This can make things complicated. What if a user
>>>>> provides both the name of the branch and a [Java]Consumer? What do they
>>>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>>>> There's no point in 'saving the space' in this map, so maybe just leave
>>>>> it as it is?
>>>>>
>>>>> ----
>>>>>
>>>>> I rewrote the KIP and also fixed a couple of typos.
>>>>>
>>>>> Looking forward for your feedback again!
>>>>>
>>>>> Regards,
>>>>>
>>>>> Ivan.
>>>>>
>>>>>
>>>>>
>>>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>>>> Thanks for updating the KIP!
>>>>>>
>>>>>> I also have some minor comment:
>>>>>>
>>>>>>
>>>>>>
>>>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>>>
>>>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>>>> so we just keep them.)
>>>>>>
>>>>>>
>>>>>>
>>>>>> (2) Quote:
>>>>>>
>>>>>>> Both branch and defaultBranch operations also have overloaded
>>>>>>> parameterless alternatives.
>>>>>>
>>>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>>>> as `branch()` would not be "parameterless".
>>>>>>
>>>>>>
>>>>>>
>>>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>>>> newly added and deprecated methods/classes (cf.
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>>>> 4th comment:
>>>>>>
>>>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>>>> chain and the result map OR using just the sink.
>>>>>>
>>>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>>>
>>>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>>>> it in the JavaDocs, this might be fine?
>>>>>>
>>>>>>
>>>>>>
>>>>>> (5) Reply to John's comments:
>>>>>>
>>>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>>>
>>>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>>>> node", ie., writing the KStream to a topic.
>>>>>>
>>>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>>>> Hi Ivan,
>>>>>>>
>>>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>>>> but I had a few last comments.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> John
>>>>>>>
>>>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>>>> Hello everyone,
>>>>>>>>
>>>>>>>> will someone please take a look at the reworked KIP?
>>>>>>>>
>>>>>>>> I believe that now it follows design principles and takes into account
>>>>>>>> all the arguments discussed here.
>>>>>>>>
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>>
>>>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>>>> rewritten the KIP, see
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> This version includes all the previous discussion results and follows
>>>>>>>>> the design principles, with one exception.
>>>>>>>>>
>>>>>>>>> The exception is
>>>>>>>>>
>>>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>>>
>>>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>>>> here it is justified.
>>>>>>>>>
>>>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>>>
>>>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>>>> as it
>>>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>>>
>>>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>>>> Ivan,
>>>>>>>>>>
>>>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>>>
>>>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>>>> `Named`
>>>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>>>> build on
>>>>>>>>>> this.
>>>>>>>>>>
>>>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>>>> want to follow:
>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -- might be worth to checkout.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>>>> Hi everyone!
>>>>>>>>>>>
>>>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>>>
>>>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>>>> June
>>>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>>>
>>>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>>>> this
>>>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>>>> the
>>>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>>>
>>>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>>>         .branch(....)
>>>>>>>>>>>         .defaultBranch(result::set)
>>>>>>>>>>>         .onTopOf(someStream);
>>>>>>>>>>> result.get()...
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>>>
>>>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>>>> from
>>>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>>>
>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>       -> KBranchedStream
>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>> //
>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>>>       -> KBranchedStream
>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>>
>>>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>>>> a good
>>>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>>>> you
>>>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>>>> Or,
>>>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>>>> finally,
>>>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>>>> original scope.
>>>>>>>>>>>
>>>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>>>> the
>>>>>>>>>>>> KIP
>>>>>>>>>>>> at any point.
>>>>>>>>>>>>
>>>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>>>> do so.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>>>> to mix
>>>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>>>> patterns
>>>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>>>> make
>>>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>>>> `Map` only
>>>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>>>> all of
>>>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>>>> `Named` is
>>>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>>>> counter
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>>>> names
>>>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>>>> misses to
>>>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>>>> required
>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>>>> specifying a
>>>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> About
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>>>> hence
>>>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          withChain(...);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>>>> does not
>>>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>>>> call
>>>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>>>> make sense
>>>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>>>> check
>>>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>>>> object
>>>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>>>> others,
>>>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>>>> methods that
>>>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>>>> two main
>>>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>>>> solves
>>>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>>>> to add
>>>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>>>> names for
>>>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>>>> does
>>>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>>>> map
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>>>> name,
>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>>>                    .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>>>                    .defaultBranch(...) // creates node
>>>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>>>> debate
>>>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>>>> general,
>>>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>          operator(function, config_object?) OR
>>>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>>>> variant.
>>>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>        operator(function, function, string)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>>>> specified
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>>>> arguments
>>>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>>>          withPredicate(...);
>>>>>>>>>>>>>>>          withChain(...);
>>>>>>>>>>>>>>>          withName(...);
>>>>>>>>>>>>>>>        }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>>>> more like
>>>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>>>> makes us
>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>>>> purely
>>>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>>>        -> KBranchedStream
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>>>        -> KBranchedStream
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>>>        -> Map<String,KStream>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>>>          for the dynamic case.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>>>          question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>>>          in the same scope” since merging the branches back
>>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>          seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>>>          when the branched streams are in the same scope.
>>>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>          for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>          solution - working around the scope thing is easy
>>>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>>>          to.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>>>          <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>>>          LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>>>          matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>>>          advantages than drawbacks.
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>>>          proposal is that all the returned branches are in
>>>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>>>          But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>>>          workaround for the scope problem, described in the
>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>>>          method references instead of lambdas. In real world
>>>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>>>          tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>>>          going to be clean.
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>          handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>>>          handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>>>          a handler?
>>>>>>>>>>>>>>>>>>          > - what if we misspell a name?
>>>>>>>>>>>>>>>>>>          > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>          writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>>>          "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>          beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>>>          identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>>>          all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>>>          point?
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > ---
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>          without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>>>          the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>>>          current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>>>          me address both comments here.
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>>>          for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>>>          see example below.
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>>>          Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>>>          value (say, RecordType). You can have something
>>>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>>>          have been much messier.*/
>>>>>>>>>>>>>>>>>>          > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>>>          > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>>>          >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>>>          recordType,
>>>>>>>>>>>>>>>>>>          >                     recordType::processRecords);
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > Regards,
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > Ivan
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>          > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>          >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>>>          >> current `branch()` implementation.
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>>>          thinking
>>>>>>>>>>>>>>>>>>          >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>>>          >> `branch()` statement and return a
>>>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>>>          >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>          >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>          >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>          >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>>>          >> An open question is the case for which no
>>>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>          >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>          `BranchedKStream`
>>>>>>>>>>>>>>>>>>          >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>>>          >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>>>          >> because users can just ignore the branch
>>>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>>>          `defaultBranch()`
>>>>>>>>>>>>>>>>>>          >> in the result `Map` ?
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>>>          >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>>>          argument" yet
>>>>>>>>>>>>>>>>>>          >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >> -Matthias
>>>>>>>>>>>>>>>>>>          >>
>>>>>>>>>>>>>>>>>>          >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>          >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>          that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>>>          it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>>>          The advantage of putting the predicate and its
>>>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>          (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>>>          to each other.
>>>>>>>>>>>>>>>>>>          >>>
>>>>>>>>>>>>>>>>>>          >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>>>          branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>>>          >>>
>>>>>>>>>>>>>>>>>>          >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>>>          <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>          <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>
>>>>>>>>>>>>>>>>>>          >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>>>          downstream
>>>>>>>>>>>>>>>>>>          >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>          quickly become
>>>>>>>>>>>>>>>>>>          >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>>>          >>>>
>>>>>>>>>>>>>>>>>>          >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>>>          <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>>          that sets a
>>>>>>>>>>>>>>>>>>          >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>>>          current
>>>>>>>>>>>>>>>>>>          >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>>>          decisions) is
>>>>>>>>>>>>>>>>>>          >>>>> roughly this:
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> stream.split()
>>>>>>>>>>>>>>>>>>          >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>          >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>>>          >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>          construct
>>>>>>>>>>>>>>>>>>          >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>          provides as much
>>>>>>>>>>>>>>>>>>          >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>          call
>>>>>>>>>>>>>>>>>>          >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>>>          The value it
>>>>>>>>>>>>>>>>>>          >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>>>          the same
>>>>>>>>>>>>>>>>>>          >>>>> scope.
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>          in the sense
>>>>>>>>>>>>>>>>>>          >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>          branches, but it is
>>>>>>>>>>>>>>>>>>          >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>>>          the "static"
>>>>>>>>>>>>>>>>>>          >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>>>          fluently declare and
>>>>>>>>>>>>>>>>>>          >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>>>          ignore a
>>>>>>>>>>>>>>>>>>          >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>>>          SortedMap
>>>>>>>>>>>>>>>>>>          >>>>> solution on top of it.
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>>>          SortedMap being
>>>>>>>>>>>>>>>>>>          >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>>>          like this:
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>          >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>          >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>>>          >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>>>          >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>>>          >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>          readable than
>>>>>>>>>>>>>>>>>>          >>>>> SortedMap
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Cons
>>>>>>>>>>>>>>>>>>          >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>          makes it harder
>>>>>>>>>>>>>>>>>>          >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>>>          unlike the KIP)
>>>>>>>>>>>>>>>>>>          >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>>>          existing
>>>>>>>>>>>>>>>>>>          >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>>          that's overdoing
>>>>>>>>>>>>>>>>>>          >>>>> it).
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>>>          access the
>>>>>>>>>>>>>>>>>>          >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>          possible that it
>>>>>>>>>>>>>>>>>>          >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>          left up to the
>>>>>>>>>>>>>>>>>>          >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> Paul
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>>>          <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>          >>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>          >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>          existing
>>>>>>>>>>>>>>>>>>          >>>>> branch
>>>>>>>>>>>>>>>>>>          >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>>>          proceed by
>>>>>>>>>>>>>>>>>>          >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>          important to be
>>>>>>>>>>>>>>>>>>          >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>>>          >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>>>          this KIP.
>>>>>>>>>>>>>>>>>>          >>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>          double brace
>>>>>>>>>>>>>>>>>>          >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>>>          >>>>> avoid...just
>>>>>>>>>>>>>>>>>>          >>>>>> my two cents.
>>>>>>>>>>>>>>>>>>          >>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>> Cheers,
>>>>>>>>>>>>>>>>>>          >>>>>> Sophie
>>>>>>>>>>>>>>>>>>          >>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>>>          >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>>>          <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>>>          To me,
>>>>>>>>>>>>>>>>>>          >>>>> there
>>>>>>>>>>>>>>>>>>          >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>>>          >>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>          >>>>>> warnings.
>>>>>>>>>>>>>>>>>>          >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>>>          positionally coupled
>>>>>>>>>>>>>>>>>>          >>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>>>          >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>          additional code
>>>>>>>>>>>>>>>>>>          >>>>>>> paths.
>>>>>>>>>>>>>>>>>>          >>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>>>          constructs
>>>>>>>>>>>>>>>>>>          >>>>>> would
>>>>>>>>>>>>>>>>>>          >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>>>          looks like
>>>>>>>>>>>>>>>>>>          >>>>>> this:
>>>>>>>>>>>>>>>>>>          >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>>>          Predicate<?
>>>>>>>>>>>>>>>>>>          >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>>>          >>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>>>          returns a
>>>>>>>>>>>>>>>>>>          >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>>>          >>>>>> maintained
>>>>>>>>>>>>>>>>>>          >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>          >>>>>> evaluation.
>>>>>>>>>>>>>>>>>>          >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>>>          solves
>>>>>>>>>>>>>>>>>>          >>>>>> problem
>>>>>>>>>>>>>>>>>>          >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>>>          branch you’re
>>>>>>>>>>>>>>>>>>          >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>          another
>>>>>>>>>>>>>>>>>>          >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>>>          structure, rather
>>>>>>>>>>>>>>>>>>          >>>>>> than
>>>>>>>>>>>>>>>>>>          >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>>>          >>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>>>          historically
>>>>>>>>>>>>>>>>>>          >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>>>          voluminously, but
>>>>>>>>>>>>>>>>>>          >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>>>          >>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>>>          <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>>>          >>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>>>          >>>>> operator
>>>>>>>>>>>>>>>>>>          >>>>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>>>          "Split"
>>>>>>>>>>>>>>>>>>          >>>>> seems
>>>>>>>>>>>>>>>>>>          >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>>>          >>>>> branching"
>>>>>>>>>>>>>>>>>>          >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>> stream
>>>>>>>>>>>>>>>>>>          >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>          >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>          >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>          >>>>> terminal.
>>>>>>>>>>>>>>>>>>          >>>>>>> That
>>>>>>>>>>>>>>>>>>          >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>>>          is, `branch`
>>>>>>>>>>>>>>>>>>          >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>>>          `void`, to
>>>>>>>>>>>>>>>>>>          >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>>>          definition of
>>>>>>>>>>>>>>>>>>          >>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>>>          there's no
>>>>>>>>>>>>>>>>>>          >>>>>>> default,
>>>>>>>>>>>>>>>>>>          >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>>>          record
>>>>>>>>>>>>>>>>>>          >>>>>> falls
>>>>>>>>>>>>>>>>>>          >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>          >>>>>>>> -John
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>>>          >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>>>          >>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>>>          >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>>>          should
>>>>>>>>>>>>>>>>>>          >>>>>> _not_
>>>>>>>>>>>>>>>>>>          >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>>>          `branch()` does
>>>>>>>>>>>>>>>>>>          >>>>> not
>>>>>>>>>>>>>>>>>>          >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>          >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>          >>>>> hidden
>>>>>>>>>>>>>>>>>>          >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>          >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>>>          I though
>>>>>>>>>>>>>>>>>>          >>>>>> it
>>>>>>>>>>>>>>>>>>          >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>> naming
>>>>>>>>>>>>>>>>>>          >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>          >>>>> cleaner
>>>>>>>>>>>>>>>>>>          >>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>> me:
>>>>>>>>>>>>>>>>>>          >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>>>          avoid
>>>>>>>>>>>>>>>>>>          >>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>          >>>>> cannot
>>>>>>>>>>>>>>>>>>          >>>>>>> have
>>>>>>>>>>>>>>>>>>          >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>>>          with a
>>>>>>>>>>>>>>>>>>          >>>>> short
>>>>>>>>>>>>>>>>>>          >>>>>>>> name?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>>          with all
>>>>>>>>>>>>>>>>>>          >>>>> it's
>>>>>>>>>>>>>>>>>>          >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>>>          contained in
>>>>>>>>>>>>>>>>>>          >>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>>>          >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>>>          `KBranchedStream#get(int
>>>>>>>>>>>>>>>>>>          >>>>>>> index)
>>>>>>>>>>>>>>>>>>          >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>>>          branched-KStreams. Would
>>>>>>>>>>>>>>>>>>          >>>>>> be
>>>>>>>>>>>>>>>>>>          >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>>>          that users
>>>>>>>>>>>>>>>>>>          >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>>>          access them.
>>>>>>>>>>>>>>>>>>          >>>>> We
>>>>>>>>>>>>>>>>>>          >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>>>          >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>>>          to get
>>>>>>>>>>>>>>>>>>          >>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>>>          >>>>>>> String#split
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>>>          loss of
>>>>>>>>>>>>>>>>>>          >>>>>>>> backwards
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>>>          without
>>>>>>>>>>>>>>>>>>          >>>>>>>> affecting
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>          >>>>> should
>>>>>>>>>>>>>>>>>>          >>>>>>> be
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>>          >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>          >>>>>>> however, a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>>>          with such
>>>>>>>>>>>>>>>>>>          >>>>>>> name
>>>>>>>>>>>>>>>>>>          >>>>>>>>> :-)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>          >>>>> think
>>>>>>>>>>>>>>>>>>          >>>>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>>>          something.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>> everybody
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>          >>>>>> optimal.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>          proposal.
>>>>>>>>>>>>>>>>>>          >>>>>>> There
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>          recommend the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>>>          BranchingKStream#default()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>>>          `Predicate` as
>>>>>>>>>>>>>>>>>>          >>>>>>>> argument,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>>>          accepted and
>>>>>>>>>>>>>>>>>>          >>>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>>>          parameter.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>>>          different
>>>>>>>>>>>>>>>>>>          >>>>>>>> scopes:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>>>          index)` method
>>>>>>>>>>>>>>>>>>          >>>>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>>>          object?
>>>>>>>>>>>>>>>>>>          >>>>>> Maybe,
>>>>>>>>>>>>>>>>>>          >>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>>>          >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>>>          >>>>>>>> but
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>          whatever
>>>>>>>>>>>>>>>>>>          >>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>          current
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>          makes sense
>>>>>>>>>>>>>>>>>>          >>>>>> for
>>>>>>>>>>>>>>>>>>          >>>>>>>> you
>>>>>>>>>>>>>>>>>>          >>>>>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>>>          we'll
>>>>>>>>>>>>>>>>>>          >>>>> need
>>>>>>>>>>>>>>>>>>          >>>>>>>> some
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>>>          >>>>> whether
>>>>>>>>>>>>>>>>>>          >>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>>>          think this
>>>>>>>>>>>>>>>>>>          >>>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>>> a
>>>>>>>>>>>>>>>>>>          >>>>>>>>> good
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>>>          starting a
>>>>>>>>>>>>>>>>>>          >>>>>>> vote,
>>>>>>>>>>>>>>>>>>          >>>>>>>>> or if
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>>>          assuming
>>>>>>>>>>>>>>>>>>          >>>>> we
>>>>>>>>>>>>>>>>>>          >>>>>> do
>>>>>>>>>>>>>>>>>>          >>>>>>>>> move
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>>>          >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>>>          >>>>>>> do
>>>>>>>>>>>>>>>>>>          >>>>>>>>> we
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>          >>>>> favor
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>> accomplish
>>>>>>>>>>>>>>>>>>          >>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>>>          similar
>>>>>>>>>>>>>>>>>>          >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>          >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>>>          >>>>>>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>>>          >>>>> it's
>>>>>>>>>>>>>>>>>>          >>>>>>>> just a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>>>          >>>>>>> particular,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>          >>>>>> modified
>>>>>>>>>>>>>>>>>>          >>>>>>>> in
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>>>          KStreamLazyBranch is
>>>>>>>>>>>>>>>>>>          >>>>> a
>>>>>>>>>>>>>>>>>>          >>>>>>> bit
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>          >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>>>          convinced: your
>>>>>>>>>>>>>>>>>>          >>>>>>>> proposal
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>> crucial
>>>>>>>>>>>>>>>>>>          >>>>>>>>> fact
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>>>          And then
>>>>>>>>>>>>>>>>>>          >>>>>> it's
>>>>>>>>>>>>>>>>>>          >>>>>>>> all
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>>>          resume the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>>>          >>>>> starting
>>>>>>>>>>>>>>>>>>          >>>>>>>> point
>>>>>>>>>>>>>>>>>>          >>>>>>>>> if
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>>>          starting
>>>>>>>>>>>>>>>>>>          >>>>>>> point.
>>>>>>>>>>>>>>>>>>          >>>>>>>>> But
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>          >>>>> details.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>>>          >>>>> stream.branch()
>>>>>>>>>>>>>>>>>>          >>>>>>>>> solution
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>>>          >>>>> invoked
>>>>>>>>>>>>>>>>>>          >>>>>> as
>>>>>>>>>>>>>>>>>>          >>>>>>>>> they’re
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>>>          still
>>>>>>>>>>>>>>>>>>          >>>>>> ought
>>>>>>>>>>>>>>>>>>          >>>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>> be
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>          depend on
>>>>>>>>>>>>>>>>>>          >>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>>>          access
>>>>>>>>>>>>>>>>>>          >>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> branched
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>>>          is, not
>>>>>>>>>>>>>>>>>>          >>>>>>>> inside
>>>>>>>>>>>>>>>>>>          >>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>          >>>>> solutions.
>>>>>>>>>>>>>>>>>>          >>>>>> It
>>>>>>>>>>>>>>>>>>          >>>>>>>>> can be
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>>>          excited
>>>>>>>>>>>>>>>>>>          >>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>> hear
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>          >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>          >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>>          >>>>> glance,
>>>>>>>>>>>>>>>>>>          >>>>>>> but
>>>>>>>>>>>>>>>>>>          >>>>>>>>> ---
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>>>          same
>>>>>>>>>>>>>>>>>>          >>>>>> scope
>>>>>>>>>>>>>>>>>>          >>>>>>> as
>>>>>>>>>>>>>>>>>>          >>>>>>>>> each
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>>>          >>>>> again
>>>>>>>>>>>>>>>>>>          >>>>>> I
>>>>>>>>>>>>>>>>>>          >>>>>>>>> don't see
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>>>          just
>>>>>>>>>>>>>>>>>>          >>>>>> going
>>>>>>>>>>>>>>>>>>          >>>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>>>          we need
>>>>>>>>>>>>>>>>>>          >>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>> identify
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>>>          in the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> electronics
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>          circumstances
>>>>>>>>>>>>>>>>>>          >>>>>> using
>>>>>>>>>>>>>>>>>>          >>>>>>>> my
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>          complex, so
>>>>>>>>>>>>>>>>>>          >>>>>>>>> creation of
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>>>          order to
>>>>>>>>>>>>>>>>>>          >>>>>>>> separate
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>>>          >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>>>          everything
>>>>>>>>>>>>>>>>>>          >>>>>>>> later,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>>>          initialize the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>          >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>>>          essentially
>>>>>>>>>>>>>>>>>>          >>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> next
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>>>          based on
>>>>>>>>>>>>>>>>>>          >>>>> my
>>>>>>>>>>>>>>>>>>          >>>>>>>>> experience,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>>>          fluent
>>>>>>>>>>>>>>>>>>          >>>>> API
>>>>>>>>>>>>>>>>>>          >>>>>>>> based
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>>>          (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>>>          >>>>>> and
>>>>>>>>>>>>>>>>>>          >>>>>>> I
>>>>>>>>>>>>>>>>>>          >>>>>>>>> think
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>>>          >>>>>>> compatibility
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>>>          that Java
>>>>>>>>>>>>>>>>>>          >>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>>>> smart
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>>>          returning one
>>>>>>>>>>>>>>>>>>          >>>>>>> thing
>>>>>>>>>>>>>>>>>>          >>>>>>>>> and
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>          need
>>>>>>>>>>>>>>>>>>          >>>>> it.
>>>>>>>>>>>>>>>>>>          >>>>>>> We
>>>>>>>>>>>>>>>>>>          >>>>>>>>> can
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>>>          shares
>>>>>>>>>>>>>>>>>>          >>>>>> its
>>>>>>>>>>>>>>>>>>          >>>>>>>>> state
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          branching.
>>>>>>>>>>>>>>>>>>          >>>>>>> It's
>>>>>>>>>>>>>>>>>>          >>>>>>>>> not
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>>>          demonstrates
>>>>>>>>>>>>>>>>>>          >>>>>> its
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>          >>>>> final
>>>>>>>>>>>>>>>>>>          >>>>>> or
>>>>>>>>>>>>>>>>>>          >>>>>>>>> even a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>>>          wanted to
>>>>>>>>>>>>>>>>>>          >>>>>> see
>>>>>>>>>>>>>>>>>>          >>>>>>>> how
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>          solution
>>>>>>>>>>>>>>>>>>          >>>>>>> could
>>>>>>>>>>>>>>>>>>          >>>>>>>> be
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>>>          >>>>> suggested
>>>>>>>>>>>>>>>>>>          >>>>>>>> was a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>          streams
>>>>>>>>>>>>>>>>>>          >>>>>> are
>>>>>>>>>>>>>>>>>>          >>>>>>>> not
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>>>          is, if we
>>>>>>>>>>>>>>>>>>          >>>>>>> wanted
>>>>>>>>>>>>>>>>>>          >>>>>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>>>          that.  The
>>>>>>>>>>>>>>>>>>          >>>>>> KIP
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>          >>>>>> either
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>>>          table.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>          >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>          >>>>>> point.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>          branch API
>>>>>>>>>>>>>>>>>>          >>>>>>> needs
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>>>          returns
>>>>>>>>>>>>>>>>>>          >>>>>> its
>>>>>>>>>>>>>>>>>>          >>>>>>>>> argument
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>      ��   >>>>> make
>>>>>>>>>>>>>>>>>>          >>>>>>>> sense
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>          instance
>>>>>>>>>>>>>>>>>>          >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>>>          >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>> defined.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>          >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>>>          >>>>>>>>> and
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>>>          miss the
>>>>>>>>>>>>>>>>>>          >>>>>> fact
>>>>>>>>>>>>>>>>>>          >>>>>>>>> that one
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>>>          methods
>>>>>>>>>>>>>>>>>>          >>>>>> are
>>>>>>>>>>>>>>>>>>          >>>>>>>> not
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>>          implemented the
>>>>>>>>>>>>>>>>>>          >>>>>>> easy
>>>>>>>>>>>>>>>>>>          >>>>>>>>> way.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>>>          >>>>> nothing
>>>>>>>>>>>>>>>>>>          >>>>>>>> will
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>>>          >>>>>> besides
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>>>          want to
>>>>>>>>>>>>>>>>>>          >>>>>> just
>>>>>>>>>>>>>>>>>>          >>>>>>>>> silently
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>>>          predicate. 2)
>>>>>>>>>>>>>>>>>>          >>>>>>> Throwing
>>>>>>>>>>>>>>>>>>          >>>>>>>>> an
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>          looks
>>>>>>>>>>>>>>>>>>          >>>>>> like a
>>>>>>>>>>>>>>>>>>          >>>>>>>> bad
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>>>          emit a
>>>>>>>>>>>>>>>>>>          >>>>>>>> special
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>          >>>>>>> `default`
>>>>>>>>>>>>>>>>>>          >>>>>>>>> can
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>          >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>          >>>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>> track
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>          a clear
>>>>>>>>>>>>>>>>>>          >>>>>>> error
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>>>          >>>>> compiled
>>>>>>>>>>>>>>>>>>          >>>>>>> and
>>>>>>>>>>>>>>>>>>          >>>>>>>>> run?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>>>          compile if
>>>>>>>>>>>>>>>>>>          >>>>> used
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>>>          method chain
>>>>>>>>>>>>>>>>>>          >>>>>>>> starting
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>>>          between
>>>>>>>>>>>>>>>>>>          >>>>>>>> runtime
>>>>>>>>>>>>>>>>>>          >>>>>>>>> and
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>>>          >>>>> instantly
>>>>>>>>>>>>>>>>>>          >>>>>> on
>>>>>>>>>>>>>>>>>>          >>>>>>>>> unit
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>>>          compilation
>>>>>>>>>>>>>>>>>>          >>>>>>>> failure.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>>>          required.
>>>>>>>>>>>>>>>>>>          >>>>>>> But
>>>>>>>>>>>>>>>>>>          >>>>>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>>>          >>>>>> defaultBranch
>>>>>>>>>>>>>>>>>>          >>>>>>>>> they
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>>>          just as
>>>>>>>>>>>>>>>>>>          >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>>>          >>>>> user
>>>>>>>>>>>>>>>>>>          >>>>>>>> could
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>> default
>>>>>>>>>>>>>>>>>>          >>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>          >>>>> seems
>>>>>>>>>>>>>>>>>>          >>>>>>> like
>>>>>>>>>>>>>>>>>>          >>>>>>>>> an
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>          which allows
>>>>>>>>>>>>>>>>>>          >>>>>> for
>>>>>>>>>>>>>>>>>>          >>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>>>          to be
>>>>>>>>>>>>>>>>>>          >>>>>> well
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>>>          >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>>>          >>>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>> track
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>>>          a clear
>>>>>>>>>>>>>>>>>>          >>>>>>> error
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>          >>>>> "build
>>>>>>>>>>>>>>>>>>          >>>>>>>> step"
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>          >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>>>          >>>>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>>>          >>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>> it's
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>>>          stream.
>>>>>>>>>>>>>>>>>>          >>>>>>> With
>>>>>>>>>>>>>>>>>>          >>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>          >>>>>>> operations
>>>>>>>>>>>>>>>>>>          >>>>>>>>> do -
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>          >>>>> times,
>>>>>>>>>>>>>>>>>>          >>>>>>> you
>>>>>>>>>>>>>>>>>>          >>>>>>>>> just
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>>>          operations
>>>>>>>>>>>>>>>>>>          >>>>>> on
>>>>>>>>>>>>>>>>>>          >>>>>>> it
>>>>>>>>>>>>>>>>>>          >>>>>>>>> as
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>>>          >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>          always need
>>>>>>>>>>>>>>>>>>          >>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>>>          >>>>> don't
>>>>>>>>>>>>>>>>>>          >>>>>>>> know
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>>>          so we
>>>>>>>>>>>>>>>>>>          >>>>> can
>>>>>>>>>>>>>>>>>>          >>>>>> do
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>>>          >>>>> object
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>          >>>>> here
>>>>>>>>>>>>>>>>>>          >>>>>> we
>>>>>>>>>>>>>>>>>>          >>>>>>>>> have a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>>>          flow,
>>>>>>>>>>>>>>>>>>          >>>>> so
>>>>>>>>>>>>>>>>>>          >>>>>> I
>>>>>>>>>>>>>>>>>>          >>>>>>>>> think
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>          API, but I
>>>>>>>>>>>>>>>>>>          >>>>>> find
>>>>>>>>>>>>>>>>>>          >>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>>>          contrasts the
>>>>>>>>>>>>>>>>>>          >>>>>>> fluency
>>>>>>>>>>>>>>>>>>          >>>>>>>>> of
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>>>          just call
>>>>>>>>>>>>>>>>>>          >>>>> a
>>>>>>>>>>>>>>>>>>          >>>>>>>>> method on
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>>>          cases
>>>>>>>>>>>>>>>>>>          >>>>> are
>>>>>>>>>>>>>>>>>>          >>>>>>>>> defined
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>>>          is very
>>>>>>>>>>>>>>>>>>          >>>>>> nice
>>>>>>>>>>>>>>>>>>          >>>>>>>>> and the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>          how we
>>>>>>>>>>>>>>>>>>          >>>>>>> specify
>>>>>>>>>>>>>>>>>>          >>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>>>          >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>>>          >>>>>>>>> or
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>>>          terminated by
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>>>          incompatible with
>>>>>>>>>>>>>>>>>>          >>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> current
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>          different
>>>>>>>>>>>>>>>>>>          >>>>>> name,
>>>>>>>>>>>>>>>>>>          >>>>>>>> but
>>>>>>>>>>>>>>>>>>          >>>>>>>>> that
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>>>          >>>>>> something
>>>>>>>>>>>>>>>>>>          >>>>>>>> like
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>          KIP?  It
>>>>>>>>>>>>>>>>>>          >>>>>> seems
>>>>>>>>>>>>>>>>>>          >>>>>>>>> like it
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>>>          while also
>>>>>>>>>>>>>>>>>>          >>>>>>>> allowing
>>>>>>>>>>>>>>>>>>          >>>>>>>>> you
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>>>          KBranchedStreams
>>>>>>>>>>>>>>>>>>          >>>>>> if
>>>>>>>>>>>>>>>>>>          >>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>>>          ks){
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>>>          String> ks){
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>>>          this::handleFirstCase)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>>>          this::handleSecondCase)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>>>          >>>>> takes a
>>>>>>>>>>>>>>>>>>          >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>          example in
>>>>>>>>>>>>>>>>>>          >>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>> KIP
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>>>          >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>          >>>>> handle
>>>>>>>>>>>>>>>>>>          >>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>> case
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>          >>>>>>>> processing
>>>>>>>>>>>>>>>>>>          >>>>>>>>> and
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>>>          >>>>>> stream
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>>>          we had
>>>>>>>>>>>>>>>>>>          >>>>>>>> something
>>>>>>>>>>>>>>>>>>          >>>>>>>>> like
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>          >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>>>          >>>>> 418.
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>>>          >>>>> Please
>>>>>>>>>>>>>>>>>>          >>>>>>>> take
>>>>>>>>>>>>>>>>>>          >>>>>>>>> a
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>>>          feedback :)
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>>>          >>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>>>          >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>          >>>>>>>>>
>>>>>>>>>>>>>>>>>>          >
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>> Attachments:
>>>> * signature.asc
>>
>>



Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Hello Ivan,

Thanks for the refinement. Actually, I did not know that "as" would
clash with a Kotlin operator. Maybe we should depart from convention
and just avoid methods named "as" in the future.

The convention is that "as(String name)" is used for the static factory
method, whereas "withName(String name)" is an instance method
inherited from NamedOperation. If you wish to propose to avoid "as"
for compatibility with Kotlin, I might suggest "fromName(String name)",
although it's somewhat dubious, since all the other configuration
classes use "as". Perhaps it would be better to stick with "as" for now
and just file a Jira to switch them all at the same time.

Re. 3:
Regarding the type inference problem, yes, it's a blemish on all of our
configuraion objects. The problem is that Java infers the type
based on the _first_ method in the chain. While it does consider what
the recipient of the method result wants, it only considers the _next_
recipient.

Thus, if you call as("foo") and immediately assign it to a 
Branched<String,String> variable, java infers the type correctly. But
when the "next recipient" is a chained method call, like "withChain",
then the chained method doesn't bound the type (by definition, 
withChain is defined on Branched<Object, Object>, so Java will take
the broadest possible inferece and bind the type to
Branched<Object, Object>, at which point, it can't be revised anymore.

As a user of Java, this is exceedingly annoying, since it doesn't seem
that hard to recursively consider the entire context when inferring the
generic type parameters, but this is what we have to work with.

To be honest, though, I don't understand how your new proposed
methods would work any differently than the ones you already
had proposed in the KIP. It seems like you'd still have to provide
the generic type parameters on the first static factory call. Can you
explain how your new interface proposal differs from the existing KIP?

Re. 4:
Regarding making the K,V types covariant also, yes, that would indeed
be nice, but I'm not sure it will actually work. You might want to give it a
try. In the past, we've run into soe truly strange interactions between the
Java type inferencer and lambdas (and/or anonymous inner classes) in
combination with nested covariant types.

Another issue is that the value type of the map you're returning is
Map<K,V>, and of course a K is not the same as "? extends K", so it
doesn't seem compatible.

Thanks again,
-John

On Thu, May 21, 2020, at 04:20, Ivan Ponomarev wrote:
> Hi,
> 
> Thanks Matthias for your suggestion: yes, I agree that getting rid of 
> `with[Java]Consumer` makes this thing 'as simple as possible, but not 
> simpler'.
> 
> I made some quick API mocking in my IDE and tried to implement examples 
> from KIP.
> 
> 1. Having to return something from lambda is not a very big deal.
> 
> 2. For a moment I thouht that I won't be able to use method references 
> for already written stream consumers, but then I realized that I can 
> just change my methods from returning void to returning the input 
> parameter and use references to them. Not very convenient, but passable.
> 
> So, I'm ready to agree: 1) we use only functions, no consumer 2) when 
> function returns null, we don't insert it into the resulting map.
> 
> Usually it's better to implement a non-perfect, but workable solution as 
> a first approximation. And later we can always add to `Branched` 
> anything we want.
> 
> 3. Do we have any guidelines on how parameter classes like Branched 
> should be built? First of all, it seems that `as` now is more preferred 
> than `withName` (although as you probably know it clashes with Kotlin's 
> `as` operator).
> 
> Then, while trying to mock the APIs, I found out that my Java cannot 
> infer types in the following construction:
> 
> .branch((key, value) -> value == null,
>     Branched.as("foo").withChain(s -> s.mapValues(...)))
> 
> 
> so I have to write
> 
> .branch((key, value) -> value == null,
>     Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))
> 
> 
> This is not tolerable IMO, so this is the list of `Branched` methods 
> that I came to (will you please validate it):
> 
> static <K, V> Branched<K, V> as(String name);
> 
> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ? 
> extends KStream<K, V>> chain);
> 
> static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ? 
> extends KStream<K, V>> chain, String name);
> 
> //non-static!
> Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends 
> KStream<K, V>> chain);
> 
> 
> 4. And one more. What do you think, do we need that flexibility:
> 
> Function<? super KStream<K, V>, ? extends KStream<K, V>> chain
> 
> vs.
> 
> Function<? super KStream<? super K, ? super V>, ? extends KStream<? 
> extends K, ? extends V>> chain
> 
> ??
> 
> Regards,
> 
> Ivan
> 
> 
> 21.05.2020 6:54, John Roesler пишет:
> > Thanks for this thought, Matthias,
> > 
> > Your idea has a few aspects I find attractive:
> > 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
> > 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
> > 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
> > 
> > The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
> > 
> > Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
> > 
> > Thanks again for sharing the idea,
> > John
> > 
> > On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> >> Thanks for updating the KIP!
> >>
> >> I guess the only open question is about `Branched.withJavaConsumer` and
> >> its relationship to the returned `Map`.
> >>
> >> Originally, we discussed two main patterns:
> >>
> >>   (1) split a stream and return the substreams for futher processing
> >>   (2) split a stream and modify the substreams with in-place method chaining
> >>
> >> To combine both patterns we wanted to allow for
> >>
> >>    -> split a stream, modify the substreams, and return the _modified_
> >> substreams for further processing
> >>
> >>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> >>
> >> That is of course possible. However, it introduces some "hidded" semantics:
> >>
> >>   - using `withChain` I get the modified sub-stream
> >>   - using `withJavaConsumer` I get the unmodifed sub-stream
> >>
> >> This seems to be quite subtle to me.
> >>
> >>
> >>
> >>  From my understanding the original idea of `withJavaConsumer` was to
> >> model a terminal operation, ie, it should be similar to:
> >>
> >> Branched.withChain(s -> {
> >>    s.to();
> >>    return null;
> >> })
> >>
> >> However, I am not sure if we should even allow `withChain()` to return
> >> `null`? IMHO, we should throw an exception for this case to avoid a `key
> >> -> null` entry in the returned Map.
> >>
> >> Following this train of through, and if we want to allow the "return
> >> null" pattern in general, we need `withJavaConsumer` that does not add
> >> an entry to the Map.
> >>
> >> Following your proposal, the semantics of `withJavaConsumer` could also
> >> be achieved with `withChain`:
> >>
> >> Branched.withChain(s -> {
> >>    s.to();
> >>    return s;
> >> })
> >>
> >> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> >> while for the first proposal it adds new functionality (if `return null`
> >> is not allowed, using `withChain()` is not possible to "hide a
> >> sub-stream in the result). Furthermore, we might need to allow `return
> >> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> >>
> >>
> >>
> >> I guess I can be convinced either way. However, if we follow your
> >> proposal, I am wondering if we need `withJavaConsumer` at all? Its
> >> benefit seems to be small? Also, having a reduced API is usually
> >> preferable as it's simpler to learn.
> >>
> >>
> >>
> >> -Matthias
> >>
> >>
> >>
> >>
> >> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> >>> Hello, John, hello Matthias!
> >>>
> >>> Thank you very much for your detailed feedback!
> >>>
> >>> -----------------------------------------
> >>>
> >>> John,
> >>>
> >>>> It looks like you missed my reply on Apr 23rd.
> >>>
> >>> For some unknown reason it didn't reach my inbox, fortunately we have
> >>> all the emails on the web.
> >>>
> >>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> >>> method?
> >>>
> >>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
> >>>
> >>>> 2. [Explain why 'branch' operator is superior to branching directly
> >>> off of the parent KStream for the needs of dynamic branching]
> >>>
> >>> Done, see an ugly counterexample in 'Dynamic Branching' section.
> >>>
> >>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> >>>
> >>> As Mathhias noted, `withSink` can also be confusing. I renamed this
> >>> method to `withJavaConsumer` per Matthias' suggestion.
> >>>
> >>>> 4. ...It seems like there are two disjoint use cases: EITHER using
> >>> chain and the result map OR using just the sink
> >>>
> >>> This is discussed below.
> >>>
> >>> ----------------------------------------------
> >>>
> >>> Mathhias,
> >>>
> >>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> >>>
> >>> Done.
> >>>
> >>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> >>> method]
> >>>
> >>> Fixed.
> >>>
> >>>
> >>>> 3. Overview of newly added methods/interfaces
> >>>
> >>> Done in `Proposed Changes` section.
> >>>
> >>>
> >>>> 4. [Concerning John's note] > I don't think that using both
> >>> `withChain()` and `withConsumer()` is the
> >>> issue, as the KIP clearly states that the result of `withChain()` will
> >>> be given to the `Consumer`.
> >>>
> >>> Yes, I agree!
> >>>
> >>>> The issue is really with the `Consumer` and the returned `Map` of
> >>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> >>> implementation would be to not add the "branch" to the result map if
> >>> `withConsumer` is used?
> >>>
> >>> But is it also an issue? With Kafka Streams, we can split the topology
> >>> graph at any point. Technically, it's OK to do both: feed the KStream to
> >>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> >>> stream in the Map, one simply does not extract it from there :-)
> >>>
> >>> In the current version of KIP it is assumed that the returned map
> >>> contains ALL the branches, either tagged with IDs explicitly set by the
> >>> programmer, or with some default auto-generated ids. Dealing with this
> >>> map is the user's responsibility.
> >>>
> >>> What seems to me to be an issue is introducing exclusions to this
> >>> general rule, like 'swallowing' some streams by provided
> >>> [Java]Consumers. This can make things complicated. What if a user
> >>> provides both the name of the branch and a [Java]Consumer? What do they
> >>> mean in this case? Should we 'swallow' the stream or save it to the map?
> >>> There's no point in 'saving the space' in this map, so maybe just leave
> >>> it as it is?
> >>>
> >>> ----
> >>>
> >>> I rewrote the KIP and also fixed a couple of typos.
> >>>
> >>> Looking forward for your feedback again!
> >>>
> >>> Regards,
> >>>
> >>> Ivan.
> >>>
> >>>
> >>>
> >>> 08.05.2020 22:55, Matthias J. Sax пишет:
> >>>> Thanks for updating the KIP!
> >>>>
> >>>> I also have some minor comment:
> >>>>
> >>>>
> >>>>
> >>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> >>>>
> >>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> >>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> >>>> and `KGroupedKTable` that we cannot rename without a breaking change...
> >>>> so we just keep them.)
> >>>>
> >>>>
> >>>>
> >>>> (2) Quote:
> >>>>
> >>>>> Both branch and defaultBranch operations also have overloaded
> >>>>> parameterless alternatives.
> >>>>
> >>>> I think `branch()` always needs to take a `Predicate` and assume you
> >>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> >>>> as `branch()` would not be "parameterless".
> >>>>
> >>>>
> >>>>
> >>>> (3) Can you maybe add an overview in the "Public Interface" section) of
> >>>> newly added and deprecated methods/classes (cf.
> >>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> >>>> and the finally returned `Map<String, KStream>`. This related to John's
> >>>> 4th comment:
> >>>>
> >>>>> It seems like there are really two disjoint use cases: EITHER using
> >>>>> chain and the result map OR using just the sink.
> >>>>
> >>>> I don't think that using both `withChain()` and `withConsumer()` is the
> >>>> issue though, as the KIP clearly states that the result of `withChain()`
> >>>> will be given to the `Consumer`. The issue is really with the `Consumer`
> >>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> >>>>
> >>>> Maybe a reasonable implementation would be to not add the "branch" to
> >>>> the result map if `withConsumer` is used? As long as we clearly document
> >>>> it in the JavaDocs, this might be fine?
> >>>>
> >>>>
> >>>>
> >>>> (5) Reply to John's comments:
> >>>>
> >>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> >>>>> were talking about the kafka Consumer interface (which doesn’t make
> >>>>> sense, of course). I get that you were referring to the java Consumer
> >>>>> interface, but we should still probably to to avoid the ambiguity.
> >>>>> Just throwing out a suggestion, how about ‘withSink’?
> >>>>
> >>>> IMHO, `withSink` has the issue that it might be confused with a "sink
> >>>> node", ie., writing the KStream to a topic.
> >>>>
> >>>> Maybe `withJavaConsumer` would make it less ambiguous?
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On 5/8/20 7:13 AM, John Roesler wrote:
> >>>>> Hi Ivan,
> >>>>>
> >>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> >>>>> but I had a few last comments.
> >>>>>
> >>>>> Thanks,
> >>>>> John
> >>>>>
> >>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> >>>>>> Hello everyone,
> >>>>>>
> >>>>>> will someone please take a look at the reworked KIP?
> >>>>>>
> >>>>>> I believe that now it follows design principles and takes into account
> >>>>>> all the arguments discussed here.
> >>>>>>
> >>>>>>
> >>>>>> Regards,
> >>>>>>
> >>>>>> Ivan
> >>>>>>
> >>>>>>
> >>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>>>> Hi,
> >>>>>>>
> >>>>>>> I have read the John's "DSL design principles" and have completely
> >>>>>>> rewritten the KIP, see
> >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> This version includes all the previous discussion results and follows
> >>>>>>> the design principles, with one exception.
> >>>>>>>
> >>>>>>> The exception is
> >>>>>>>
> >>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> >>>>>>>
> >>>>>>> which formally violates 'no more than one parameter' rule, but I think
> >>>>>>> here it is justified.
> >>>>>>>
> >>>>>>> We must provide a predicate for a branch and don't need to provide one
> >>>>>>> for the default branch. Thus for both operations we may use a single
> >>>>>>> Branched parameter class, with an extra method parameter for `branch`.
> >>>>>>>
> >>>>>>> Since predicate is a natural, necessary part of a branch, no
> >>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> >>>>>>> as it
> >>>>>>> is said in the rationale for the 'single parameter rule'.
> >>>>>>>
> >>>>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>>
> >>>>>>> Ivan
> >>>>>>>
> >>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>>>> Ivan,
> >>>>>>>>
> >>>>>>>> no worries about getting side tracked. Glad to have you back!
> >>>>>>>>
> >>>>>>>> The DSL improved further in the meantime and we already have a
> >>>>>>>> `Named`
> >>>>>>>> config object to name operators. It seems reasonable to me to
> >>>>>>>> build on
> >>>>>>>> this.
> >>>>>>>>
> >>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> >>>>>>>> want to follow:
> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -- might be worth to checkout.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>>>> Hi everyone!
> >>>>>>>>>
> >>>>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>>>
> >>>>>>>>> I'm very sorry for stopping my participation in the discussion in
> >>>>>>>>> June
> >>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
> >>>>>>>>> spare time. But I think I must finish this, because we invested
> >>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
> >>>>>>>>> propose other things before this one is finalized.
> >>>>>>>>>
> >>>>>>>>> During these months I proceeded with writing and reviewing Kafka
> >>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> >>>>>>>>> this
> >>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> >>>>>>>>> the
> >>>>>>>>> KIP forward. When I was coming across the problem with the scope of
> >>>>>>>>> branches, I worked around it this way:
> >>>>>>>>>
> >>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>>>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>>>        .branch(....)
> >>>>>>>>>        .defaultBranch(result::set)
> >>>>>>>>>        .onTopOf(someStream);
> >>>>>>>>> result.get()...
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> And yes, of course I don't feel very happy with this approach.
> >>>>>>>>>
> >>>>>>>>> I think that Matthias came up with a bright solution in his post
> >>>>>>>>> from
> >>>>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>>>
> >>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>      -> KBranchedStream
> >>>>>>>>> // assign a name to the branch and
> >>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>> //
> >>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>>>      -> KBranchedStream
> >>>>>>>>> // default branch is not easily accessible
> >>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>> // assign custom name to default-branch
> >>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>> // assign a default name for default
> >>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>>
> >>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
> >>>>>>>>> a good
> >>>>>>>>> idea: when you don't need to have the branches in the same scope,
> >>>>>>>>> you
> >>>>>>>>> just don't use names and you don't risk making your code brittle.
> >>>>>>>>> Or,
> >>>>>>>>> you might want to add names just for debugging purposes. Or,
> >>>>>>>>> finally,
> >>>>>>>>> you might use the returned Map to have the named branches in the
> >>>>>>>>> original scope.
> >>>>>>>>>
> >>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> >>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
> >>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
> >>>>>>>>> familiar with Streams API design principles than me.
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>>
> >>>>>>>>> Ivan
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> >>>>>>>>>> the
> >>>>>>>>>> KIP
> >>>>>>>>>> at any point.
> >>>>>>>>>>
> >>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> >>>>>>>>>> do so.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>>>> Ivan,
> >>>>>>>>>>>
> >>>>>>>>>>> did you see my last reply? What do you think about my proposal
> >>>>>>>>>>> to mix
> >>>>>>>>>>> both approaches and try to get best-of-both worlds?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> -Matthias
> >>>>>>>>>>>
> >>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>>>
> >>>>>>>>>>>>> under your suggestion, it seems that the name is required
> >>>>>>>>>>>>
> >>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> >>>>>>>>>>>> using a
> >>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> >>>>>>>>>>>> using a
> >>>>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> >>>>>>>>>>>> patterns
> >>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
> >>>>>>>>>>>> make
> >>>>>>>>>>>> totally sense.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>>>>>>>
> >>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> >>>>>>>>>>>> `Map` only
> >>>>>>>>>>>> contains the corresponding branches, so why should we prefix
> >>>>>>>>>>>> all of
> >>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>>>
> >>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> >>>>>>>>>>>> `Named` is
> >>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> >>>>>>>>>>>> counter
> >>>>>>>>>>>> for
> >>>>>>>>>>>> different branch name. However, this might lead to the problem of
> >>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
> >>>>>>>>>>>> names
> >>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>>>
> >>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
> >>>>>>>>>>>> misses to
> >>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> >>>>>>>>>>>> keep
> >>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> >>>>>>>>>>>> required
> >>>>>>>>>>>> if a
> >>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>>>>>>>> specifying a
> >>>>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> About
> >>>>>>>>>>>>
> >>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>
> >>>>>>>>>>>> I don't think that the branching predicate is a configuration and
> >>>>>>>>>>>> hence
> >>>>>>>>>>>> would not include it in a configuration object.
> >>>>>>>>>>>>
> >>>>>>>>>>>>>         withChain(...);
> >>>>>>>>>>>>
> >>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> >>>>>>>>>>>> does not
> >>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> >>>>>>>>>>>> call
> >>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
> >>>>>>>>>>>> make sense
> >>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> >>>>>>>>>>>> check
> >>>>>>>>>>>> seems less appealing. Also, it could happen that neither
> >>>>>>>>>>>> `withChain()`
> >>>>>>>>>>>> not `withName()` is called and the branch is missing in the
> >>>>>>>>>>>> returned
> >>>>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> >>>>>>>>>>>> object
> >>>>>>>>>>>> is helpful if each configuration can be set independently of all
> >>>>>>>>>>>> others,
> >>>>>>>>>>>> but this seems not to be the case here. If we add new
> >>>>>>>>>>>> configuration
> >>>>>>>>>>>> later, we can also just move forward by deprecating the
> >>>>>>>>>>>> methods that
> >>>>>>>>>>>> accept `Named` and add new methods that accepted
> >>>>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> >>>>>>>>>>>> two main
> >>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> >>>>>>>>>>>>> satisfy
> >>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
> >>>>>>>>>>>>> solves
> >>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> >>>>>>>>>>>>> to add
> >>>>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
> >>>>>>>>>>>>> Otherwise,
> >>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> >>>>>>>>>>>>> actually
> >>>>>>>>>>>>> not too bad, since experience has taught us that, although
> >>>>>>>>>>>>> names for
> >>>>>>>>>>>>> operations are not required to define stream processing
> >>>>>>>>>>>>> logic, it
> >>>>>>>>>>>>> does
> >>>>>>>>>>>>> significantly improve the operational experience when you can
> >>>>>>>>>>>>> map
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> >>>>>>>>>>>>> processing onto
> >>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> >>>>>>>>>>>>> names
> >>>>>>>>>>>>> built from the branch operator name. I guess under this
> >>>>>>>>>>>>> proposal, we
> >>>>>>>>>>>>> could naturally append the branch name to the branching operator
> >>>>>>>>>>>>> name,
> >>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        stream.split(Named.withName("mysplit")) //creates node
> >>>>>>>>>>>>> "mysplit"
> >>>>>>>>>>>>>                   .branch(..., ..., "abranch") // creates node
> >>>>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>>>                   .defaultBranch(...) // creates node
> >>>>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> >>>>>>>>>>>>> debate
> >>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
> >>>>>>>>>>>>> general,
> >>>>>>>>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>         operator(function, config_object?) OR
> >>>>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> where config_object is often just Named in the "function"
> >>>>>>>>>>>>> variant.
> >>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>       operator(function, function, string)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> where the string is the name.
> >>>>>>>>>>>>> My first question is whether the name should instead be
> >>>>>>>>>>>>> specified
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> My second question is whether we should just roll all these
> >>>>>>>>>>>>> arguments
> >>>>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        interface BranchConfig extends NamedOperation {
> >>>>>>>>>>>>>         withPredicate(...);
> >>>>>>>>>>>>>         withChain(...);
> >>>>>>>>>>>>>         withName(...);
> >>>>>>>>>>>>>       }
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> >>>>>>>>>>>>> more like
> >>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> This makes the source code a little noisier, but it also
> >>>>>>>>>>>>> makes us
> >>>>>>>>>>>>> more
> >>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> >>>>>>>>>>>>> purely
> >>>>>>>>>>>>> in the config interface, and never have to deal with adding
> >>>>>>>>>>>>> overloads
> >>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> WDYT?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>> -John
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> >>>>>>>>>>>>>> view.
> >>>>>>>>>>>>>> Good
> >>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> >>>>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>>>       -> KBranchedStream
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>>>> //
> >>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> >>>>>>>>>>>>>>> String)
> >>>>>>>>>>>>>>>       -> KBranchedStream
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>>>       -> Map<String,KStream>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>>>>>>>> returned
> >>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> >>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
> >>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> >>>>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>>>> their nested closure situation got out of hand.
> >>>>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> >>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> >>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>>>> the end of the story either, and you can see that by
> >>>>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> >>>>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> >>>>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> >>>>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>>>> language
> >>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> >>>>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>> programming style to solve the problem (because you get
> >>>>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> >>>>>>>>>>>>>>>> take
> >>>>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> >>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> >>>>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> >>>>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> >>>>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         Ivan, I’ll definitely forfeit my point on the
> >>>>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         branch(predicate, consumer) solution, I don’t see
> >>>>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>>>         for the dynamic case.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         IMO the one trade off to consider at this point is the
> >>>>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>>>         question. I don’t know if I totally agree that “we
> >>>>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>>>         in the same scope” since merging the branches back
> >>>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>         seems like a perfectly plausible use case that can
> >>>>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>>>         when the branched streams are in the same scope.
> >>>>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>         for the reasons Ivan listed, I think it is overall the
> >>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>         solution - working around the scope thing is easy
> >>>>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>>>         to.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>>>>>>>         <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > Hello everyone, thank you all for joining the
> >>>>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > Well, I don't think the idea of named branches,
> >>>>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>>>         LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>>>         matters) or `branch` method  taking name and Consumer
> >>>>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>>>         advantages than drawbacks.
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > In my opinion, the only real positive outcome from
> >>>>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>>>         proposal is that all the returned branches are in
> >>>>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>>>         But 1) we rarely need them in the same scope 2)
> >>>>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>>>         workaround for the scope problem, described in the
> >>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > 'Inlining the complex logic' is not a problem,
> >>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>>>         method references instead of lambdas. In real world
> >>>>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>>>         tend to split the complex logic to methods anyway,
> >>>>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>>>         going to be clean.
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > The drawbacks are strong. The cohesion between
> >>>>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>         handlers is lost. We have to define predicates in one
> >>>>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>>>         handlers in another. This opens the door for bugs:
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > - what if we forget to define a handler for a
> >>>>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>>>         a handler?
> >>>>>>>>>>>>>>>>         > - what if we misspell a name?
> >>>>>>>>>>>>>>>>         > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > What Michael propose would have been totally OK
> >>>>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>         writing the API in Lua, Ruby or Python. In those
> >>>>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>>>         "dynamic naming" approach would have looked most
> >>>>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>         beautiful. But in Java we expect all the problems
> >>>>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>>>         identifiers to be eliminated in compile time.
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > And if we do, what advantage are we supposed to get
> >>>>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>>>         all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>>>         point?
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > ---
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > Earlier in this discussion John Roesler also
> >>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>         without "start branching" operator, and later Paul
> >>>>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>>>         the case when we have to add a dynamic number of
> >>>>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>>>         current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>>>         me address both comments here.
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > 1) "Start branching" operator (I think that
> >>>>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>>>         for it indeed) is critical when we need to do a
> >>>>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>>>         see example below.
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > 2) No, dynamic branching in current KIP is not
> >>>>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>>>         Imagine a real-world scenario when you need one
> >>>>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>>>         value (say, RecordType). You can have something
> >>>>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > /*John:if we had to start with stream.branch(...)
> >>>>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>>>         have been much messier.*/
> >>>>>>>>>>>>>>>>         > KBranchedStream branched = stream.split();
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > /*Not clumsy at all :-)*/
> >>>>>>>>>>>>>>>>         > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>>>>>>>         >             branched = branched.branch((k, v) ->
> >>>>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>>>         recordType,
> >>>>>>>>>>>>>>>>         >                     recordType::processRecords);
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > Regards,
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > Ivan
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>         > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>         >> I also agree with Michael's observation about
> >>>>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>>>         >> current `branch()` implementation.
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>>>         thinking
> >>>>>>>>>>>>>>>>         >> was more aligned with Paul's proposal to just
> >>>>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>>>         >> `branch()` statement and return a
> >>>>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >> It makes the code easier to read, and also make the
> >>>>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>>>         >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >>>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>         >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>         >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>         >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>>>         >> An open question is the case for which no
> >>>>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>         >> specified. Atm, `split()` and `branch()` would
> >>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>         `BranchedKStream`
> >>>>>>>>>>>>>>>>         >> and the call to `defaultBranch()` that returns the
> >>>>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>>>         >> (what is not the case atm). Or is this actually
> >>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>> real
> >>>>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>>>         >> because users can just ignore the branch
> >>>>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>>>         `defaultBranch()`
> >>>>>>>>>>>>>>>>         >> in the result `Map` ?
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >> About "inlining": So far, it seems to be a
> >>>>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>>>         >> preference. I can see arguments for both, but no
> >>>>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>>>         argument" yet
> >>>>>>>>>>>>>>>>         >> that clearly make the case for one or the other.
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >> -Matthias
> >>>>>>>>>>>>>>>>         >>
> >>>>>>>>>>>>>>>>         >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>         >>> Perhaps inlining is the wrong terminology. It
> >>>>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>         that a lambda with the full downstream topology be
> >>>>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>>>         it can be a method reference as with Ivan’s original
> >>>>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>>>         The advantage of putting the predicate and its
> >>>>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>         (Consumer) together in branch() is that they are
> >>>>>>>>>>>>>>>> required
> >>>>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>>>         to each other.
> >>>>>>>>>>>>>>>>         >>>
> >>>>>>>>>>>>>>>>         >>> Ultimately the downstream code has to live
> >>>>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>>>         branch trees will be hard to read regardless.
> >>>>>>>>>>>>>>>>         >>>
> >>>>>>>>>>>>>>>>         >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>>>         <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>         <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>         >>>>
> >>>>>>>>>>>>>>>>         >>>> I'm less enthusiastic about inlining the
> >>>>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>>>         downstream
> >>>>>>>>>>>>>>>>         >>>> functionality. Programs that have deep branch
> >>>>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>         quickly become
> >>>>>>>>>>>>>>>>         >>>> harder to read as a single unit.
> >>>>>>>>>>>>>>>>         >>>>
> >>>>>>>>>>>>>>>>         >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>>>         <pgwhalen@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Also +1 on the issues/goals as Michael
> >>>>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>>         that sets a
> >>>>>>>>>>>>>>>>         >>>>> great framework for the discussion.
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Regarding the SortedMap solution, my
> >>>>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>>>         current
> >>>>>>>>>>>>>>>>         >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>>>         decisions) is
> >>>>>>>>>>>>>>>>         >>>>> roughly this:
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> stream.split()
> >>>>>>>>>>>>>>>>         >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>         >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>>>         >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Obviously some ordering is necessary, since
> >>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>         construct
> >>>>>>>>>>>>>>>>         >>>>> doesn't work without it, but this solution seems
> >>>>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>>>         provides as much
> >>>>>>>>>>>>>>>>         >>>>> associativity as the SortedMap solution,
> >>>>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>         call
> >>>>>>>>>>>>>>>>         >>>>> directly associates the "conditional" with
> >>>>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>>>         The value it
> >>>>>>>>>>>>>>>>         >>>>> provides over the KIP solution is the
> >>>>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>>>         the same
> >>>>>>>>>>>>>>>>         >>>>> scope.
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> The KIP solution is less "dynamic" than the
> >>>>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>         in the sense
> >>>>>>>>>>>>>>>>         >>>>> that it is slightly clumsier to add a dynamic
> >>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>         branches, but it is
> >>>>>>>>>>>>>>>>         >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>>>         the "static"
> >>>>>>>>>>>>>>>>         >>>>> case anyway, and should make it simple and
> >>>>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>>>         fluently declare and
> >>>>>>>>>>>>>>>>         >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>>>         ignore a
> >>>>>>>>>>>>>>>>         >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>>>         SortedMap
> >>>>>>>>>>>>>>>>         >>>>> solution on top of it.
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> I could also see a middle ground where
> >>>>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>>>         SortedMap being
> >>>>>>>>>>>>>>>>         >>>>> taken in, branch() takes a name and not a
> >>>>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>>>         like this:
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>         >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>>>         >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>>>         >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Pros for that solution:
> >>>>>>>>>>>>>>>>         >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>>>>>>>         >>>>> - no double brace initialization, hopefully
> >>>>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>         readable than
> >>>>>>>>>>>>>>>>         >>>>> SortedMap
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Cons
> >>>>>>>>>>>>>>>>         >>>>> - downstream branch logic cannot be specified
> >>>>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>         makes it harder
> >>>>>>>>>>>>>>>>         >>>>> to read top to bottom (like existing API and
> >>>>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>>>         unlike the KIP)
> >>>>>>>>>>>>>>>>         >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>>>         existing
> >>>>>>>>>>>>>>>>         >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> (KBranchedStreams could even work *both* ways
> >>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>>>         that's overdoing
> >>>>>>>>>>>>>>>>         >>>>> it).
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Overall I'm curious how important it is to be
> >>>>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>>>         access the
> >>>>>>>>>>>>>>>>         >>>>> branched KStream in the same scope as the
> >>>>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>         possible that it
> >>>>>>>>>>>>>>>>         >>>>> doesn't need to be handled directly by the
> >>>>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>         left up to the
> >>>>>>>>>>>>>>>>         >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> Paul
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> >>>>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>>>         <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>         >>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>         >>>>>> I'd like to +1 what Michael said about the
> >>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>         existing
> >>>>>>>>>>>>>>>>         >>>>> branch
> >>>>>>>>>>>>>>>>         >>>>>> method, I agree with what he's outlined and
> >>>>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>>>         proceed by
> >>>>>>>>>>>>>>>>         >>>>>> trying to alleviate these problems.
> >>>>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>         important to be
> >>>>>>>>>>>>>>>>         >>>>>> able to cleanly access the individual
> >>>>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>>>         >>>>>> name->stream), which I thought was the original
> >>>>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>>>         this KIP.
> >>>>>>>>>>>>>>>>         >>>>>>
> >>>>>>>>>>>>>>>>         >>>>>> That said, I don't think we should so easily
> >>>>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>         double brace
> >>>>>>>>>>>>>>>>         >>>>>> anti-pattern or force ours users into it if
> >>>>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>>>         >>>>> avoid...just
> >>>>>>>>>>>>>>>>         >>>>>> my two cents.
> >>>>>>>>>>>>>>>>         >>>>>>
> >>>>>>>>>>>>>>>>         >>>>>> Cheers,
> >>>>>>>>>>>>>>>>         >>>>>> Sophie
> >>>>>>>>>>>>>>>>         >>>>>>
> >>>>>>>>>>>>>>>>         >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> >>>>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>>>         >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>>>         <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>> I’d like to propose a different way of
> >>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>>>         To me,
> >>>>>>>>>>>>>>>>         >>>>> there
> >>>>>>>>>>>>>>>>         >>>>>>> are three problems with the existing branch
> >>>>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>>>         >>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>>>> type
> >>>>>>>>>>>>>>>>         >>>>>> warnings.
> >>>>>>>>>>>>>>>>         >>>>>>> 2. The way in which you use the stream
> >>>>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>>>         positionally coupled
> >>>>>>>>>>>>>>>>         >>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>>>>>>>         >>>>>>> 3. It is brittle to extend existing branch
> >>>>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>         additional code
> >>>>>>>>>>>>>>>>         >>>>>>> paths.
> >>>>>>>>>>>>>>>>         >>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>> Using associative constructs instead of
> >>>>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>>>         constructs
> >>>>>>>>>>>>>>>>         >>>>>> would
> >>>>>>>>>>>>>>>>         >>>>>>> be a stronger approach. Consider a
> >>>>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>>>         looks like
> >>>>>>>>>>>>>>>>         >>>>>> this:
> >>>>>>>>>>>>>>>>         >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>>>         Predicate<?
> >>>>>>>>>>>>>>>>         >>>>>>> super K,? super V>>);
> >>>>>>>>>>>>>>>>         >>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>> Branches are given names in a map, and as a
> >>>>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>>>         returns a
> >>>>>>>>>>>>>>>>         >>>>>>> mapping of names to streams. The ordering
> >>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>>>         >>>>>> maintained
> >>>>>>>>>>>>>>>>         >>>>>>> because it’s a sorted map. Insert order
> >>>>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>>>> the order
> >>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>         >>>>>> evaluation.
> >>>>>>>>>>>>>>>>         >>>>>>> This solves problem 1 because there are no
> >>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>>>         solves
> >>>>>>>>>>>>>>>>         >>>>>> problem
> >>>>>>>>>>>>>>>>         >>>>>>> 2 because you no longer lean on ordering to
> >>>>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>>>         branch you’re
> >>>>>>>>>>>>>>>>         >>>>>>> interested in. It solves problem 3 because
> >>>>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>         another
> >>>>>>>>>>>>>>>>         >>>>>>> conditional by simply attaching another
> >>>>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>>>         structure, rather
> >>>>>>>>>>>>>>>>         >>>>>> than
> >>>>>>>>>>>>>>>>         >>>>>>> messing with the existing indices.
> >>>>>>>>>>>>>>>>         >>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>>>         historically
> >>>>>>>>>>>>>>>>         >>>>>>> awkward in Java. I know it’s an
> >>>>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>>>         voluminously, but
> >>>>>>>>>>>>>>>>         >>>>>>> double brace initialization would clean up the
> >>>>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>>>         >>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>>>         <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>>>         >>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>> Thanks for the update.
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>> FWIW, I agree with Matthias that the current
> >>>>>>>>>>>>>>>> "start
> >>>>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>>>         >>>>> operator
> >>>>>>>>>>>>>>>>         >>>>>>> is
> >>>>>>>>>>>>>>>>         >>>>>>>> confusing when named the same way as the
> >>>>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>>>         "Split"
> >>>>>>>>>>>>>>>>         >>>>> seems
> >>>>>>>>>>>>>>>>         >>>>>>>> like a good name. Alternatively, we can do
> >>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>>>         >>>>> branching"
> >>>>>>>>>>>>>>>>         >>>>>>>> operator at all, and just do:
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>> stream
> >>>>>>>>>>>>>>>>         >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>         >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>         >>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>> Tentatively, I think that this branching
> >>>>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>         >>>>> terminal.
> >>>>>>>>>>>>>>>>         >>>>>>> That
> >>>>>>>>>>>>>>>>         >>>>>>>> way, we don't create ambiguity about how
> >>>>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>>>         is, `branch`
> >>>>>>>>>>>>>>>>         >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>>>         `void`, to
> >>>>>>>>>>>>>>>>         >>>>>>>> enforce that it comes last, and that there
> >>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>>>         definition of
> >>>>>>>>>>>>>>>>         >>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>>>         there's no
> >>>>>>>>>>>>>>>>         >>>>>>> default,
> >>>>>>>>>>>>>>>>         >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>>>         record
> >>>>>>>>>>>>>>>>         >>>>>> falls
> >>>>>>>>>>>>>>>>         >>>>>>>> though with no default.
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>         >>>>>>>> -John
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>>>         >>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>>>         >>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> Thanks for updating the KIP and your
> >>>>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> this is to make the name similar to
> >>>>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>>>>>>>         >>>>>>>>> The intend was to avoid name duplication.
> >>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>>>         should
> >>>>>>>>>>>>>>>>         >>>>>> _not_
> >>>>>>>>>>>>>>>>         >>>>>>>>> be an array.
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> The current proposal is
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>         >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> IMHO, this reads a little odd, because
> >>>>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>>>         `branch()` does
> >>>>>>>>>>>>>>>>         >>>>> not
> >>>>>>>>>>>>>>>>         >>>>>>>>> take any parameters and has different
> >>>>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>>>> than the
> >>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>         >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>         >>>>> hidden
> >>>>>>>>>>>>>>>>         >>>>>>>>> that the first call is `KStream#branch()`
> >>>>>>>>>>>>>>>> while
> >>>>>>>>>>>>>>>> the others
> >>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>         >>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>>>> code
> >>>>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> Because I suggested to rename
> >>>>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>>>         I though
> >>>>>>>>>>>>>>>>         >>>>>> it
> >>>>>>>>>>>>>>>>         >>>>>>>>> might be better to also rename
> >>>>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>> naming
> >>>>>>>>>>>>>>>>         >>>>>>>>> overlap that seems to be confusing. The
> >>>>>>>>>>>>>>>> following
> >>>>>>>>>>>>>>>> reads
> >>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>         >>>>> cleaner
> >>>>>>>>>>>>>>>>         >>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>> me:
> >>>>>>>>>>>>>>>>         >>>>>>>>> stream.split()
> >>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>>>         >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> Maybe there is a better alternative to
> >>>>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>>>         avoid
> >>>>>>>>>>>>>>>>         >>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> naming overlap.
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>         >>>>> cannot
> >>>>>>>>>>>>>>>>         >>>>>>> have
> >>>>>>>>>>>>>>>>         >>>>>>>>> a method with such name :-)
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> Bummer. Didn't consider this. Maybe we
> >>>>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>>>         with a
> >>>>>>>>>>>>>>>>         >>>>> short
> >>>>>>>>>>>>>>>>         >>>>>>>> name?
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> Can you add the interface
> >>>>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>>         with all
> >>>>>>>>>>>>>>>>         >>>>> it's
> >>>>>>>>>>>>>>>>         >>>>>>>>> methods? It will be part of public API and
> >>>>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>>>         contained in
> >>>>>>>>>>>>>>>>         >>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>>>         >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>>>>>>>         `KBranchedStream#get(int
> >>>>>>>>>>>>>>>>         >>>>>>> index)
> >>>>>>>>>>>>>>>>         >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>>>>>>>         branched-KStreams. Would
> >>>>>>>>>>>>>>>>         >>>>>> be
> >>>>>>>>>>>>>>>>         >>>>>>>>> nice to get your feedback about it. It
> >>>>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>>>         that users
> >>>>>>>>>>>>>>>>         >>>>>>>>> would need to write custom utility code
> >>>>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>>>         access them.
> >>>>>>>>>>>>>>>>         >>>>> We
> >>>>>>>>>>>>>>>>         >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>>>         >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>>>         to get
> >>>>>>>>>>>>>>>>         >>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> I have updated the KIP-418 according to
> >>>>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>         >>>>>>>>>> I can see your point: this is to make
> >>>>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>>>         >>>>>>> String#split
> >>>>>>>>>>>>>>>>         >>>>>>>>>> that also returns an array, right? But
> >>>>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>>>         loss of
> >>>>>>>>>>>>>>>>         >>>>>>>> backwards
> >>>>>>>>>>>>>>>>         >>>>>>>>>> compatibility? We can have overloaded
> >>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>>>         without
> >>>>>>>>>>>>>>>>         >>>>>>>> affecting
> >>>>>>>>>>>>>>>>         >>>>>>>>>> the existing code. Maybe the old
> >>>>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>         >>>>> should
> >>>>>>>>>>>>>>>>         >>>>>>> be
> >>>>>>>>>>>>>>>>         >>>>>>>>>> deprecated, but this is a subject for
> >>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>>         >>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>>>         >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Totally agree with 'addBranch->branch'
> >>>>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>         >>>>>>> however, a
> >>>>>>>>>>>>>>>>         >>>>>>>>>> reserved word, so unfortunately we
> >>>>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>>>         with such
> >>>>>>>>>>>>>>>>         >>>>>>> name
> >>>>>>>>>>>>>>>>         >>>>>>>>> :-)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> defaultBranch() does take an
> >>>>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>         >>>>> think
> >>>>>>>>>>>>>>>>         >>>>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>>>>> is not required?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Absolutely! I think that was just
> >>>>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>>>         something.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> please revise the new version of the KIP
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>>>         >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> Thanks for driving the discussion of
> >>>>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>> everybody
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> agrees that the current branch() method
> >>>>>>>>>>>>>>>> using
> >>>>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>         >>>>>> optimal.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> I had a quick look into the PR and I
> >>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>         proposal.
> >>>>>>>>>>>>>>>>         >>>>>>> There
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> are some minor things we need to
> >>>>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>         recommend the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> following renaming:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>>>         BranchingKStream#default()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> It's just a suggestion to get slightly
> >>>>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> In the current PR, defaultBranch() does
> >>>>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>>>         `Predicate` as
> >>>>>>>>>>>>>>>>         >>>>>>>> argument,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>>>         accepted and
> >>>>>>>>>>>>>>>>         >>>>>> is
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> Ie, we should add overloads that
> >>>>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>>>         parameter.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> For the issue that the created
> >>>>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>>>         different
> >>>>>>>>>>>>>>>>         >>>>>>>> scopes:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> could we extend `KBranchedStream` with a
> >>>>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>>>         index)` method
> >>>>>>>>>>>>>>>>         >>>>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> returns the corresponding "branched"
> >>>>>>>>>>>>>>>> result
> >>>>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>>>         object?
> >>>>>>>>>>>>>>>>         >>>>>> Maybe,
> >>>>>>>>>>>>>>>>         >>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> second argument of `addBranch()` should
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>>>         >>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>>>         >>>>>>>> but
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> a `Function<KStream,KStream>` and
> >>>>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>         whatever
> >>>>>>>>>>>>>>>>         >>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> Finally, I would also suggest to update
> >>>>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>         current
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>         makes sense
> >>>>>>>>>>>>>>>>         >>>>>> for
> >>>>>>>>>>>>>>>>         >>>>>>>> you
> >>>>>>>>>>>>>>>>         >>>>>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> revise the KIP and continue the
> >>>>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>>>         we'll
> >>>>>>>>>>>>>>>>         >>>>> need
> >>>>>>>>>>>>>>>>         >>>>>>>> some
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>>>         >>>>> whether
> >>>>>>>>>>>>>>>>         >>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> KIP
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> could be adopted.  It would be great
> >>>>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>>>         think this
> >>>>>>>>>>>>>>>>         >>>>>> is
> >>>>>>>>>>>>>>>>         >>>>>>> a
> >>>>>>>>>>>>>>>>         >>>>>>>>> good
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> idea overall.  I'm not sure if that
> >>>>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>>>         starting a
> >>>>>>>>>>>>>>>>         >>>>>>> vote,
> >>>>>>>>>>>>>>>>         >>>>>>>>> or if
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> there is generally some indication of
> >>>>>>>>>>>>>>>> interest
> >>>>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> That being said, I'll continue the
> >>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>>>         assuming
> >>>>>>>>>>>>>>>>         >>>>> we
> >>>>>>>>>>>>>>>>         >>>>>> do
> >>>>>>>>>>>>>>>>         >>>>>>>>> move
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> forward the solution of "stream.branch()
> >>>>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>>>         >>>>>> KBranchedStream",
> >>>>>>>>>>>>>>>>         >>>>>>> do
> >>>>>>>>>>>>>>>>         >>>>>>>>> we
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>         >>>>> favor
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>> accomplish
> >>>>>>>>>>>>>>>>         >>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>>>         similar
> >>>>>>>>>>>>>>>>         >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> just need to be sure we're not making
> >>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>         >>>>>>> impossible/difficult
> >>>>>>>>>>>>>>>>         >>>>>>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>>         >>>>> it's
> >>>>>>>>>>>>>>>>         >>>>>>>> just a
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> little sloppy overall in terms of
> >>>>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>>>         >>>>>>> particular,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> passing in the "predicates" and
> >>>>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>         >>>>>> modified
> >>>>>>>>>>>>>>>>         >>>>>>>> in
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>>>>>>>         KStreamLazyBranch is
> >>>>>>>>>>>>>>>>         >>>>> a
> >>>>>>>>>>>>>>>>         >>>>>>> bit
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>         >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>         >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> I read your code carefully and now I
> >>>>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>>>         convinced: your
> >>>>>>>>>>>>>>>>         >>>>>>>> proposal
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> looks better and should work. We just
> >>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>> crucial
> >>>>>>>>>>>>>>>>         >>>>>>>>> fact
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> that KStream consumers are invoked as
> >>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>>>         And then
> >>>>>>>>>>>>>>>>         >>>>>> it's
> >>>>>>>>>>>>>>>>         >>>>>>>> all
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> What shall we do now? I should
> >>>>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>>>         resume the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Why are you telling that your PR
> >>>>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>>>         >>>>> starting
> >>>>>>>>>>>>>>>>         >>>>>>>> point
> >>>>>>>>>>>>>>>>         >>>>>>>>> if
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> we go in this direction'? To me it
> >>>>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>>>         starting
> >>>>>>>>>>>>>>>>         >>>>>>> point.
> >>>>>>>>>>>>>>>>         >>>>>>>>> But
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> as a novice in this project I might
> >>>>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>>>         >>>>> details.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> >>>>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>>>         >>>>> stream.branch()
> >>>>>>>>>>>>>>>>         >>>>>>>>> solution
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>>>         >>>>> invoked
> >>>>>>>>>>>>>>>>         >>>>>> as
> >>>>>>>>>>>>>>>>         >>>>>>>>> they’re
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> added, not during
> >>>>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>>>         still
> >>>>>>>>>>>>>>>>         >>>>>> ought
> >>>>>>>>>>>>>>>>         >>>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>> be
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> able to call couponIssuer.coupons()
> >>>>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>         depend on
> >>>>>>>>>>>>>>>>         >>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> branched
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> The issue I mean to point out is
> >>>>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>>>         access
> >>>>>>>>>>>>>>>>         >>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> branched
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> streams in the same scope as the
> >>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>>>         is, not
> >>>>>>>>>>>>>>>>         >>>>>>>> inside
> >>>>>>>>>>>>>>>>         >>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>>>         >>>>> solutions.
> >>>>>>>>>>>>>>>>         >>>>>> It
> >>>>>>>>>>>>>>>>         >>>>>>>>> can be
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> [Also, great to hear additional
> >>>>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>>>         excited
> >>>>>>>>>>>>>>>>         >>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>> hear
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>         >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>>>         >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> The idea to postpone the wiring of
> >>>>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>>>> me at
> >>>>>>>>>>>>>>> first
> >>>>>>>>>>>>>>>>         >>>>> glance,
> >>>>>>>>>>>>>>>>         >>>>>>> but
> >>>>>>>>>>>>>>>>         >>>>>>>>> ---
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>>>         same
> >>>>>>>>>>>>>>>>         >>>>>> scope
> >>>>>>>>>>>>>>>>         >>>>>>> as
> >>>>>>>>>>>>>>>>         >>>>>>>>> each
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> other.  That is, if we wanted to merge
> >>>>>>>>>>>>>>>> them back
> >>>>>>>>>>>>>>> together
> >>>>>>>>>>>>>>>>         >>>>> again
> >>>>>>>>>>>>>>>>         >>>>>> I
> >>>>>>>>>>>>>>>>         >>>>>>>>> don't see
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> You just took the words right out
> >>>>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>>>         just
> >>>>>>>>>>>>>>>>         >>>>>> going
> >>>>>>>>>>>>>>>>         >>>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Consider the example from Bill's
> >>>>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>>>         we need
> >>>>>>>>>>>>>>>>         >>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>> identify
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> customers who have bought coffee and
> >>>>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>>>         in the
> >>>>>>>>>>>>>>>>         >>>>>>>>> electronics
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> This is the code I usually write under
> >>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>         circumstances
> >>>>>>>>>>>>>>>>         >>>>>> using
> >>>>>>>>>>>>>>>>         >>>>>>>> my
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>       return
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>       /*In the real world the code
> >>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>         complex, so
> >>>>>>>>>>>>>>>>         >>>>>>>>> creation of
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>>>         order to
> >>>>>>>>>>>>>>>>         >>>>>>>> separate
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>>>>>>>         >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> >>>>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>>>         everything
> >>>>>>>>>>>>>>>>         >>>>>>>> later,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Does this make sense?  In order to
> >>>>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>>>         initialize the
> >>>>>>>>>>>>>>>>         >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> we need the terminal operation to be
> >>>>>>>>>>>>>>>> called
> >>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>         >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>>>         essentially
> >>>>>>>>>>>>>>>>         >>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> next
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> KIP I was going to write here. I have
> >>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>>>         based on
> >>>>>>>>>>>>>>>>         >>>>> my
> >>>>>>>>>>>>>>>>         >>>>>>>>> experience,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> so I will join the discussion on KIP-401
> >>>>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>>>         fluent
> >>>>>>>>>>>>>>>>         >>>>> API
> >>>>>>>>>>>>>>>>         >>>>>>>> based
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>>>>>>>         (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>>>>>>>         >>>>>> and
> >>>>>>>>>>>>>>>>         >>>>>>> I
> >>>>>>>>>>>>>>>>         >>>>>>>>> think
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>>>         >>>>>>> compatibility
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    there aren't any direct ones.
> >>>>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>>>         that Java
> >>>>>>>>>>>>>>>>         >>>>> is
> >>>>>>>>>>>>>>>>         >>>>>>>> smart
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    distinguish between a
> >>>>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>>>         returning one
> >>>>>>>>>>>>>>>>         >>>>>>> thing
> >>>>>>>>>>>>>>>>         >>>>>>>>> and
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    with no arguments returning
> >>>>>>>>>>>>>>>> another
> >>>>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> >>>>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>         need
> >>>>>>>>>>>>>>>>         >>>>> it.
> >>>>>>>>>>>>>>>>         >>>>>>> We
> >>>>>>>>>>>>>>>>         >>>>>>>>> can
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>>>         shares
> >>>>>>>>>>>>>>>>         >>>>>> its
> >>>>>>>>>>>>>>>>         >>>>>>>>> state
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> >>>>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         branching.
> >>>>>>>>>>>>>>>>         >>>>>>> It's
> >>>>>>>>>>>>>>>>         >>>>>>>>> not
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    pretty in its current form, but I
> >>>>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>>>         demonstrates
> >>>>>>>>>>>>>>>>         >>>>>> its
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>>>>>>>> request should
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>         >>>>> final
> >>>>>>>>>>>>>>>>         >>>>>> or
> >>>>>>>>>>>>>>>>         >>>>>>>>> even a
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> starting point if we go in this
> >>>>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>>>         wanted to
> >>>>>>>>>>>>>>>>         >>>>>> see
> >>>>>>>>>>>>>>>>         >>>>>>>> how
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>>>> API
> >>>>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> I will say though, that I'm not
> >>>>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>         solution
> >>>>>>>>>>>>>>>>         >>>>>>> could
> >>>>>>>>>>>>>>>>         >>>>>>>> be
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> deprecated in favor of this, which
> >>>>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>>>         >>>>> suggested
> >>>>>>>>>>>>>>>>         >>>>>>>> was a
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>         streams
> >>>>>>>>>>>>>>>>         >>>>>> are
> >>>>>>>>>>>>>>>>         >>>>>>>> not
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>>>         is, if we
> >>>>>>>>>>>>>>>>         >>>>>>> wanted
> >>>>>>>>>>>>>>>>         >>>>>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> them back together again I don't
> >>>>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>>>         that.  The
> >>>>>>>>>>>>>>>>         >>>>>> KIP
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>         >>>>>> either
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>>>         table.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>         >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>         >>>>>> point.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> First, it seems that it's
> >>>>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>         branch API
> >>>>>>>>>>>>>>>>         >>>>>>> needs
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> There are two potential ways to
> >>>>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>>>         returns
> >>>>>>>>>>>>>>>>         >>>>>> its
> >>>>>>>>>>>>>>>>         >>>>>>>>> argument
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> >>>>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>>>> The code
> >>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>         >>>>> make
> >>>>>>>>>>>>>>>>         >>>>>>>> sense
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> all the necessary ingredients are
> >>>>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>         instance
> >>>>>>>>>>>>>>>>         >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>>>         >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> >>>>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>         >>>>>> defined.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> >>>>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>>>         >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>>>         >>>>>>>>> and
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>>>         miss the
> >>>>>>>>>>>>>>>>         >>>>>> fact
> >>>>>>>>>>>>>>>>         >>>>>>>>> that one
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> of the terminal methods should be
> >>>>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>>>         methods
> >>>>>>>>>>>>>>>>         >>>>>> are
> >>>>>>>>>>>>>>>>         >>>>>>>> not
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>>>> we do
> >>>>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> I see your point when you are
> >>>>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> >>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>>         implemented the
> >>>>>>>>>>>>>>>>         >>>>>>> easy
> >>>>>>>>>>>>>>>>         >>>>>>>>> way.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> >>>>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> >>>>>>>>>>>>>>>> method that
> >>>>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>>>         >>>>> nothing
> >>>>>>>>>>>>>>>>         >>>>>>>> will
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> >>>>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>>> the only
> >>>>>>>>>>>>>>> option
> >>>>>>>>>>>>>>>>         >>>>>> besides
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>>>         want to
> >>>>>>>>>>>>>>>>         >>>>>> just
> >>>>>>>>>>>>>>>>         >>>>>>>>> silently
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> >>>>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>>>         predicate. 2)
> >>>>>>>>>>>>>>>>         >>>>>>> Throwing
> >>>>>>>>>>>>>>>>         >>>>>>>>> an
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>         looks
> >>>>>>>>>>>>>>>>         >>>>>> like a
> >>>>>>>>>>>>>>>>         >>>>>>>> bad
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> >>>>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>>>         emit a
> >>>>>>>>>>>>>>>>         >>>>>>>> special
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> >>>>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>         >>>>>>> `default`
> >>>>>>>>>>>>>>>>         >>>>>>>>> can
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>         >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>         >>>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>> track
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>         a clear
> >>>>>>>>>>>>>>>>         >>>>>>> error
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> >>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>>>         >>>>> compiled
> >>>>>>>>>>>>>>>>         >>>>>>> and
> >>>>>>>>>>>>>>>>         >>>>>>>>> run?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> >>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>>>         compile if
> >>>>>>>>>>>>>>>>         >>>>> used
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>>>         method chain
> >>>>>>>>>>>>>>>>         >>>>>>>> starting
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> >>>>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>>>         between
> >>>>>>>>>>>>>>>>         >>>>>>>> runtime
> >>>>>>>>>>>>>>>>         >>>>>>>>> and
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>>>         >>>>> instantly
> >>>>>>>>>>>>>>>>         >>>>>> on
> >>>>>>>>>>>>>>>>         >>>>>>>>> unit
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> >>>>>>>>>>>>>>>> project
> >>>>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>>>         compilation
> >>>>>>>>>>>>>>>>         >>>>>>>> failure.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>>>         required.
> >>>>>>>>>>>>>>>>         >>>>>>> But
> >>>>>>>>>>>>>>>>         >>>>>>>> is
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> >>>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>>>         >>>>>> defaultBranch
> >>>>>>>>>>>>>>>>         >>>>>>>>> they
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>>>         just as
> >>>>>>>>>>>>>>>>         >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> think it creates an
> >>>>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>>>         >>>>> user
> >>>>>>>>>>>>>>>>         >>>>>>>> could
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>> default
> >>>>>>>>>>>>>>>>         >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>         >>>>> seems
> >>>>>>>>>>>>>>>>         >>>>>>> like
> >>>>>>>>>>>>>>>>         >>>>>>>>> an
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>         which allows
> >>>>>>>>>>>>>>>>         >>>>>> for
> >>>>>>>>>>>>>>>>         >>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>>>         to be
> >>>>>>>>>>>>>>>>         >>>>>> well
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>>>         >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>>>         >>>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>> track
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>>>         a clear
> >>>>>>>>>>>>>>>>         >>>>>>> error
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>>>> there is
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>         >>>>> "build
> >>>>>>>>>>>>>>>>         >>>>>>>> step"
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>         >>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>>>         >>>>>>> is
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>>>         >>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>> it's
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> allow users to do other
> >>>>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>>>         stream.
> >>>>>>>>>>>>>>>>         >>>>>>> With
> >>>>>>>>>>>>>>>>         >>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>> way all
> >>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>         >>>>>>> operations
> >>>>>>>>>>>>>>>>         >>>>>>>>> do -
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> want to process off the original
> >>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>>>         >>>>> times,
> >>>>>>>>>>>>>>>>         >>>>>>> you
> >>>>>>>>>>>>>>>>         >>>>>>>>> just
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> >>>>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>>>         operations
> >>>>>>>>>>>>>>>>         >>>>>> on
> >>>>>>>>>>>>>>>>         >>>>>>> it
> >>>>>>>>>>>>>>>>         >>>>>>>>> as
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>>>         >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> >>>>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>         always need
> >>>>>>>>>>>>>>>>         >>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>>>         >>>>> don't
> >>>>>>>>>>>>>>>>         >>>>>>>> know
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> >>>>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>>>         so we
> >>>>>>>>>>>>>>>>         >>>>> can
> >>>>>>>>>>>>>>>>         >>>>>> do
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> more with the original branch
> >>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> >>>>>>>>>>>>>>>> need of
> >>>>>>>>>>>>>>> special
> >>>>>>>>>>>>>>>>         >>>>> object
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>         >>>>> here
> >>>>>>>>>>>>>>>>         >>>>>> we
> >>>>>>>>>>>>>>>>         >>>>>>>>> have a
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>>>         flow,
> >>>>>>>>>>>>>>>>         >>>>> so
> >>>>>>>>>>>>>>>>         >>>>>> I
> >>>>>>>>>>>>>>>>         >>>>>>>>> think
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> >>>>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>         API, but I
> >>>>>>>>>>>>>>>>         >>>>>> find
> >>>>>>>>>>>>>>>>         >>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>>>         contrasts the
> >>>>>>>>>>>>>>>>         >>>>>>> fluency
> >>>>>>>>>>>>>>>>         >>>>>>>>> of
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
> >>>>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>>>         just call
> >>>>>>>>>>>>>>>>         >>>>> a
> >>>>>>>>>>>>>>>>         >>>>>>>>> method on
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> >>>>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>>>         cases
> >>>>>>>>>>>>>>>>         >>>>> are
> >>>>>>>>>>>>>>>>         >>>>>>>>> defined
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> I think the
> >>>>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>>>         is very
> >>>>>>>>>>>>>>>>         >>>>>> nice
> >>>>>>>>>>>>>>>>         >>>>>>>>> and the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> >>>>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>         how we
> >>>>>>>>>>>>>>>>         >>>>>>> specify
> >>>>>>>>>>>>>>>>         >>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>>>         >>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>>>         >>>>>>>>> or
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>>>         terminated by
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>>>         incompatible with
> >>>>>>>>>>>>>>>>         >>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> current
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> >>>>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>         different
> >>>>>>>>>>>>>>>>         >>>>>> name,
> >>>>>>>>>>>>>>>>         >>>>>>>> but
> >>>>>>>>>>>>>>>>         >>>>>>>>> that
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> >>>>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>>>         >>>>>> something
> >>>>>>>>>>>>>>>>         >>>>>>>> like
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> >>>>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>         KIP?  It
> >>>>>>>>>>>>>>>>         >>>>>> seems
> >>>>>>>>>>>>>>>>         >>>>>>>>> like it
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>>>         while also
> >>>>>>>>>>>>>>>>         >>>>>>>> allowing
> >>>>>>>>>>>>>>>>         >>>>>>>>> you
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>>>         KBranchedStreams
> >>>>>>>>>>>>>>>>         >>>>>> if
> >>>>>>>>>>>>>>>>         >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>>>         ks){
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>>>         String> ks){
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>>>>>>>         this::handleFirstCase)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>>>>>>>         this::handleSecondCase)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>>>         >>>>> takes a
> >>>>>>>>>>>>>>>>         >>>>>>>>> Consumer
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>>>         example in
> >>>>>>>>>>>>>>>>         >>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>> KIP
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>>>         >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>>>> how would
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>         >>>>> handle
> >>>>>>>>>>>>>>>>         >>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>> case
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>         >>>>>>>> processing
> >>>>>>>>>>>>>>>>         >>>>>>>>> and
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>>>         >>>>>> stream
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>>>         we had
> >>>>>>>>>>>>>>>>         >>>>>>>> something
> >>>>>>>>>>>>>>>>         >>>>>>>>> like
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>         >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>>>         >>>>> 418.
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>>>         >>>>> Please
> >>>>>>>>>>>>>>>>         >>>>>>>> take
> >>>>>>>>>>>>>>>>         >>>>>>>>> a
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>>>         feedback :)
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>>>         >>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>>>         >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         >>>>>>>>>
> >>>>>>>>>>>>>>>>         >
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>>
> >>
> >>
> >> Attachments:
> >> * signature.asc
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi,

Thanks Matthias for your suggestion: yes, I agree that getting rid of 
`with[Java]Consumer` makes this thing 'as simple as possible, but not 
simpler'.

I made some quick API mocking in my IDE and tried to implement examples 
from KIP.

1. Having to return something from lambda is not a very big deal.

2. For a moment I thouht that I won't be able to use method references 
for already written stream consumers, but then I realized that I can 
just change my methods from returning void to returning the input 
parameter and use references to them. Not very convenient, but passable.

So, I'm ready to agree: 1) we use only functions, no consumer 2) when 
function returns null, we don't insert it into the resulting map.

Usually it's better to implement a non-perfect, but workable solution as 
a first approximation. And later we can always add to `Branched` 
anything we want.

3. Do we have any guidelines on how parameter classes like Branched 
should be built? First of all, it seems that `as` now is more preferred 
than `withName` (although as you probably know it clashes with Kotlin's 
`as` operator).

Then, while trying to mock the APIs, I found out that my Java cannot 
infer types in the following construction:

.branch((key, value) -> value == null,
    Branched.as("foo").withChain(s -> s.mapValues(...)))


so I have to write

.branch((key, value) -> value == null,
    Branched.<String, String>as("foo").withChain(s -> s.mapValues(...)))


This is not tolerable IMO, so this is the list of `Branched` methods 
that I came to (will you please validate it):

static <K, V> Branched<K, V> as(String name);

static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ? 
extends KStream<K, V>> chain);

static <K, V> Branched<K, V> with(Function<? super KStream<K, V>, ? 
extends KStream<K, V>> chain, String name);

//non-static!
Branched<K, V> withChain(Function<? super KStream<K, V>, ? extends 
KStream<K, V>> chain);


4. And one more. What do you think, do we need that flexibility:

Function<? super KStream<K, V>, ? extends KStream<K, V>> chain

vs.

Function<? super KStream<? super K, ? super V>, ? extends KStream<? 
extends K, ? extends V>> chain

??

Regards,

Ivan


21.05.2020 6:54, John Roesler пишет:
> Thanks for this thought, Matthias,
> 
> Your idea has a few aspects I find attractive:
> 1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
> 2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
> 3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration.
> 
> The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me.
> 
> Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.
> 
> Thanks again for sharing the idea,
> John
> 
> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
>> Thanks for updating the KIP!
>>
>> I guess the only open question is about `Branched.withJavaConsumer` and
>> its relationship to the returned `Map`.
>>
>> Originally, we discussed two main patterns:
>>
>>   (1) split a stream and return the substreams for futher processing
>>   (2) split a stream and modify the substreams with in-place method chaining
>>
>> To combine both patterns we wanted to allow for
>>
>>    -> split a stream, modify the substreams, and return the _modified_
>> substreams for further processing
>>
>>> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
>>
>> That is of course possible. However, it introduces some "hidded" semantics:
>>
>>   - using `withChain` I get the modified sub-stream
>>   - using `withJavaConsumer` I get the unmodifed sub-stream
>>
>> This seems to be quite subtle to me.
>>
>>
>>
>>  From my understanding the original idea of `withJavaConsumer` was to
>> model a terminal operation, ie, it should be similar to:
>>
>> Branched.withChain(s -> {
>>    s.to();
>>    return null;
>> })
>>
>> However, I am not sure if we should even allow `withChain()` to return
>> `null`? IMHO, we should throw an exception for this case to avoid a `key
>> -> null` entry in the returned Map.
>>
>> Following this train of through, and if we want to allow the "return
>> null" pattern in general, we need `withJavaConsumer` that does not add
>> an entry to the Map.
>>
>> Following your proposal, the semantics of `withJavaConsumer` could also
>> be achieved with `withChain`:
>>
>> Branched.withChain(s -> {
>>    s.to();
>>    return s;
>> })
>>
>> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
>> while for the first proposal it adds new functionality (if `return null`
>> is not allowed, using `withChain()` is not possible to "hide a
>> sub-stream in the result). Furthermore, we might need to allow `return
>> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
>>
>>
>>
>> I guess I can be convinced either way. However, if we follow your
>> proposal, I am wondering if we need `withJavaConsumer` at all? Its
>> benefit seems to be small? Also, having a reduced API is usually
>> preferable as it's simpler to learn.
>>
>>
>>
>> -Matthias
>>
>>
>>
>>
>> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
>>> Hello, John, hello Matthias!
>>>
>>> Thank you very much for your detailed feedback!
>>>
>>> -----------------------------------------
>>>
>>> John,
>>>
>>>> It looks like you missed my reply on Apr 23rd.
>>>
>>> For some unknown reason it didn't reach my inbox, fortunately we have
>>> all the emails on the web.
>>>
>>>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
>>> method?
>>>
>>> Done, in "Compatibility, Deprecation, and Migration Plan" section.
>>>
>>>> 2. [Explain why 'branch' operator is superior to branching directly
>>> off of the parent KStream for the needs of dynamic branching]
>>>
>>> Done, see an ugly counterexample in 'Dynamic Branching' section.
>>>
>>>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
>>>
>>> As Mathhias noted, `withSink` can also be confusing. I renamed this
>>> method to `withJavaConsumer` per Matthias' suggestion.
>>>
>>>> 4. ...It seems like there are two disjoint use cases: EITHER using
>>> chain and the result map OR using just the sink
>>>
>>> This is discussed below.
>>>
>>> ----------------------------------------------
>>>
>>> Mathhias,
>>>
>>>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
>>>
>>> Done.
>>>
>>>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
>>> method]
>>>
>>> Fixed.
>>>
>>>
>>>> 3. Overview of newly added methods/interfaces
>>>
>>> Done in `Proposed Changes` section.
>>>
>>>
>>>> 4. [Concerning John's note] > I don't think that using both
>>> `withChain()` and `withConsumer()` is the
>>> issue, as the KIP clearly states that the result of `withChain()` will
>>> be given to the `Consumer`.
>>>
>>> Yes, I agree!
>>>
>>>> The issue is really with the `Consumer` and the returned `Map` of
>>> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
>>> implementation would be to not add the "branch" to the result map if
>>> `withConsumer` is used?
>>>
>>> But is it also an issue? With Kafka Streams, we can split the topology
>>> graph at any point. Technically, it's OK to do both: feed the KStream to
>>> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
>>> stream in the Map, one simply does not extract it from there :-)
>>>
>>> In the current version of KIP it is assumed that the returned map
>>> contains ALL the branches, either tagged with IDs explicitly set by the
>>> programmer, or with some default auto-generated ids. Dealing with this
>>> map is the user's responsibility.
>>>
>>> What seems to me to be an issue is introducing exclusions to this
>>> general rule, like 'swallowing' some streams by provided
>>> [Java]Consumers. This can make things complicated. What if a user
>>> provides both the name of the branch and a [Java]Consumer? What do they
>>> mean in this case? Should we 'swallow' the stream or save it to the map?
>>> There's no point in 'saving the space' in this map, so maybe just leave
>>> it as it is?
>>>
>>> ----
>>>
>>> I rewrote the KIP and also fixed a couple of typos.
>>>
>>> Looking forward for your feedback again!
>>>
>>> Regards,
>>>
>>> Ivan.
>>>
>>>
>>>
>>> 08.05.2020 22:55, Matthias J. Sax пишет:
>>>> Thanks for updating the KIP!
>>>>
>>>> I also have some minor comment:
>>>>
>>>>
>>>>
>>>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>>>
>>>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>>>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>>>> and `KGroupedKTable` that we cannot rename without a breaking change...
>>>> so we just keep them.)
>>>>
>>>>
>>>>
>>>> (2) Quote:
>>>>
>>>>> Both branch and defaultBranch operations also have overloaded
>>>>> parameterless alternatives.
>>>>
>>>> I think `branch()` always needs to take a `Predicate` and assume you
>>>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>>>> as `branch()` would not be "parameterless".
>>>>
>>>>
>>>>
>>>> (3) Can you maybe add an overview in the "Public Interface" section) of
>>>> newly added and deprecated methods/classes (cf.
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>>>
>>>>
>>>>
>>>>
>>>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>>>> and the finally returned `Map<String, KStream>`. This related to John's
>>>> 4th comment:
>>>>
>>>>> It seems like there are really two disjoint use cases: EITHER using
>>>>> chain and the result map OR using just the sink.
>>>>
>>>> I don't think that using both `withChain()` and `withConsumer()` is the
>>>> issue though, as the KIP clearly states that the result of `withChain()`
>>>> will be given to the `Consumer`. The issue is really with the `Consumer`
>>>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>>>
>>>> Maybe a reasonable implementation would be to not add the "branch" to
>>>> the result map if `withConsumer` is used? As long as we clearly document
>>>> it in the JavaDocs, this might be fine?
>>>>
>>>>
>>>>
>>>> (5) Reply to John's comments:
>>>>
>>>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>>>> were talking about the kafka Consumer interface (which doesn’t make
>>>>> sense, of course). I get that you were referring to the java Consumer
>>>>> interface, but we should still probably to to avoid the ambiguity.
>>>>> Just throwing out a suggestion, how about ‘withSink’?
>>>>
>>>> IMHO, `withSink` has the issue that it might be confused with a "sink
>>>> node", ie., writing the KStream to a topic.
>>>>
>>>> Maybe `withJavaConsumer` would make it less ambiguous?
>>>>
>>>>
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>>
>>>>
>>>> On 5/8/20 7:13 AM, John Roesler wrote:
>>>>> Hi Ivan,
>>>>>
>>>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>>>> but I had a few last comments.
>>>>>
>>>>> Thanks,
>>>>> John
>>>>>
>>>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>>>> Hello everyone,
>>>>>>
>>>>>> will someone please take a look at the reworked KIP?
>>>>>>
>>>>>> I believe that now it follows design principles and takes into account
>>>>>> all the arguments discussed here.
>>>>>>
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>>
>>>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>>>> Hi,
>>>>>>>
>>>>>>> I have read the John's "DSL design principles" and have completely
>>>>>>> rewritten the KIP, see
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> This version includes all the previous discussion results and follows
>>>>>>> the design principles, with one exception.
>>>>>>>
>>>>>>> The exception is
>>>>>>>
>>>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>>>
>>>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>>>> here it is justified.
>>>>>>>
>>>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>>>> for the default branch. Thus for both operations we may use a single
>>>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>>>
>>>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>>>> as it
>>>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>>>
>>>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan
>>>>>>>
>>>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>>>> Ivan,
>>>>>>>>
>>>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>>>
>>>>>>>> The DSL improved further in the meantime and we already have a
>>>>>>>> `Named`
>>>>>>>> config object to name operators. It seems reasonable to me to
>>>>>>>> build on
>>>>>>>> this.
>>>>>>>>
>>>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>>>> want to follow:
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>>>
>>>>>>>>
>>>>>>>> -- might be worth to checkout.
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>>>> Hi everyone!
>>>>>>>>>
>>>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>>>
>>>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>>>> June
>>>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>>>> propose other things before this one is finalized.
>>>>>>>>>
>>>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>>>> this
>>>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>>>> the
>>>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>>>> branches, I worked around it this way:
>>>>>>>>>
>>>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>>>        .branch(....)
>>>>>>>>>        .defaultBranch(result::set)
>>>>>>>>>        .onTopOf(someStream);
>>>>>>>>> result.get()...
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>>>
>>>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>>>> from
>>>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>>>
>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>      -> KBranchedStream
>>>>>>>>> // assign a name to the branch and
>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>> //
>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>>      -> KBranchedStream
>>>>>>>>> // default branch is not easily accessible
>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>> // assign custom name to default-branch
>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>> // assign a default name for default
>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>
>>>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>>>> a good
>>>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>>>> you
>>>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>>>> Or,
>>>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>>>> finally,
>>>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>>>> original scope.
>>>>>>>>>
>>>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>>>> familiar with Streams API design principles than me.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>>>> the
>>>>>>>>>> KIP
>>>>>>>>>> at any point.
>>>>>>>>>>
>>>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>>>> do so.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>>>> Ivan,
>>>>>>>>>>>
>>>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>>>> to mix
>>>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>>>
>>>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>>>
>>>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>>>> using a
>>>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>>>> using a
>>>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>>>
>>>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>>>
>>>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>>>> patterns
>>>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>>>> make
>>>>>>>>>>>> totally sense.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>>>
>>>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>>>> `Map` only
>>>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>>>> all of
>>>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>>>
>>>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>>>> `Named` is
>>>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>>>> counter
>>>>>>>>>>>> for
>>>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>>>> names
>>>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>>>
>>>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>>>> misses to
>>>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>>>> keep
>>>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>>>> required
>>>>>>>>>>>> if a
>>>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>>>> specifying a
>>>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> About
>>>>>>>>>>>>
>>>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>
>>>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>>>> hence
>>>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>>>
>>>>>>>>>>>>>         withChain(...);
>>>>>>>>>>>>
>>>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>>>> does not
>>>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>>>> call
>>>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>>>> make sense
>>>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>>>> check
>>>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>>>> `withChain()`
>>>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>>>> returned
>>>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>>>
>>>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>>>> object
>>>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>>>> others,
>>>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>>>> configuration
>>>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>>>> methods that
>>>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>>>> two main
>>>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>>>> satisfy
>>>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>>>> solves
>>>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>>>> to add
>>>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>>>> Otherwise,
>>>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>>>> actually
>>>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>>>> names for
>>>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>>>> logic, it
>>>>>>>>>>>>> does
>>>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>>>> map
>>>>>>>>>>>>> the
>>>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>>>> processing onto
>>>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>>>
>>>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>>>> names
>>>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>>>> proposal, we
>>>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>>>> name,
>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>>>> "mysplit"
>>>>>>>>>>>>>                   .branch(..., ..., "abranch") // creates node
>>>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>>>                   .defaultBranch(...) // creates node
>>>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>>>
>>>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>>>
>>>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>>>> debate
>>>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>>>> general,
>>>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>>>
>>>>>>>>>>>>>         operator(function, config_object?) OR
>>>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>>>
>>>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>>>> variant.
>>>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>>>
>>>>>>>>>>>>>       operator(function, function, string)
>>>>>>>>>>>>>
>>>>>>>>>>>>> where the string is the name.
>>>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>>>> specified
>>>>>>>>>>>>> with
>>>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>>>
>>>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>>>> arguments
>>>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>>>
>>>>>>>>>>>>>        interface BranchConfig extends NamedOperation {
>>>>>>>>>>>>>         withPredicate(...);
>>>>>>>>>>>>>         withChain(...);
>>>>>>>>>>>>>         withName(...);
>>>>>>>>>>>>>       }
>>>>>>>>>>>>>
>>>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>>>> more like
>>>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>>>
>>>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>>>> makes us
>>>>>>>>>>>>> more
>>>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>>>> purely
>>>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>>>> overloads
>>>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>>>
>>>>>>>>>>>>> WDYT?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> -John
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>>>> view.
>>>>>>>>>>>>>> Good
>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>>>       -> KBranchedStream
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>>>> //
>>>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>>>> String)
>>>>>>>>>>>>>>>       -> KBranchedStream
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>>>       -> Map<String,KStream>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>>>> returned
>>>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>>>> language
>>>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>>>> take
>>>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>>>         for the dynamic case.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>>>         question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>>>         in the same scope” since merging the branches back
>>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>         seems like a perfectly plausible use case that can
>>>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>>>         when the branched streams are in the same scope.
>>>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>         for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>         solution - working around the scope thing is easy
>>>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>>>         to.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>>>         <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>>>         LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>>>         matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>>>         advantages than drawbacks.
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>>>         proposal is that all the returned branches are in
>>>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>>>         But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>>>         workaround for the scope problem, described in the
>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>>>         method references instead of lambdas. In real world
>>>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>>>         tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>>>         going to be clean.
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>         handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>>>         handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > - what if we forget to define a handler for a
>>>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>>>         a handler?
>>>>>>>>>>>>>>>>         > - what if we misspell a name?
>>>>>>>>>>>>>>>>         > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > What Michael propose would have been totally OK
>>>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>         writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>>>         "dynamic naming" approach would have looked most
>>>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>         beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>>>         identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>>>         all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>>>         point?
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > ---
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>         without "start branching" operator, and later Paul
>>>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>>>         the case when we have to add a dynamic number of
>>>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>>>         current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>>>         me address both comments here.
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>>>         for it indeed) is critical when we need to do a
>>>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>>>         see example below.
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>>>         Imagine a real-world scenario when you need one
>>>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>>>         value (say, RecordType). You can have something
>>>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>>>         have been much messier.*/
>>>>>>>>>>>>>>>>         > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>>>         > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>>>         >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>>>         recordType,
>>>>>>>>>>>>>>>>         >                     recordType::processRecords);
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > Regards,
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > Ivan
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>         > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>         >> I also agree with Michael's observation about
>>>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>>>         >> current `branch()` implementation.
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>>>         thinking
>>>>>>>>>>>>>>>>         >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>>>         >> `branch()` statement and return a
>>>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>>>         >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>         >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>         >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>         >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>>>         >> An open question is the case for which no
>>>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>         >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>         `BranchedKStream`
>>>>>>>>>>>>>>>>         >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>>>         >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>> real
>>>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>>>         >> because users can just ignore the branch
>>>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>>>         `defaultBranch()`
>>>>>>>>>>>>>>>>         >> in the result `Map` ?
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>>>         >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>>>         argument" yet
>>>>>>>>>>>>>>>>         >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >> -Matthias
>>>>>>>>>>>>>>>>         >>
>>>>>>>>>>>>>>>>         >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>         >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>         that a lambda with the full downstream topology be
>>>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>>>         it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>>>         The advantage of putting the predicate and its
>>>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>         (Consumer) together in branch() is that they are
>>>>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>>>         to each other.
>>>>>>>>>>>>>>>>         >>>
>>>>>>>>>>>>>>>>         >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>>>         branch trees will be hard to read regardless.
>>>>>>>>>>>>>>>>         >>>
>>>>>>>>>>>>>>>>         >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>>>         <michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>         <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>         >>>>
>>>>>>>>>>>>>>>>         >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>>>         downstream
>>>>>>>>>>>>>>>>         >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>         quickly become
>>>>>>>>>>>>>>>>         >>>> harder to read as a single unit.
>>>>>>>>>>>>>>>>         >>>>
>>>>>>>>>>>>>>>>         >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>>>         <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>>         that sets a
>>>>>>>>>>>>>>>>         >>>>> great framework for the discussion.
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>>>         current
>>>>>>>>>>>>>>>>         >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>>>         decisions) is
>>>>>>>>>>>>>>>>         >>>>> roughly this:
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> stream.split()
>>>>>>>>>>>>>>>>         >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>         >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>>>         >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>         construct
>>>>>>>>>>>>>>>>         >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>>>         provides as much
>>>>>>>>>>>>>>>>         >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>         call
>>>>>>>>>>>>>>>>         >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>>>         The value it
>>>>>>>>>>>>>>>>         >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>>>         the same
>>>>>>>>>>>>>>>>         >>>>> scope.
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>         in the sense
>>>>>>>>>>>>>>>>         >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>         branches, but it is
>>>>>>>>>>>>>>>>         >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>>>         the "static"
>>>>>>>>>>>>>>>>         >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>>>         fluently declare and
>>>>>>>>>>>>>>>>         >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>>>         ignore a
>>>>>>>>>>>>>>>>         >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>>>         SortedMap
>>>>>>>>>>>>>>>>         >>>>> solution on top of it.
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> I could also see a middle ground where
>>>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>>>         SortedMap being
>>>>>>>>>>>>>>>>         >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>>>         like this:
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>         >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>>>         >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>>>         >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Pros for that solution:
>>>>>>>>>>>>>>>>         >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>>>         >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>         readable than
>>>>>>>>>>>>>>>>         >>>>> SortedMap
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Cons
>>>>>>>>>>>>>>>>         >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>         makes it harder
>>>>>>>>>>>>>>>>         >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>>>         unlike the KIP)
>>>>>>>>>>>>>>>>         >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>>>         existing
>>>>>>>>>>>>>>>>         >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>         that's overdoing
>>>>>>>>>>>>>>>>         >>>>> it).
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>>>         access the
>>>>>>>>>>>>>>>>         >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>         possible that it
>>>>>>>>>>>>>>>>         >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>         left up to the
>>>>>>>>>>>>>>>>         >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> Paul
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>>>         <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>         >>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>         >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>         existing
>>>>>>>>>>>>>>>>         >>>>> branch
>>>>>>>>>>>>>>>>         >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>>>         proceed by
>>>>>>>>>>>>>>>>         >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>         important to be
>>>>>>>>>>>>>>>>         >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>>>         >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>>>         this KIP.
>>>>>>>>>>>>>>>>         >>>>>>
>>>>>>>>>>>>>>>>         >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>         double brace
>>>>>>>>>>>>>>>>         >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>>>         >>>>> avoid...just
>>>>>>>>>>>>>>>>         >>>>>> my two cents.
>>>>>>>>>>>>>>>>         >>>>>>
>>>>>>>>>>>>>>>>         >>>>>> Cheers,
>>>>>>>>>>>>>>>>         >>>>>> Sophie
>>>>>>>>>>>>>>>>         >>>>>>
>>>>>>>>>>>>>>>>         >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>>>         >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>>>         <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>
>>>>>>>>>>>>>>>>         >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>>>         To me,
>>>>>>>>>>>>>>>>         >>>>> there
>>>>>>>>>>>>>>>>         >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>>>         >>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>         >>>>>> warnings.
>>>>>>>>>>>>>>>>         >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>>>         positionally coupled
>>>>>>>>>>>>>>>>         >>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>>>         >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>         additional code
>>>>>>>>>>>>>>>>         >>>>>>> paths.
>>>>>>>>>>>>>>>>         >>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>>>         constructs
>>>>>>>>>>>>>>>>         >>>>>> would
>>>>>>>>>>>>>>>>         >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>>>         looks like
>>>>>>>>>>>>>>>>         >>>>>> this:
>>>>>>>>>>>>>>>>         >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>>>         Predicate<?
>>>>>>>>>>>>>>>>         >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>>>         >>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>>>         returns a
>>>>>>>>>>>>>>>>         >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>>>         >>>>>> maintained
>>>>>>>>>>>>>>>>         >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>>>> the order
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>         >>>>>> evaluation.
>>>>>>>>>>>>>>>>         >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>>>         solves
>>>>>>>>>>>>>>>>         >>>>>> problem
>>>>>>>>>>>>>>>>         >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>>>         branch you’re
>>>>>>>>>>>>>>>>         >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>         another
>>>>>>>>>>>>>>>>         >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>>>         structure, rather
>>>>>>>>>>>>>>>>         >>>>>> than
>>>>>>>>>>>>>>>>         >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>>>         >>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>>>         historically
>>>>>>>>>>>>>>>>         >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>>>         voluminously, but
>>>>>>>>>>>>>>>>         >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>>>         >>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>>>         <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>>         >>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>>>> "start
>>>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>>>         >>>>> operator
>>>>>>>>>>>>>>>>         >>>>>>> is
>>>>>>>>>>>>>>>>         >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>>>         "Split"
>>>>>>>>>>>>>>>>         >>>>> seems
>>>>>>>>>>>>>>>>         >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>>>         >>>>> branching"
>>>>>>>>>>>>>>>>         >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>> stream
>>>>>>>>>>>>>>>>         >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>         >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>         >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>         >>>>> terminal.
>>>>>>>>>>>>>>>>         >>>>>>> That
>>>>>>>>>>>>>>>>         >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>>>         is, `branch`
>>>>>>>>>>>>>>>>         >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>>>         `void`, to
>>>>>>>>>>>>>>>>         >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>>>         definition of
>>>>>>>>>>>>>>>>         >>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>>>         there's no
>>>>>>>>>>>>>>>>         >>>>>>> default,
>>>>>>>>>>>>>>>>         >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>>>         record
>>>>>>>>>>>>>>>>         >>>>>> falls
>>>>>>>>>>>>>>>>         >>>>>>>> though with no default.
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>> Thanks,
>>>>>>>>>>>>>>>>         >>>>>>>> -John
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>>>         >>>>> matthias@confluent.io
>>>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>>>         >>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>>>         >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>>>         >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>>>         should
>>>>>>>>>>>>>>>>         >>>>>> _not_
>>>>>>>>>>>>>>>>         >>>>>>>>> be an array.
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>         >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>>>         `branch()` does
>>>>>>>>>>>>>>>>         >>>>> not
>>>>>>>>>>>>>>>>         >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>>>> than the
>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>         >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>         >>>>> hidden
>>>>>>>>>>>>>>>>         >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>>>> while
>>>>>>>>>>>>>>>> the others
>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>         >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>>>> code
>>>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>>>         I though
>>>>>>>>>>>>>>>>         >>>>>> it
>>>>>>>>>>>>>>>>         >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>> naming
>>>>>>>>>>>>>>>>         >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>>>> following
>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>         >>>>> cleaner
>>>>>>>>>>>>>>>>         >>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>> me:
>>>>>>>>>>>>>>>>         >>>>>>>>> stream.split()
>>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>         >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>>>         >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>>>         avoid
>>>>>>>>>>>>>>>>         >>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>         >>>>> cannot
>>>>>>>>>>>>>>>>         >>>>>>> have
>>>>>>>>>>>>>>>>         >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>>>         with a
>>>>>>>>>>>>>>>>         >>>>> short
>>>>>>>>>>>>>>>>         >>>>>>>> name?
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>>         with all
>>>>>>>>>>>>>>>>         >>>>> it's
>>>>>>>>>>>>>>>>         >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>>>         contained in
>>>>>>>>>>>>>>>>         >>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>>>         >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>>>         `KBranchedStream#get(int
>>>>>>>>>>>>>>>>         >>>>>>> index)
>>>>>>>>>>>>>>>>         >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>>>         branched-KStreams. Would
>>>>>>>>>>>>>>>>         >>>>>> be
>>>>>>>>>>>>>>>>         >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>>>         that users
>>>>>>>>>>>>>>>>         >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>>>         access them.
>>>>>>>>>>>>>>>>         >>>>> We
>>>>>>>>>>>>>>>>         >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>>>         >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>>>         to get
>>>>>>>>>>>>>>>>         >>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>>>         >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>>>         >>>>>>> String#split
>>>>>>>>>>>>>>>>         >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>>>         loss of
>>>>>>>>>>>>>>>>         >>>>>>>> backwards
>>>>>>>>>>>>>>>>         >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>>>         without
>>>>>>>>>>>>>>>>         >>>>>>>> affecting
>>>>>>>>>>>>>>>>         >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>         >>>>> should
>>>>>>>>>>>>>>>>         >>>>>>> be
>>>>>>>>>>>>>>>>         >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>>         >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>>>         >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>         >>>>>>> however, a
>>>>>>>>>>>>>>>>         >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>>>         with such
>>>>>>>>>>>>>>>>         >>>>>>> name
>>>>>>>>>>>>>>>>         >>>>>>>>> :-)
>>>>>>>>>>>>>>>>         >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>         >>>>> think
>>>>>>>>>>>>>>>>         >>>>>>> that
>>>>>>>>>>>>>>>>         >>>>>>>>>> is not required?
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>>>         something.
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>>>         >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>         >>>>>>> everybody
>>>>>>>>>>>>>>>>         >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>         >>>>>> optimal.
>>>>>>>>>>>>>>>>         >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>         proposal.
>>>>>>>>>>>>>>>>         >>>>>>> There
>>>>>>>>>>>>>>>>         >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>         recommend the
>>>>>>>>>>>>>>>>         >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>>>         >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>>>         >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>>>         BranchingKStream#default()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>>>         `Predicate` as
>>>>>>>>>>>>>>>>         >>>>>>>> argument,
>>>>>>>>>>>>>>>>         >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>>>         accepted and
>>>>>>>>>>>>>>>>         >>>>>> is
>>>>>>>>>>>>>>>>         >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>>>         parameter.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>>>         different
>>>>>>>>>>>>>>>>         >>>>>>>> scopes:
>>>>>>>>>>>>>>>>         >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>>>         index)` method
>>>>>>>>>>>>>>>>         >>>>>>> that
>>>>>>>>>>>>>>>>         >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>>>> result
>>>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>>>         object?
>>>>>>>>>>>>>>>>         >>>>>> Maybe,
>>>>>>>>>>>>>>>>         >>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>>>         >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>>>         >>>>>>>> but
>>>>>>>>>>>>>>>>         >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>         whatever
>>>>>>>>>>>>>>>>         >>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>         current
>>>>>>>>>>>>>>>>         >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>         makes sense
>>>>>>>>>>>>>>>>         >>>>>> for
>>>>>>>>>>>>>>>>         >>>>>>>> you
>>>>>>>>>>>>>>>>         >>>>>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>>>         we'll
>>>>>>>>>>>>>>>>         >>>>> need
>>>>>>>>>>>>>>>>         >>>>>>>> some
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>>>         >>>>> whether
>>>>>>>>>>>>>>>>         >>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> KIP
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>>>         think this
>>>>>>>>>>>>>>>>         >>>>>> is
>>>>>>>>>>>>>>>>         >>>>>>> a
>>>>>>>>>>>>>>>>         >>>>>>>>> good
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>>>         starting a
>>>>>>>>>>>>>>>>         >>>>>>> vote,
>>>>>>>>>>>>>>>>         >>>>>>>>> or if
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>>>> interest
>>>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>>>         assuming
>>>>>>>>>>>>>>>>         >>>>> we
>>>>>>>>>>>>>>>>         >>>>>> do
>>>>>>>>>>>>>>>>         >>>>>>>>> move
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>>>         >>>>>> KBranchedStream",
>>>>>>>>>>>>>>>>         >>>>>>> do
>>>>>>>>>>>>>>>>         >>>>>>>>> we
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>         >>>>> favor
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>         >>>>>>> accomplish
>>>>>>>>>>>>>>>>         >>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>>>         similar
>>>>>>>>>>>>>>>>         >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>         >>>>>>> impossible/difficult
>>>>>>>>>>>>>>>>         >>>>>>>>> that
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>>         >>>>> it's
>>>>>>>>>>>>>>>>         >>>>>>>> just a
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>>>         >>>>>>> particular,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>         >>>>>> modified
>>>>>>>>>>>>>>>>         >>>>>>>> in
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>>>         KStreamLazyBranch is
>>>>>>>>>>>>>>>>         >>>>> a
>>>>>>>>>>>>>>>>         >>>>>>> bit
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>         >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>         >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>>>         convinced: your
>>>>>>>>>>>>>>>>         >>>>>>>> proposal
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>> crucial
>>>>>>>>>>>>>>>>         >>>>>>>>> fact
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>>>         And then
>>>>>>>>>>>>>>>>         >>>>>> it's
>>>>>>>>>>>>>>>>         >>>>>>>> all
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>>>         resume the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>>>         >>>>> starting
>>>>>>>>>>>>>>>>         >>>>>>>> point
>>>>>>>>>>>>>>>>         >>>>>>>>> if
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>>>         starting
>>>>>>>>>>>>>>>>         >>>>>>> point.
>>>>>>>>>>>>>>>>         >>>>>>>>> But
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>         >>>>> details.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>>>         >>>>> stream.branch()
>>>>>>>>>>>>>>>>         >>>>>>>>> solution
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>>>         >>>>> invoked
>>>>>>>>>>>>>>>>         >>>>>> as
>>>>>>>>>>>>>>>>         >>>>>>>>> they’re
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>>>         still
>>>>>>>>>>>>>>>>         >>>>>> ought
>>>>>>>>>>>>>>>>         >>>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>> be
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>         depend on
>>>>>>>>>>>>>>>>         >>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> branched
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>>>         access
>>>>>>>>>>>>>>>>         >>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> branched
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>>>         is, not
>>>>>>>>>>>>>>>>         >>>>>>>> inside
>>>>>>>>>>>>>>>>         >>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>         >>>>> solutions.
>>>>>>>>>>>>>>>>         >>>>>> It
>>>>>>>>>>>>>>>>         >>>>>>>>> can be
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>>>         excited
>>>>>>>>>>>>>>>>         >>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>> hear
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>         >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>>>         >>>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>>>> me at
>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>         >>>>> glance,
>>>>>>>>>>>>>>>>         >>>>>>> but
>>>>>>>>>>>>>>>>         >>>>>>>>> ---
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>>>         same
>>>>>>>>>>>>>>>>         >>>>>> scope
>>>>>>>>>>>>>>>>         >>>>>>> as
>>>>>>>>>>>>>>>>         >>>>>>>>> each
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>>>> them back
>>>>>>>>>>>>>>> together
>>>>>>>>>>>>>>>>         >>>>> again
>>>>>>>>>>>>>>>>         >>>>>> I
>>>>>>>>>>>>>>>>         >>>>>>>>> don't see
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>>>         just
>>>>>>>>>>>>>>>>         >>>>>> going
>>>>>>>>>>>>>>>>         >>>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>>>         we need
>>>>>>>>>>>>>>>>         >>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>> identify
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>>>         in the
>>>>>>>>>>>>>>>>         >>>>>>>>> electronics
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>         circumstances
>>>>>>>>>>>>>>>>         >>>>>> using
>>>>>>>>>>>>>>>>         >>>>>>>> my
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>         complex, so
>>>>>>>>>>>>>>>>         >>>>>>>>> creation of
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>>>         order to
>>>>>>>>>>>>>>>>         >>>>>>>> separate
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>>>         >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>>>         everything
>>>>>>>>>>>>>>>>         >>>>>>>> later,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>>>         initialize the
>>>>>>>>>>>>>>>>         >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>>>> called
>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>         >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>>>         essentially
>>>>>>>>>>>>>>>>         >>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> next
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>>>         based on
>>>>>>>>>>>>>>>>         >>>>> my
>>>>>>>>>>>>>>>>         >>>>>>>>> experience,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>>>         fluent
>>>>>>>>>>>>>>>>         >>>>> API
>>>>>>>>>>>>>>>>         >>>>>>>> based
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>>>         (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>>>         >>>>>> and
>>>>>>>>>>>>>>>>         >>>>>>> I
>>>>>>>>>>>>>>>>         >>>>>>>>> think
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>>>         >>>>>>> compatibility
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    there aren't any direct ones.
>>>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>>>         that Java
>>>>>>>>>>>>>>>>         >>>>> is
>>>>>>>>>>>>>>>>         >>>>>>>> smart
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>>>         returning one
>>>>>>>>>>>>>>>>         >>>>>>> thing
>>>>>>>>>>>>>>>>         >>>>>>>>> and
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>>>> another
>>>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>         need
>>>>>>>>>>>>>>>>         >>>>> it.
>>>>>>>>>>>>>>>>         >>>>>>> We
>>>>>>>>>>>>>>>>         >>>>>>>>> can
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>>>         shares
>>>>>>>>>>>>>>>>         >>>>>> its
>>>>>>>>>>>>>>>>         >>>>>>>>> state
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         branching.
>>>>>>>>>>>>>>>>         >>>>>>> It's
>>>>>>>>>>>>>>>>         >>>>>>>>> not
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>>>         demonstrates
>>>>>>>>>>>>>>>>         >>>>>> its
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>>>> request should
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>         >>>>> final
>>>>>>>>>>>>>>>>         >>>>>> or
>>>>>>>>>>>>>>>>         >>>>>>>>> even a
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>>>         wanted to
>>>>>>>>>>>>>>>>         >>>>>> see
>>>>>>>>>>>>>>>>         >>>>>>>> how
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>>>> API
>>>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>         solution
>>>>>>>>>>>>>>>>         >>>>>>> could
>>>>>>>>>>>>>>>>         >>>>>>>> be
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>>>         >>>>> suggested
>>>>>>>>>>>>>>>>         >>>>>>>> was a
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>         streams
>>>>>>>>>>>>>>>>         >>>>>> are
>>>>>>>>>>>>>>>>         >>>>>>>> not
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>>>         is, if we
>>>>>>>>>>>>>>>>         >>>>>>> wanted
>>>>>>>>>>>>>>>>         >>>>>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>>>         that.  The
>>>>>>>>>>>>>>>>         >>>>>> KIP
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>         >>>>>> either
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>>>         table.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>         >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>         >>>>>> point.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>         branch API
>>>>>>>>>>>>>>>>         >>>>>>> needs
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>>>         returns
>>>>>>>>>>>>>>>>         >>>>>> its
>>>>>>>>>>>>>>>>         >>>>>>>>> argument
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>>>> The code
>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>         >>>>> make
>>>>>>>>>>>>>>>>         >>>>>>>> sense
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>         instance
>>>>>>>>>>>>>>>>         >>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>>>         >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>         >>>>>> defined.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>         >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>>>         >>>>>>>>> and
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>>>         miss the
>>>>>>>>>>>>>>>>         >>>>>> fact
>>>>>>>>>>>>>>>>         >>>>>>>>> that one
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>>>         methods
>>>>>>>>>>>>>>>>         >>>>>> are
>>>>>>>>>>>>>>>>         >>>>>>>> not
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>>>> we do
>>>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>>         implemented the
>>>>>>>>>>>>>>>>         >>>>>>> easy
>>>>>>>>>>>>>>>>         >>>>>>>>> way.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>>>> method that
>>>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>>>         >>>>> nothing
>>>>>>>>>>>>>>>>         >>>>>>>> will
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>>> the only
>>>>>>>>>>>>>>> option
>>>>>>>>>>>>>>>>         >>>>>> besides
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>>>         want to
>>>>>>>>>>>>>>>>         >>>>>> just
>>>>>>>>>>>>>>>>         >>>>>>>>> silently
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>>>         predicate. 2)
>>>>>>>>>>>>>>>>         >>>>>>> Throwing
>>>>>>>>>>>>>>>>         >>>>>>>>> an
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>         looks
>>>>>>>>>>>>>>>>         >>>>>> like a
>>>>>>>>>>>>>>>>         >>>>>>>> bad
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>>>         emit a
>>>>>>>>>>>>>>>>         >>>>>>>> special
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>         >>>>>>> `default`
>>>>>>>>>>>>>>>>         >>>>>>>>> can
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>         >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>         >>>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>> track
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>         a clear
>>>>>>>>>>>>>>>>         >>>>>>> error
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>>>         >>>>> compiled
>>>>>>>>>>>>>>>>         >>>>>>> and
>>>>>>>>>>>>>>>>         >>>>>>>>> run?
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>>>         compile if
>>>>>>>>>>>>>>>>         >>>>> used
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>>>         method chain
>>>>>>>>>>>>>>>>         >>>>>>>> starting
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>>>         between
>>>>>>>>>>>>>>>>         >>>>>>>> runtime
>>>>>>>>>>>>>>>>         >>>>>>>>> and
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>>>         >>>>> instantly
>>>>>>>>>>>>>>>>         >>>>>> on
>>>>>>>>>>>>>>>>         >>>>>>>>> unit
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>>>> project
>>>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>>>         compilation
>>>>>>>>>>>>>>>>         >>>>>>>> failure.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>>>         required.
>>>>>>>>>>>>>>>>         >>>>>>> But
>>>>>>>>>>>>>>>>         >>>>>>>> is
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>>>         >>>>>> defaultBranch
>>>>>>>>>>>>>>>>         >>>>>>>>> they
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>>>         just as
>>>>>>>>>>>>>>>>         >>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>>>         >>>>> user
>>>>>>>>>>>>>>>>         >>>>>>>> could
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>> default
>>>>>>>>>>>>>>>>         >>>>>>>>> branch,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>         >>>>> seems
>>>>>>>>>>>>>>>>         >>>>>>> like
>>>>>>>>>>>>>>>>         >>>>>>>>> an
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>         which allows
>>>>>>>>>>>>>>>>         >>>>>> for
>>>>>>>>>>>>>>>>         >>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>>>         to be
>>>>>>>>>>>>>>>>         >>>>>> well
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>>>         >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>>>         >>>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>> track
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>>>         a clear
>>>>>>>>>>>>>>>>         >>>>>>> error
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>>>> there is
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>         >>>>> "build
>>>>>>>>>>>>>>>>         >>>>>>>> step"
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>         >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>>>         >>>>>>> is
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>>>         >>>>> that
>>>>>>>>>>>>>>>>         >>>>>>> it's
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>>>         stream.
>>>>>>>>>>>>>>>>         >>>>>>> With
>>>>>>>>>>>>>>>>         >>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>> way all
>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>         >>>>>>> operations
>>>>>>>>>>>>>>>>         >>>>>>>>> do -
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>         >>>>> times,
>>>>>>>>>>>>>>>>         >>>>>>> you
>>>>>>>>>>>>>>>>         >>>>>>>>> just
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>>>         operations
>>>>>>>>>>>>>>>>         >>>>>> on
>>>>>>>>>>>>>>>>         >>>>>>> it
>>>>>>>>>>>>>>>>         >>>>>>>>> as
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>>>         >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>         always need
>>>>>>>>>>>>>>>>         >>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>>>         >>>>> don't
>>>>>>>>>>>>>>>>         >>>>>>>> know
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>>>         so we
>>>>>>>>>>>>>>>>         >>>>> can
>>>>>>>>>>>>>>>>         >>>>>> do
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>>>> need of
>>>>>>>>>>>>>>> special
>>>>>>>>>>>>>>>>         >>>>> object
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>         >>>>> here
>>>>>>>>>>>>>>>>         >>>>>> we
>>>>>>>>>>>>>>>>         >>>>>>>>> have a
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>>>         flow,
>>>>>>>>>>>>>>>>         >>>>> so
>>>>>>>>>>>>>>>>         >>>>>> I
>>>>>>>>>>>>>>>>         >>>>>>>>> think
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>         API, but I
>>>>>>>>>>>>>>>>         >>>>>> find
>>>>>>>>>>>>>>>>         >>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>>>         contrasts the
>>>>>>>>>>>>>>>>         >>>>>>> fluency
>>>>>>>>>>>>>>>>         >>>>>>>>> of
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> KStream method calls.
>>>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>>>         just call
>>>>>>>>>>>>>>>>         >>>>> a
>>>>>>>>>>>>>>>>         >>>>>>>>> method on
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>>>         cases
>>>>>>>>>>>>>>>>         >>>>> are
>>>>>>>>>>>>>>>>         >>>>>>>>> defined
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>>>         is very
>>>>>>>>>>>>>>>>         >>>>>> nice
>>>>>>>>>>>>>>>>         >>>>>>>>> and the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>         how we
>>>>>>>>>>>>>>>>         >>>>>>> specify
>>>>>>>>>>>>>>>>         >>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>>>         >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>>>         >>>>>>>>> or
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>>>         terminated by
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>>>         incompatible with
>>>>>>>>>>>>>>>>         >>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> current
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>         different
>>>>>>>>>>>>>>>>         >>>>>> name,
>>>>>>>>>>>>>>>>         >>>>>>>> but
>>>>>>>>>>>>>>>>         >>>>>>>>> that
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>>>         >>>>>> something
>>>>>>>>>>>>>>>>         >>>>>>>> like
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>         KIP?  It
>>>>>>>>>>>>>>>>         >>>>>> seems
>>>>>>>>>>>>>>>>         >>>>>>>>> like it
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>>>         while also
>>>>>>>>>>>>>>>>         >>>>>>>> allowing
>>>>>>>>>>>>>>>>         >>>>>>>>> you
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>>>         KBranchedStreams
>>>>>>>>>>>>>>>>         >>>>>> if
>>>>>>>>>>>>>>>>         >>>>>>>>> desired.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>>>         ks){
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>>>         String> ks){
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>>>         this::handleFirstCase)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>>>         this::handleSecondCase)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>>>         >>>>> takes a
>>>>>>>>>>>>>>>>         >>>>>>>>> Consumer
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>>>         example in
>>>>>>>>>>>>>>>>         >>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>> KIP
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>>>         >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>>>> how would
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>         >>>>> handle
>>>>>>>>>>>>>>>>         >>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>> case
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>         >>>>>>>> processing
>>>>>>>>>>>>>>>>         >>>>>>>>> and
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>>>         >>>>>> stream
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>>>         we had
>>>>>>>>>>>>>>>>         >>>>>>>> something
>>>>>>>>>>>>>>>>         >>>>>>>>> like
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>         >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>>>         >>>>> 418.
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>>>         >>>>> Please
>>>>>>>>>>>>>>>>         >>>>>>>> take
>>>>>>>>>>>>>>>>         >>>>>>>>> a
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>>>         feedback :)
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>>>         >>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>
>>>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>>>         >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         >>>>>>>>>
>>>>>>>>>>>>>>>>         >
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>
>>
>>
>> Attachments:
>> * signature.asc


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Please excuse my typo:
“since I’m your example”
Should have been:
“as in your example”

Thanks,
John

On Wed, May 20, 2020, at 22:54, John Roesler wrote:
> Thanks for this thought, Matthias,
> 
> Your idea has a few aspects I find attractive:
> 1. There’s no ambiguity at all about what will be in the map, because 
> there’s only one thing that could be there, which is whatever is 
> returned from the chain function.
> 2. We keep the API smaller. Thanks to the extensible way this KIP is 
> designed, it would be trivially easy to add the “terminal” chain later. 
> As you say, fewer concepts leads to an API that is easier to learn.
> 3. We get to side-step the naming of this method. Although I didn’t 
> complain about withJavaConsumer, it was only because I couldn’t think 
> of a better name. Still, it’s somewhat unsatisfying to name a method 
> after its argument type, since this provides no information at all 
> about what the method does. I was willing to accept it because I didn’t 
> have an alternative, but I would be happy to skip this method for now 
> to avoid the problem until we have more inspiration. 
> 
> The only con I see is that it makes the code a little less ergonomic to 
> write when you don’t want to return the result of the chain (such as 
> when the chain is terminal), since I’m your example, you have to 
> declare a block with a return statement at the end. It’s not ideal, but 
> it doesn’t seem too bad to me. 
> 
> Lastly, on the null question, I’d be fine with allowing a null result, 
> which would just remove the branch from the returned map. It seems 
> nicer than forcing people to pick a stream to return when their chain 
> is terminal and they don’t want to use the result later.
> 
> Thanks again for sharing the idea,
> John
> 
> On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> > Thanks for updating the KIP!
> > 
> > I guess the only open question is about `Branched.withJavaConsumer` and
> > its relationship to the returned `Map`.
> > 
> > Originally, we discussed two main patterns:
> > 
> >  (1) split a stream and return the substreams for futher processing
> >  (2) split a stream and modify the substreams with in-place method chaining
> > 
> > To combine both patterns we wanted to allow for
> > 
> >   -> split a stream, modify the substreams, and return the _modified_
> > substreams for further processing
> > 
> > > But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> > 
> > That is of course possible. However, it introduces some "hidded" semantics:
> > 
> >  - using `withChain` I get the modified sub-stream
> >  - using `withJavaConsumer` I get the unmodifed sub-stream
> > 
> > This seems to be quite subtle to me.
> > 
> > 
> > 
> > From my understanding the original idea of `withJavaConsumer` was to
> > model a terminal operation, ie, it should be similar to:
> > 
> > Branched.withChain(s -> {
> >   s.to();
> >   return null;
> > })
> > 
> > However, I am not sure if we should even allow `withChain()` to return
> > `null`? IMHO, we should throw an exception for this case to avoid a `key
> > -> null` entry in the returned Map.
> > 
> > Following this train of through, and if we want to allow the "return
> > null" pattern in general, we need `withJavaConsumer` that does not add
> > an entry to the Map.
> > 
> > Following your proposal, the semantics of `withJavaConsumer` could also
> > be achieved with `withChain`:
> > 
> > Branched.withChain(s -> {
> >   s.to();
> >   return s;
> > })
> > 
> > Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> > while for the first proposal it adds new functionality (if `return null`
> > is not allowed, using `withChain()` is not possible to "hide a
> > sub-stream in the result). Furthermore, we might need to allow `return
> > null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> > 
> > 
> > 
> > I guess I can be convinced either way. However, if we follow your
> > proposal, I am wondering if we need `withJavaConsumer` at all? Its
> > benefit seems to be small? Also, having a reduced API is usually
> > preferable as it's simpler to learn.
> > 
> > 
> > 
> > -Matthias
> > 
> > 
> > 
> > 
> > On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> > > Hello, John, hello Matthias!
> > > 
> > > Thank you very much for your detailed feedback!
> > > 
> > > -----------------------------------------
> > > 
> > > John,
> > > 
> > >> It looks like you missed my reply on Apr 23rd.
> > > 
> > > For some unknown reason it didn't reach my inbox, fortunately we have
> > > all the emails on the web.
> > > 
> > >> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> > > method?
> > > 
> > > Done, in "Compatibility, Deprecation, and Migration Plan" section.
> > > 
> > >> 2. [Explain why 'branch' operator is superior to branching directly
> > > off of the parent KStream for the needs of dynamic branching]
> > > 
> > > Done, see an ugly counterexample in 'Dynamic Branching' section.
> > > 
> > >> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> > > 
> > > As Mathhias noted, `withSink` can also be confusing. I renamed this
> > > method to `withJavaConsumer` per Matthias' suggestion.
> > > 
> > >> 4. ...It seems like there are two disjoint use cases: EITHER using
> > > chain and the result map OR using just the sink
> > > 
> > > This is discussed below.
> > > 
> > > ----------------------------------------------
> > > 
> > > Mathhias,
> > > 
> > >> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> > > 
> > > Done.
> > > 
> > >> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> > > method]
> > > 
> > > Fixed.
> > > 
> > > 
> > >> 3. Overview of newly added methods/interfaces
> > > 
> > > Done in `Proposed Changes` section.
> > > 
> > > 
> > >> 4. [Concerning John's note] > I don't think that using both
> > > `withChain()` and `withConsumer()` is the
> > > issue, as the KIP clearly states that the result of `withChain()` will
> > > be given to the `Consumer`.
> > > 
> > > Yes, I agree!
> > > 
> > >> The issue is really with the `Consumer` and the returned `Map` of
> > > `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> > > implementation would be to not add the "branch" to the result map if
> > > `withConsumer` is used?
> > > 
> > > But is it also an issue? With Kafka Streams, we can split the topology
> > > graph at any point. Technically, it's OK to do both: feed the KStream to
> > > a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> > > stream in the Map, one simply does not extract it from there :-)
> > > 
> > > In the current version of KIP it is assumed that the returned map
> > > contains ALL the branches, either tagged with IDs explicitly set by the
> > > programmer, or with some default auto-generated ids. Dealing with this
> > > map is the user's responsibility.
> > > 
> > > What seems to me to be an issue is introducing exclusions to this
> > > general rule, like 'swallowing' some streams by provided
> > > [Java]Consumers. This can make things complicated. What if a user
> > > provides both the name of the branch and a [Java]Consumer? What do they
> > > mean in this case? Should we 'swallow' the stream or save it to the map?
> > > There's no point in 'saving the space' in this map, so maybe just leave
> > > it as it is?
> > > 
> > > ----
> > > 
> > > I rewrote the KIP and also fixed a couple of typos.
> > > 
> > > Looking forward for your feedback again!
> > > 
> > > Regards,
> > > 
> > > Ivan.
> > > 
> > > 
> > > 
> > > 08.05.2020 22:55, Matthias J. Sax пишет:
> > >> Thanks for updating the KIP!
> > >>
> > >> I also have some minor comment:
> > >>
> > >>
> > >>
> > >> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> > >>
> > >> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> > >> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> > >> and `KGroupedKTable` that we cannot rename without a breaking change...
> > >> so we just keep them.)
> > >>
> > >>
> > >>
> > >> (2) Quote:
> > >>
> > >>> Both branch and defaultBranch operations also have overloaded
> > >>> parameterless alternatives.
> > >>
> > >> I think `branch()` always needs to take a `Predicate` and assume you
> > >> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> > >> as `branch()` would not be "parameterless".
> > >>
> > >>
> > >>
> > >> (3) Can you maybe add an overview in the "Public Interface" section) of
> > >> newly added and deprecated methods/classes (cf.
> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> > >>
> > >>
> > >>
> > >>
> > >> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> > >> and the finally returned `Map<String, KStream>`. This related to John's
> > >> 4th comment:
> > >>
> > >>> It seems like there are really two disjoint use cases: EITHER using
> > >>> chain and the result map OR using just the sink.
> > >>
> > >> I don't think that using both `withChain()` and `withConsumer()` is the
> > >> issue though, as the KIP clearly states that the result of `withChain()`
> > >> will be given to the `Consumer`. The issue is really with the `Consumer`
> > >> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> > >>
> > >> Maybe a reasonable implementation would be to not add the "branch" to
> > >> the result map if `withConsumer` is used? As long as we clearly document
> > >> it in the JavaDocs, this might be fine?
> > >>
> > >>
> > >>
> > >> (5) Reply to John's comments:
> > >>
> > >>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> > >>> were talking about the kafka Consumer interface (which doesn’t make
> > >>> sense, of course). I get that you were referring to the java Consumer
> > >>> interface, but we should still probably to to avoid the ambiguity.
> > >>> Just throwing out a suggestion, how about ‘withSink’?
> > >>
> > >> IMHO, `withSink` has the issue that it might be confused with a "sink
> > >> node", ie., writing the KStream to a topic.
> > >>
> > >> Maybe `withJavaConsumer` would make it less ambiguous?
> > >>
> > >>
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>
> > >>
> > >>
> > >> On 5/8/20 7:13 AM, John Roesler wrote:
> > >>> Hi Ivan,
> > >>>
> > >>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> > >>> but I had a few last comments.
> > >>>
> > >>> Thanks,
> > >>> John
> > >>>
> > >>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> > >>>> Hello everyone,
> > >>>>
> > >>>> will someone please take a look at the reworked KIP?
> > >>>>
> > >>>> I believe that now it follows design principles and takes into account
> > >>>> all the arguments discussed here.
> > >>>>
> > >>>>
> > >>>> Regards,
> > >>>>
> > >>>> Ivan
> > >>>>
> > >>>>
> > >>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> > >>>>> Hi,
> > >>>>>
> > >>>>> I have read the John's "DSL design principles" and have completely
> > >>>>> rewritten the KIP, see
> > >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> This version includes all the previous discussion results and follows
> > >>>>> the design principles, with one exception.
> > >>>>>
> > >>>>> The exception is
> > >>>>>
> > >>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> > >>>>>
> > >>>>> which formally violates 'no more than one parameter' rule, but I think
> > >>>>> here it is justified.
> > >>>>>
> > >>>>> We must provide a predicate for a branch and don't need to provide one
> > >>>>> for the default branch. Thus for both operations we may use a single
> > >>>>> Branched parameter class, with an extra method parameter for `branch`.
> > >>>>>
> > >>>>> Since predicate is a natural, necessary part of a branch, no
> > >>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> > >>>>> as it
> > >>>>> is said in the rationale for the 'single parameter rule'.
> > >>>>>
> > >>>>> WDYT, is this KIP mature enough to begin voting?
> > >>>>>
> > >>>>> Regards,
> > >>>>>
> > >>>>> Ivan
> > >>>>>
> > >>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> > >>>>>> Ivan,
> > >>>>>>
> > >>>>>> no worries about getting side tracked. Glad to have you back!
> > >>>>>>
> > >>>>>> The DSL improved further in the meantime and we already have a
> > >>>>>> `Named`
> > >>>>>> config object to name operators. It seems reasonable to me to
> > >>>>>> build on
> > >>>>>> this.
> > >>>>>>
> > >>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> > >>>>>> want to follow:
> > >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> > >>>>>>
> > >>>>>>
> > >>>>>> -- might be worth to checkout.
> > >>>>>>
> > >>>>>>
> > >>>>>> -Matthias
> > >>>>>>
> > >>>>>>
> > >>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> > >>>>>>> Hi everyone!
> > >>>>>>>
> > >>>>>>> Let me revive the discussion of this KIP.
> > >>>>>>>
> > >>>>>>> I'm very sorry for stopping my participation in the discussion in
> > >>>>>>> June
> > >>>>>>> 2019. My project work was very intensive then and it didn't leave me
> > >>>>>>> spare time. But I think I must finish this, because we invested
> > >>>>>>> substantial effort into this discussion and I'm not feel entitled to
> > >>>>>>> propose other things before this one is finalized.
> > >>>>>>>
> > >>>>>>> During these months I proceeded with writing and reviewing Kafka
> > >>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> > >>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> > >>>>>>> this
> > >>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> > >>>>>>> the
> > >>>>>>> KIP forward. When I was coming across the problem with the scope of
> > >>>>>>> branches, I worked around it this way:
> > >>>>>>>
> > >>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> > >>>>>>> new KafkaStreamBrancher<....>()
> > >>>>>>>       .branch(....)
> > >>>>>>>       .defaultBranch(result::set)
> > >>>>>>>       .onTopOf(someStream);
> > >>>>>>> result.get()...
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> And yes, of course I don't feel very happy with this approach.
> > >>>>>>>
> > >>>>>>> I think that Matthias came up with a bright solution in his post
> > >>>>>>> from
> > >>>>>>> May, 24th 2019. Let me quote it:
> > >>>>>>>
> > >>>>>>> KStream#split() -> KBranchedStream
> > >>>>>>> // branch is not easily accessible in current scope
> > >>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> > >>>>>>>     -> KBranchedStream
> > >>>>>>> // assign a name to the branch and
> > >>>>>>> // return the sub-stream to the current scope later
> > >>>>>>> //
> > >>>>>>> // can be simple as `#branch(p, s->s, "name")`
> > >>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> > >>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> > >>>>>>>     -> KBranchedStream
> > >>>>>>> // default branch is not easily accessible
> > >>>>>>> // return map of all named sub-stream into current scope
> > >>>>>>> KBranchedStream#default(Cosumer<KStream>)
> > >>>>>>>     -> Map<String,KStream>
> > >>>>>>> // assign custom name to default-branch
> > >>>>>>> // return map of all named sub-stream into current scope
> > >>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> > >>>>>>>     -> Map<String,KStream>
> > >>>>>>> // assign a default name for default
> > >>>>>>> // return map of all named sub-stream into current scope
> > >>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> > >>>>>>>     -> Map<String,KStream>
> > >>>>>>> // return map of all names sub-stream into current scope
> > >>>>>>> KBranchedStream#noDefaultBranch()
> > >>>>>>>     -> Map<String,KStream>
> > >>>>>>>
> > >>>>>>> I believe this would satisfy everyone. Optional names seems to be
> > >>>>>>> a good
> > >>>>>>> idea: when you don't need to have the branches in the same scope,
> > >>>>>>> you
> > >>>>>>> just don't use names and you don't risk making your code brittle.
> > >>>>>>> Or,
> > >>>>>>> you might want to add names just for debugging purposes. Or,
> > >>>>>>> finally,
> > >>>>>>> you might use the returned Map to have the named branches in the
> > >>>>>>> original scope.
> > >>>>>>>
> > >>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> > >>>>>>> suggested using Named class. I can't comment on this. The idea seems
> > >>>>>>> reasonable, but in this matter I'd rather trust people who are more
> > >>>>>>> familiar with Streams API design principles than me.
> > >>>>>>>
> > >>>>>>> Regards,
> > >>>>>>>
> > >>>>>>> Ivan
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> > >>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> > >>>>>>>> the
> > >>>>>>>> KIP
> > >>>>>>>> at any point.
> > >>>>>>>>
> > >>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> > >>>>>>>> do so.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> -Matthias
> > >>>>>>>>
> > >>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> > >>>>>>>>> Ivan,
> > >>>>>>>>>
> > >>>>>>>>> did you see my last reply? What do you think about my proposal
> > >>>>>>>>> to mix
> > >>>>>>>>> both approaches and try to get best-of-both worlds?
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> -Matthias
> > >>>>>>>>>
> > >>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> > >>>>>>>>>> Thanks for the input John!
> > >>>>>>>>>>
> > >>>>>>>>>>> under your suggestion, it seems that the name is required
> > >>>>>>>>>>
> > >>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> > >>>>>>>>>> using a
> > >>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> > >>>>>>>>>> using a
> > >>>>>>>>>> `Consumer`, no.
> > >>>>>>>>>>
> > >>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> > >>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> > >>>>>>>>>>
> > >>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> > >>>>>>>>>> patterns
> > >>>>>>>>>> only. Your suggestions to align the new API with the existing API
> > >>>>>>>>>> make
> > >>>>>>>>>> totally sense.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> One follow up question: Would `Named` be optional or required in
> > >>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> > >>>>>>>>>>
> > >>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> > >>>>>>>>>> `Map` only
> > >>>>>>>>>> contains the corresponding branches, so why should we prefix
> > >>>>>>>>>> all of
> > >>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> > >>>>>>>>>> `split()`, the same question raises?
> > >>>>>>>>>>
> > >>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> > >>>>>>>>>> `Named` is
> > >>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> > >>>>>>>>>> counter
> > >>>>>>>>>> for
> > >>>>>>>>>> different branch name. However, this might lead to the problem of
> > >>>>>>>>>> changing names if branches are added/removed. Also, how would the
> > >>>>>>>>>> names
> > >>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> > >>>>>>>>>> returned in the `Map`).
> > >>>>>>>>>>
> > >>>>>>>>>> If `Named` is optional for both, it could happen that a user
> > >>>>>>>>>> misses to
> > >>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> > >>>>>>>>>> keep
> > >>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> > >>>>>>>>>> required
> > >>>>>>>>>> if a
> > >>>>>>>>>> `Function` is used. This makes it explicit to the user that
> > >>>>>>>>>> specifying a
> > >>>>>>>>>> name is required if a `Function` is used.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> About
> > >>>>>>>>>>
> > >>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> > >>>>>>>>>>
> > >>>>>>>>>> I don't think that the branching predicate is a configuration and
> > >>>>>>>>>> hence
> > >>>>>>>>>> would not include it in a configuration object.
> > >>>>>>>>>>
> > >>>>>>>>>>>        withChain(...);
> > >>>>>>>>>>
> > >>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> > >>>>>>>>>> does not
> > >>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> > >>>>>>>>>> call
> > >>>>>>>>>> `withName()` in combination of `withChain()` what does not
> > >>>>>>>>>> make sense
> > >>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> > >>>>>>>>>> check
> > >>>>>>>>>> seems less appealing. Also, it could happen that neither
> > >>>>>>>>>> `withChain()`
> > >>>>>>>>>> not `withName()` is called and the branch is missing in the
> > >>>>>>>>>> returned
> > >>>>>>>>>> `Map` what lead to runtime issues, too.
> > >>>>>>>>>>
> > >>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> > >>>>>>>>>> object
> > >>>>>>>>>> is helpful if each configuration can be set independently of all
> > >>>>>>>>>> others,
> > >>>>>>>>>> but this seems not to be the case here. If we add new
> > >>>>>>>>>> configuration
> > >>>>>>>>>> later, we can also just move forward by deprecating the
> > >>>>>>>>>> methods that
> > >>>>>>>>>> accept `Named` and add new methods that accepted
> > >>>>>>>>>> `BranchConfig` (that
> > >>>>>>>>>> would of course implement `Named`).
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> Thoughts?
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> > >>>>>>>>>> two main
> > >>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> -Matthias
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> > >>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> > >>>>>>>>>>> satisfy
> > >>>>>>>>>>> everyone. Returning the map from the terminal operations also
> > >>>>>>>>>>> solves
> > >>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> > >>>>>>>>>>> to add
> > >>>>>>>>>>> support for the compliment later on.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Under your suggestion, it seems that the name is required.
> > >>>>>>>>>>> Otherwise,
> > >>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> > >>>>>>>>>>> actually
> > >>>>>>>>>>> not too bad, since experience has taught us that, although
> > >>>>>>>>>>> names for
> > >>>>>>>>>>> operations are not required to define stream processing
> > >>>>>>>>>>> logic, it
> > >>>>>>>>>>> does
> > >>>>>>>>>>> significantly improve the operational experience when you can
> > >>>>>>>>>>> map
> > >>>>>>>>>>> the
> > >>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> > >>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> > >>>>>>>>>>> processing onto
> > >>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> > >>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> > >>>>>>>>>>>
> > >>>>>>>>>>> In the current implementation of Branch, you can name the branch
> > >>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> > >>>>>>>>>>> names
> > >>>>>>>>>>> built from the branch operator name. I guess under this
> > >>>>>>>>>>> proposal, we
> > >>>>>>>>>>> could naturally append the branch name to the branching operator
> > >>>>>>>>>>> name,
> > >>>>>>>>>>> like this:
> > >>>>>>>>>>>
> > >>>>>>>>>>>       stream.split(Named.withName("mysplit")) //creates node
> > >>>>>>>>>>> "mysplit"
> > >>>>>>>>>>>                  .branch(..., ..., "abranch") // creates node
> > >>>>>>>>>>> "mysplit-abranch"
> > >>>>>>>>>>>                  .defaultBranch(...) // creates node
> > >>>>>>>>>>> "mysplit-default"
> > >>>>>>>>>>>
> > >>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> > >>>>>>>>>>>
> > >>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> > >>>>>>>>>>> debate
> > >>>>>>>>>>> the "best" syntax in the context of each operation, but in
> > >>>>>>>>>>> general,
> > >>>>>>>>>>> the KStream DSL operators follow this pattern:
> > >>>>>>>>>>>
> > >>>>>>>>>>>        operator(function, config_object?) OR
> > >>>>>>>>>>> operator(config_object)
> > >>>>>>>>>>>
> > >>>>>>>>>>> where config_object is often just Named in the "function"
> > >>>>>>>>>>> variant.
> > >>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> > >>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Here, we're introducing a totally different pattern:
> > >>>>>>>>>>>
> > >>>>>>>>>>>      operator(function, function, string)
> > >>>>>>>>>>>
> > >>>>>>>>>>> where the string is the name.
> > >>>>>>>>>>> My first question is whether the name should instead be
> > >>>>>>>>>>> specified
> > >>>>>>>>>>> with
> > >>>>>>>>>>> the NamedOperation interface.
> > >>>>>>>>>>>
> > >>>>>>>>>>> My second question is whether we should just roll all these
> > >>>>>>>>>>> arguments
> > >>>>>>>>>>> up into a config object like:
> > >>>>>>>>>>>
> > >>>>>>>>>>>       KBranchedStream#branch(BranchConfig)
> > >>>>>>>>>>>
> > >>>>>>>>>>>       interface BranchConfig extends NamedOperation {
> > >>>>>>>>>>>        withPredicate(...);
> > >>>>>>>>>>>        withChain(...);
> > >>>>>>>>>>>        withName(...);
> > >>>>>>>>>>>      }
> > >>>>>>>>>>>
> > >>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> > >>>>>>>>>>> more like
> > >>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> > >>>>>>>>>>>
> > >>>>>>>>>>> This makes the source code a little noisier, but it also
> > >>>>>>>>>>> makes us
> > >>>>>>>>>>> more
> > >>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> > >>>>>>>>>>> purely
> > >>>>>>>>>>> in the config interface, and never have to deal with adding
> > >>>>>>>>>>> overloads
> > >>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> > >>>>>>>>>>> optional, or the KStream->KStream to be optional.
> > >>>>>>>>>>>
> > >>>>>>>>>>> WDYT?
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks,
> > >>>>>>>>>>> -John
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> > >>>>>>>>>>> <mi...@confluent.io> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> > >>>>>>>>>>>> view.
> > >>>>>>>>>>>> Good
> > >>>>>>>>>>>> suggestion.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> > >>>>>>>>>>>> <ma...@confluent.io>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> Interesting discussion.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> > >>>>>>>>>>>>> approaches:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> KStream#split() -> KBranchedStream
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // branch is not easily accessible in current scope
> > >>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> > >>>>>>>>>>>>>      -> KBranchedStream
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // assign a name to the branch and
> > >>>>>>>>>>>>> // return the sub-stream to the current scope later
> > >>>>>>>>>>>>> //
> > >>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> > >>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> > >>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> > >>>>>>>>>>>>> String)
> > >>>>>>>>>>>>>      -> KBranchedStream
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // default branch is not easily accessible
> > >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> > >>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> > >>>>>>>>>>>>>      -> Map<String,KStream>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // assign custom name to default-branch
> > >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> > >>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> > >>>>>>>>>>>>>      -> Map<String,KStream>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // assign a default name for default
> > >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> > >>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> > >>>>>>>>>>>>>      -> Map<String,KStream>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> // return map of all names sub-stream into current scope
> > >>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> > >>>>>>>>>>>>>      -> Map<String,KStream>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> > >>>>>>>>>>>>> and
> > >>>>>>>>>>>>> return
> > >>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> > >>>>>>>>>>>>> implementation can
> > >>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> > >>>>>>>>>>>>> returned
> > >>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thoughts?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> > >>>>>>>>>>>>>> Ivan,
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> > >>>>>>>>>>>>>> dynamic case.
> > >>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> > >>>>>>>>>>>>>> necessity.
> > >>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> > >>>>>>>>>>>>>> of the
> > >>>>>>>>>>>>>> "split()" start operator. Thanks!
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Separately, I'm interested to see where the present
> > >>>>>>>>>>>>>> discussion
> > >>>>>>>>>>>>>> leads.
> > >>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> > >>>>>>>>>>>>>> suspicious of
> > >>>>>>>>>>>>>> nested closures. You have a good point about using method
> > >>>>>>>>>>>>>> references (or
> > >>>>>>>>>>>>>> indeed function literals also work). It should be validating
> > >>>>>>>>>>>>>> that this
> > >>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> > >>>>>>>>>>>>>> logic when
> > >>>>>>>>>>>>>> their nested closure situation got out of hand.
> > >>>>>>>>>>>>>> Unfortunately,
> > >>>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> > >>>>>>>>>>>>>> code
> > >>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> > >>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> > >>>>>>>>>>>>>> solution if
> > >>>>>>>>>>>>>> the nested code does indeed become a problem.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> However, the history of JS also tells us that function
> > >>>>>>>>>>>>>> references aren't
> > >>>>>>>>>>>>>> the end of the story either, and you can see that by
> > >>>>>>>>>>>>>> observing
> > >>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> > >>>>>>>>>>>>>> trying to
> > >>>>>>>>>>>>>> cope with
> > >>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> > >>>>>>>>>>>>>> First, you
> > >>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> > >>>>>>>>>>>>>> nested
> > >>>>>>>>>>>>>> code to
> > >>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> > >>>>>>>>>>>>>> this).
> > >>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> > >>>>>>>>>>>>>> apply
> > >>>>>>>>>>>>>> language
> > >>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> > >>>>>>>>>>>>>> "flattest"
> > >>>>>>>>>>>>>> possible
> > >>>>>>>>>>>>>> programming style to solve the problem (because you get
> > >>>>>>>>>>>>>> back to
> > >>>>>>>>>>>>>> just one
> > >>>>>>>>>>>>>> code block per functional unit).
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> > >>>>>>>>>>>>>> nowhere
> > >>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> > >>>>>>>>>>>>>> take
> > >>>>>>>>>>>>>> the JS
> > >>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> > >>>>>>>>>>>>>> valuable
> > >>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> > >>>>>>>>>>>>>> bringing this
> > >>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> > >>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>> like JS,
> > >>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> > >>>>>>>>>>>>>> (and
> > >>>>>>>>>>>>>> I'd also
> > >>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> > >>>>>>>>>>>>>> punt on
> > >>>>>>>>>>>>>> it, by
> > >>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> > >>>>>>>>>>>>>> there a DSL
> > >>>>>>>>>>>>>> way to do it?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks again for your driving this,
> > >>>>>>>>>>>>>> -John
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> > >>>>>>>>>>>>>> <pgwhalen@gmail.com
> > >>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        Ivan, I’ll definitely forfeit my point on the
> > >>>>>>>>>>>>>> clumsiness of
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        branch(predicate, consumer) solution, I don’t see
> > >>>>>>>>>>>>>> any real
> > >>>>>>>>>>>>>> drawbacks
> > >>>>>>>>>>>>>>        for the dynamic case.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        IMO the one trade off to consider at this point is the
> > >>>>>>>>>>>>>> scope
> > >>>>>>>>>>>>>>        question. I don’t know if I totally agree that “we
> > >>>>>>>>>>>>>> rarely
> > >>>>>>>>>>>>>> need them
> > >>>>>>>>>>>>>>        in the same scope” since merging the branches back
> > >>>>>>>>>>>>>> together
> > >>>>>>>>>>>>>> later
> > >>>>>>>>>>>>>>        seems like a perfectly plausible use case that can
> > >>>>>>>>>>>>>> be a lot
> > >>>>>>>>>>>>>> nicer
> > >>>>>>>>>>>>>>        when the branched streams are in the same scope.
> > >>>>>>>>>>>>>> That being
> > >>>>>>>>>>>>>> said,
> > >>>>>>>>>>>>>>        for the reasons Ivan listed, I think it is overall the
> > >>>>>>>>>>>>>> better
> > >>>>>>>>>>>>>>        solution - working around the scope thing is easy
> > >>>>>>>>>>>>>> enough if
> > >>>>>>>>>>>>>> you need
> > >>>>>>>>>>>>>>        to.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> > >>>>>>>>>>>>>>        <ip...@mail.ru.invalid> wrote:
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > Hello everyone, thank you all for joining the
> > >>>>>>>>>>>>>> discussion!
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > Well, I don't think the idea of named branches,
> > >>>>>>>>>>>>>> be it a
> > >>>>>>>>>>>>>>        LinkedHashMap (no other Map will do, because order of
> > >>>>>>>>>>>>>> definition
> > >>>>>>>>>>>>>>        matters) or `branch` method  taking name and Consumer
> > >>>>>>>>>>>>>> has more
> > >>>>>>>>>>>>>>        advantages than drawbacks.
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > In my opinion, the only real positive outcome from
> > >>>>>>>>>>>>>> Michael's
> > >>>>>>>>>>>>>>        proposal is that all the returned branches are in
> > >>>>>>>>>>>>>> the same
> > >>>>>>>>>>>>>> scope.
> > >>>>>>>>>>>>>>        But 1) we rarely need them in the same scope 2)
> > >>>>>>>>>>>>>> there is a
> > >>>>>>>>>>>>>>        workaround for the scope problem, described in the
> > >>>>>>>>>>>>>> KIP.
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > 'Inlining the complex logic' is not a problem,
> > >>>>>>>>>>>>>> because we
> > >>>>>>>>>>>>>> can use
> > >>>>>>>>>>>>>>        method references instead of lambdas. In real world
> > >>>>>>>>>>>>>> scenarios you
> > >>>>>>>>>>>>>>        tend to split the complex logic to methods anyway,
> > >>>>>>>>>>>>>> so the
> > >>>>>>>>>>>>>> code is
> > >>>>>>>>>>>>>>        going to be clean.
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > The drawbacks are strong. The cohesion between
> > >>>>>>>>>>>>>> predicates
> > >>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>        handlers is lost. We have to define predicates in one
> > >>>>>>>>>>>>>> place, and
> > >>>>>>>>>>>>>>        handlers in another. This opens the door for bugs:
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > - what if we forget to define a handler for a
> > >>>>>>>>>>>>>> name? or a
> > >>>>>>>>>>>>>> name for
> > >>>>>>>>>>>>>>        a handler?
> > >>>>>>>>>>>>>>        > - what if we misspell a name?
> > >>>>>>>>>>>>>>        > - what if we copy-paste and duplicate a name?
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > What Michael propose would have been totally OK
> > >>>>>>>>>>>>>> if we had
> > >>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>        writing the API in Lua, Ruby or Python. In those
> > >>>>>>>>>>>>>> languages the
> > >>>>>>>>>>>>>>        "dynamic naming" approach would have looked most
> > >>>>>>>>>>>>>> concise
> > >>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>        beautiful. But in Java we expect all the problems
> > >>>>>>>>>>>>>> related to
> > >>>>>>>>>>>>>>        identifiers to be eliminated in compile time.
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > Do we have to invent duck-typing for the Java API?
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > And if we do, what advantage are we supposed to get
> > >>>>>>>>>>>>>> besides having
> > >>>>>>>>>>>>>>        all the branches in the same scope? Michael, maybe I'm
> > >>>>>>>>>>>>>> missing your
> > >>>>>>>>>>>>>>        point?
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > ---
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > Earlier in this discussion John Roesler also
> > >>>>>>>>>>>>>> proposed
> > >>>>>>>>>>>>>> to do
> > >>>>>>>>>>>>>>        without "start branching" operator, and later Paul
> > >>>>>>>>>>>>>> mentioned that in
> > >>>>>>>>>>>>>>        the case when we have to add a dynamic number of
> > >>>>>>>>>>>>>> branches, the
> > >>>>>>>>>>>>>>        current KIP is 'clumsier' compared to Michael's 'Map'
> > >>>>>>>>>>>>>> solution. Let
> > >>>>>>>>>>>>>>        me address both comments here.
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > 1) "Start branching" operator (I think that
> > >>>>>>>>>>>>>> *split* is a
> > >>>>>>>>>>>>>> good name
> > >>>>>>>>>>>>>>        for it indeed) is critical when we need to do a
> > >>>>>>>>>>>>>> dynamic
> > >>>>>>>>>>>>>> branching,
> > >>>>>>>>>>>>>>        see example below.
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > 2) No, dynamic branching in current KIP is not
> > >>>>>>>>>>>>>> clumsy at
> > >>>>>>>>>>>>>> all.
> > >>>>>>>>>>>>>>        Imagine a real-world scenario when you need one
> > >>>>>>>>>>>>>> branch per
> > >>>>>>>>>>>>>> enum
> > >>>>>>>>>>>>>>        value (say, RecordType). You can have something
> > >>>>>>>>>>>>>> like this:
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > /*John:if we had to start with stream.branch(...)
> > >>>>>>>>>>>>>> here,
> > >>>>>>>>>>>>>> it would
> > >>>>>>>>>>>>>>        have been much messier.*/
> > >>>>>>>>>>>>>>        > KBranchedStream branched = stream.split();
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > /*Not clumsy at all :-)*/
> > >>>>>>>>>>>>>>        > for (RecordType recordType : RecordType.values())
> > >>>>>>>>>>>>>>        >             branched = branched.branch((k, v) ->
> > >>>>>>>>>>>>>> v.getRecType() ==
> > >>>>>>>>>>>>>>        recordType,
> > >>>>>>>>>>>>>>        >                     recordType::processRecords);
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > Regards,
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > Ivan
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>        > 02.05.2019 14:40, Matthias J. Sax пишет:
> > >>>>>>>>>>>>>>        >> I also agree with Michael's observation about
> > >>>>>>>>>>>>>> the core
> > >>>>>>>>>>>>>> problem of
> > >>>>>>>>>>>>>>        >> current `branch()` implementation.
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >> However, I also don't like to pass in a clumsy Map
> > >>>>>>>>>>>>>> object. My
> > >>>>>>>>>>>>>>        thinking
> > >>>>>>>>>>>>>>        >> was more aligned with Paul's proposal to just
> > >>>>>>>>>>>>>> add a name
> > >>>>>>>>>>>>>> to each
> > >>>>>>>>>>>>>>        >> `branch()` statement and return a
> > >>>>>>>>>>>>>> `Map<String,KStream>`.
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >> It makes the code easier to read, and also make the
> > >>>>>>>>>>>>>> order of
> > >>>>>>>>>>>>>>        >> `Predicates` (that is essential) easier to grasp.
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >>>>>> Map<String, KStream<K, V>> branches =
> > >>>>>>>>>>>>>> stream.split()
> > >>>>>>>>>>>>>>        >>>>>>    .branch("branchOne", Predicate<K, V>)
> > >>>>>>>>>>>>>>        >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> > >>>>>>>>>>>>>>        >>>>>>    .defaultBranch("defaultBranch");
> > >>>>>>>>>>>>>>        >> An open question is the case for which no
> > >>>>>>>>>>>>>> defaultBranch() should
> > >>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>        >> specified. Atm, `split()` and `branch()` would
> > >>>>>>>>>>>>>> return
> > >>>>>>>>>>>>>>        `BranchedKStream`
> > >>>>>>>>>>>>>>        >> and the call to `defaultBranch()` that returns the
> > >>>>>>>>>>>>>> `Map` is
> > >>>>>>>>>>>>> mandatory
> > >>>>>>>>>>>>>>        >> (what is not the case atm). Or is this actually
> > >>>>>>>>>>>>>> not a
> > >>>>>>>>>>>>>> real
> > >>>>>>>>>>>>> problem,
> > >>>>>>>>>>>>>>        >> because users can just ignore the branch
> > >>>>>>>>>>>>>> returned by
> > >>>>>>>>>>>>>>        `defaultBranch()`
> > >>>>>>>>>>>>>>        >> in the result `Map` ?
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >> About "inlining": So far, it seems to be a
> > >>>>>>>>>>>>>> matter of
> > >>>>>>>>>>>>>> personal
> > >>>>>>>>>>>>>>        >> preference. I can see arguments for both, but no
> > >>>>>>>>>>>>>> "killer
> > >>>>>>>>>>>>>>        argument" yet
> > >>>>>>>>>>>>>>        >> that clearly make the case for one or the other.
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >> -Matthias
> > >>>>>>>>>>>>>>        >>
> > >>>>>>>>>>>>>>        >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> > >>>>>>>>>>>>>>        >>> Perhaps inlining is the wrong terminology. It
> > >>>>>>>>>>>>>> doesn’t
> > >>>>>>>>>>>>>> require
> > >>>>>>>>>>>>>>        that a lambda with the full downstream topology be
> > >>>>>>>>>>>>>> defined
> > >>>>>>>>>>>>>> inline -
> > >>>>>>>>>>>>>>        it can be a method reference as with Ivan’s original
> > >>>>>>>>>>>>>> suggestion.
> > >>>>>>>>>>>>>>        The advantage of putting the predicate and its
> > >>>>>>>>>>>>>> downstream
> > >>>>>>>>>>>>>> logic
> > >>>>>>>>>>>>>>        (Consumer) together in branch() is that they are
> > >>>>>>>>>>>>>> required
> > >>>>>>>>>>>>>> to be near
> > >>>>>>>>>>>>>>        to each other.
> > >>>>>>>>>>>>>>        >>>
> > >>>>>>>>>>>>>>        >>> Ultimately the downstream code has to live
> > >>>>>>>>>>>>>> somewhere,
> > >>>>>>>>>>>>>> and deep
> > >>>>>>>>>>>>>>        branch trees will be hard to read regardless.
> > >>>>>>>>>>>>>>        >>>
> > >>>>>>>>>>>>>>        >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> > >>>>>>>>>>>>>>        <michael.drogalis@confluent.io
> > >>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
> > >>>>>>>>>>>>>>        >>>>
> > >>>>>>>>>>>>>>        >>>> I'm less enthusiastic about inlining the
> > >>>>>>>>>>>>>> branch logic
> > >>>>>>>>>>>>>> with its
> > >>>>>>>>>>>>>>        downstream
> > >>>>>>>>>>>>>>        >>>> functionality. Programs that have deep branch
> > >>>>>>>>>>>>>> trees
> > >>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>        quickly become
> > >>>>>>>>>>>>>>        >>>> harder to read as a single unit.
> > >>>>>>>>>>>>>>        >>>>
> > >>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> > >>>>>>>>>>>>>>        <pgwhalen@gmail.com <ma...@gmail.com>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Also +1 on the issues/goals as Michael
> > >>>>>>>>>>>>>> outlined them,
> > >>>>>>>>>>>>>> I think
> > >>>>>>>>>>>>>>        that sets a
> > >>>>>>>>>>>>>>        >>>>> great framework for the discussion.
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Regarding the SortedMap solution, my
> > >>>>>>>>>>>>>> understanding is
> > >>>>>>>>>>>>>> that the
> > >>>>>>>>>>>>>>        current
> > >>>>>>>>>>>>>>        >>>>> proposal in the KIP is what is in my PR which
> > >>>>>>>>>>>>>> (pending naming
> > >>>>>>>>>>>>>>        decisions) is
> > >>>>>>>>>>>>>>        >>>>> roughly this:
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> stream.split()
> > >>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>,
> > >>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> > >>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>,
> > >>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> > >>>>>>>>>>>>>>        >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Obviously some ordering is necessary, since
> > >>>>>>>>>>>>>> branching
> > >>>>>>>>>>>>>> as a
> > >>>>>>>>>>>>>>        construct
> > >>>>>>>>>>>>>>        >>>>> doesn't work without it, but this solution seems
> > >>>>>>>>>>>>>> like it
> > >>>>>>>>>>>>>>        provides as much
> > >>>>>>>>>>>>>>        >>>>> associativity as the SortedMap solution,
> > >>>>>>>>>>>>>> because each
> > >>>>>>>>>>>>>> branch()
> > >>>>>>>>>>>>>>        call
> > >>>>>>>>>>>>>>        >>>>> directly associates the "conditional" with
> > >>>>>>>>>>>>>> the "code
> > >>>>>>>>>>>>>> block."
> > >>>>>>>>>>>>>>        The value it
> > >>>>>>>>>>>>>>        >>>>> provides over the KIP solution is the
> > >>>>>>>>>>>>>> accessing of
> > >>>>>>>>>>>>>> streams in
> > >>>>>>>>>>>>>>        the same
> > >>>>>>>>>>>>>>        >>>>> scope.
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> The KIP solution is less "dynamic" than the
> > >>>>>>>>>>>>>> SortedMap
> > >>>>>>>>>>>>>> solution
> > >>>>>>>>>>>>>>        in the sense
> > >>>>>>>>>>>>>>        >>>>> that it is slightly clumsier to add a dynamic
> > >>>>>>>>>>>>>> number of
> > >>>>>>>>>>>>>>        branches, but it is
> > >>>>>>>>>>>>>>        >>>>> certainly possible.  It seems to me like the API
> > >>>>>>>>>>>>>> should favor
> > >>>>>>>>>>>>>>        the "static"
> > >>>>>>>>>>>>>>        >>>>> case anyway, and should make it simple and
> > >>>>>>>>>>>>>> readable to
> > >>>>>>>>>>>>>>        fluently declare and
> > >>>>>>>>>>>>>>        >>>>> access your branches in-line.  It also makes it
> > >>>>>>>>>>>>>> impossible to
> > >>>>>>>>>>>>>>        ignore a
> > >>>>>>>>>>>>>>        >>>>> branch, and it is possible to build an (almost)
> > >>>>>>>>>>>>>> identical
> > >>>>>>>>>>>>>>        SortedMap
> > >>>>>>>>>>>>>>        >>>>> solution on top of it.
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> I could also see a middle ground where
> > >>>>>>>>>>>>>> instead of
> > >>>>>>>>>>>>>> a raw
> > >>>>>>>>>>>>>>        SortedMap being
> > >>>>>>>>>>>>>>        >>>>> taken in, branch() takes a name and not a
> > >>>>>>>>>>>>>> Consumer.
> > >>>>>>>>>>>>>> Something
> > >>>>>>>>>>>>>>        like this:
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Map<String, KStream<K, V>> branches =
> > >>>>>>>>>>>>>> stream.split()
> > >>>>>>>>>>>>>>        >>>>>    .branch("branchOne", Predicate<K, V>)
> > >>>>>>>>>>>>>>        >>>>>    .branch( "branchTwo", Predicate<K, V>)
> > >>>>>>>>>>>>>>        >>>>>    .defaultBranch("defaultBranch",
> > >>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Pros for that solution:
> > >>>>>>>>>>>>>>        >>>>> - accessing branched KStreams in same scope
> > >>>>>>>>>>>>>>        >>>>> - no double brace initialization, hopefully
> > >>>>>>>>>>>>>> slightly
> > >>>>>>>>>>>>>> more
> > >>>>>>>>>>>>>>        readable than
> > >>>>>>>>>>>>>>        >>>>> SortedMap
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Cons
> > >>>>>>>>>>>>>>        >>>>> - downstream branch logic cannot be specified
> > >>>>>>>>>>>>>> inline
> > >>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>        makes it harder
> > >>>>>>>>>>>>>>        >>>>> to read top to bottom (like existing API and
> > >>>>>>>>>>>>>> SortedMap, but
> > >>>>>>>>>>>>>>        unlike the KIP)
> > >>>>>>>>>>>>>>        >>>>> - you can forget to "handle" one of the branched
> > >>>>>>>>>>>>>> streams (like
> > >>>>>>>>>>>>>>        existing
> > >>>>>>>>>>>>>>        >>>>> API and SortedMap, but unlike the KIP)
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> (KBranchedStreams could even work *both* ways
> > >>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>> perhaps
> > >>>>>>>>>>>>>>        that's overdoing
> > >>>>>>>>>>>>>>        >>>>> it).
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Overall I'm curious how important it is to be
> > >>>>>>>>>>>>>> able to
> > >>>>>>>>>>>>>> easily
> > >>>>>>>>>>>>>>        access the
> > >>>>>>>>>>>>>>        >>>>> branched KStream in the same scope as the
> > >>>>>>>>>>>>>> original.
> > >>>>>>>>>>>>>> It's
> > >>>>>>>>>>>>>>        possible that it
> > >>>>>>>>>>>>>>        >>>>> doesn't need to be handled directly by the
> > >>>>>>>>>>>>>> API, but
> > >>>>>>>>>>>>>> instead
> > >>>>>>>>>>>>>>        left up to the
> > >>>>>>>>>>>>>>        >>>>> user.  I'm sort of in the middle on it.
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> Paul
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> > >>>>>>>>>>>>>> Blee-Goldman
> > >>>>>>>>>>>>>>        <sophie@confluent.io <ma...@confluent.io>>
> > >>>>>>>>>>>>>>        >>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>        >>>>>> I'd like to +1 what Michael said about the
> > >>>>>>>>>>>>>> issues
> > >>>>>>>>>>>>>> with the
> > >>>>>>>>>>>>>>        existing
> > >>>>>>>>>>>>>>        >>>>> branch
> > >>>>>>>>>>>>>>        >>>>>> method, I agree with what he's outlined and
> > >>>>>>>>>>>>>> I think
> > >>>>>>>>>>>>>> we should
> > >>>>>>>>>>>>>>        proceed by
> > >>>>>>>>>>>>>>        >>>>>> trying to alleviate these problems.
> > >>>>>>>>>>>>>> Specifically it
> > >>>>>>>>>>>>>> seems
> > >>>>>>>>>>>>>>        important to be
> > >>>>>>>>>>>>>>        >>>>>> able to cleanly access the individual
> > >>>>>>>>>>>>>> branches (eg
> > >>>>>>>>>>>>>> by mapping
> > >>>>>>>>>>>>>>        >>>>>> name->stream), which I thought was the original
> > >>>>>>>>>>>>>> intention of
> > >>>>>>>>>>>>>>        this KIP.
> > >>>>>>>>>>>>>>        >>>>>>
> > >>>>>>>>>>>>>>        >>>>>> That said, I don't think we should so easily
> > >>>>>>>>>>>>>> give in
> > >>>>>>>>>>>>>> to the
> > >>>>>>>>>>>>>>        double brace
> > >>>>>>>>>>>>>>        >>>>>> anti-pattern or force ours users into it if
> > >>>>>>>>>>>>>> at all
> > >>>>>>>>>>>>>> possible to
> > >>>>>>>>>>>>>>        >>>>> avoid...just
> > >>>>>>>>>>>>>>        >>>>>> my two cents.
> > >>>>>>>>>>>>>>        >>>>>>
> > >>>>>>>>>>>>>>        >>>>>> Cheers,
> > >>>>>>>>>>>>>>        >>>>>> Sophie
> > >>>>>>>>>>>>>>        >>>>>>
> > >>>>>>>>>>>>>>        >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> > >>>>>>>>>>>>>> Drogalis <
> > >>>>>>>>>>>>>>        >>>>>> michael.drogalis@confluent.io
> > >>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>
> > >>>>>>>>>>>>>>        >>>>>>> I’d like to propose a different way of
> > >>>>>>>>>>>>>> thinking
> > >>>>>>>>>>>>>> about this.
> > >>>>>>>>>>>>>>        To me,
> > >>>>>>>>>>>>>>        >>>>> there
> > >>>>>>>>>>>>>>        >>>>>>> are three problems with the existing branch
> > >>>>>>>>>>>>>> signature:
> > >>>>>>>>>>>>>>        >>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>> 1. If you use it the way most people do, Java
> > >>>>>>>>>>>>>> raises unsafe
> > >>>>>>>>>>>>> type
> > >>>>>>>>>>>>>>        >>>>>> warnings.
> > >>>>>>>>>>>>>>        >>>>>>> 2. The way in which you use the stream
> > >>>>>>>>>>>>>> branches is
> > >>>>>>>>>>>>>>        positionally coupled
> > >>>>>>>>>>>>>>        >>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>> the ordering of the conditionals.
> > >>>>>>>>>>>>>>        >>>>>>> 3. It is brittle to extend existing branch
> > >>>>>>>>>>>>>> calls
> > >>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>        additional code
> > >>>>>>>>>>>>>>        >>>>>>> paths.
> > >>>>>>>>>>>>>>        >>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>> Using associative constructs instead of
> > >>>>>>>>>>>>>> relying on
> > >>>>>>>>>>>>>> ordered
> > >>>>>>>>>>>>>>        constructs
> > >>>>>>>>>>>>>>        >>>>>> would
> > >>>>>>>>>>>>>>        >>>>>>> be a stronger approach. Consider a
> > >>>>>>>>>>>>>> signature that
> > >>>>>>>>>>>>>> instead
> > >>>>>>>>>>>>>>        looks like
> > >>>>>>>>>>>>>>        >>>>>> this:
> > >>>>>>>>>>>>>>        >>>>>>> Map<String, KStream<K,V>>
> > >>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> > >>>>>>>>>>>>>>        Predicate<?
> > >>>>>>>>>>>>>>        >>>>>>> super K,? super V>>);
> > >>>>>>>>>>>>>>        >>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>> Branches are given names in a map, and as a
> > >>>>>>>>>>>>>> result,
> > >>>>>>>>>>>>>> the API
> > >>>>>>>>>>>>>>        returns a
> > >>>>>>>>>>>>>>        >>>>>>> mapping of names to streams. The ordering
> > >>>>>>>>>>>>>> of the
> > >>>>>>>>>>>>> conditionals is
> > >>>>>>>>>>>>>>        >>>>>> maintained
> > >>>>>>>>>>>>>>        >>>>>>> because it’s a sorted map. Insert order
> > >>>>>>>>>>>>>> determines
> > >>>>>>>>>>>>>> the order
> > >>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>        >>>>>> evaluation.
> > >>>>>>>>>>>>>>        >>>>>>> This solves problem 1 because there are no
> > >>>>>>>>>>>>>> more
> > >>>>>>>>>>>>>> varargs. It
> > >>>>>>>>>>>>>>        solves
> > >>>>>>>>>>>>>>        >>>>>> problem
> > >>>>>>>>>>>>>>        >>>>>>> 2 because you no longer lean on ordering to
> > >>>>>>>>>>>>>> access the
> > >>>>>>>>>>>>>>        branch you’re
> > >>>>>>>>>>>>>>        >>>>>>> interested in. It solves problem 3 because
> > >>>>>>>>>>>>>> you can
> > >>>>>>>>>>>>>> introduce
> > >>>>>>>>>>>>>>        another
> > >>>>>>>>>>>>>>        >>>>>>> conditional by simply attaching another
> > >>>>>>>>>>>>>> name to the
> > >>>>>>>>>>>>>>        structure, rather
> > >>>>>>>>>>>>>>        >>>>>> than
> > >>>>>>>>>>>>>>        >>>>>>> messing with the existing indices.
> > >>>>>>>>>>>>>>        >>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>> One of the drawbacks is that creating the map
> > >>>>>>>>>>>>>> inline is
> > >>>>>>>>>>>>>>        historically
> > >>>>>>>>>>>>>>        >>>>>>> awkward in Java. I know it’s an
> > >>>>>>>>>>>>>> anti-pattern to use
> > >>>>>>>>>>>>>>        voluminously, but
> > >>>>>>>>>>>>>>        >>>>>>> double brace initialization would clean up the
> > >>>>>>>>>>>>>> aesthetics.
> > >>>>>>>>>>>>>>        >>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> > >>>>>>>>>>>>>>        <john@confluent.io <ma...@confluent.io>>
> > >>>>>>>>>>>>>>        >>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>> Hi Ivan,
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>> Thanks for the update.
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>> FWIW, I agree with Matthias that the current
> > >>>>>>>>>>>>>> "start
> > >>>>>>>>>>>>> branching"
> > >>>>>>>>>>>>>>        >>>>> operator
> > >>>>>>>>>>>>>>        >>>>>>> is
> > >>>>>>>>>>>>>>        >>>>>>>> confusing when named the same way as the
> > >>>>>>>>>>>>>> actual
> > >>>>>>>>>>>>>> branches.
> > >>>>>>>>>>>>>>        "Split"
> > >>>>>>>>>>>>>>        >>>>> seems
> > >>>>>>>>>>>>>>        >>>>>>>> like a good name. Alternatively, we can do
> > >>>>>>>>>>>>>> without
> > >>>>>>>>>>>>>> a "start
> > >>>>>>>>>>>>>>        >>>>> branching"
> > >>>>>>>>>>>>>>        >>>>>>>> operator at all, and just do:
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>> stream
> > >>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
> > >>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
> > >>>>>>>>>>>>>>        >>>>>>>>      .defaultBranch();
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>> Tentatively, I think that this branching
> > >>>>>>>>>>>>>> operation
> > >>>>>>>>>>>>>> should be
> > >>>>>>>>>>>>>>        >>>>> terminal.
> > >>>>>>>>>>>>>>        >>>>>>> That
> > >>>>>>>>>>>>>>        >>>>>>>> way, we don't create ambiguity about how
> > >>>>>>>>>>>>>> to use
> > >>>>>>>>>>>>>> it. That
> > >>>>>>>>>>>>>>        is, `branch`
> > >>>>>>>>>>>>>>        >>>>>>>> should return `KBranchedStream`, while
> > >>>>>>>>>>>>>> `defaultBranch` is
> > >>>>>>>>>>>>>>        `void`, to
> > >>>>>>>>>>>>>>        >>>>>>>> enforce that it comes last, and that there
> > >>>>>>>>>>>>>> is only
> > >>>>>>>>>>>>>> one
> > >>>>>>>>>>>>>>        definition of
> > >>>>>>>>>>>>>>        >>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>> default branch. Potentially, we should log a
> > >>>>>>>>>>>>>> warning if
> > >>>>>>>>>>>>>>        there's no
> > >>>>>>>>>>>>>>        >>>>>>> default,
> > >>>>>>>>>>>>>>        >>>>>>>> and additionally log a warning (or throw an
> > >>>>>>>>>>>>>> exception) if a
> > >>>>>>>>>>>>>>        record
> > >>>>>>>>>>>>>>        >>>>>> falls
> > >>>>>>>>>>>>>>        >>>>>>>> though with no default.
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>> Thoughts?
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>> Thanks,
> > >>>>>>>>>>>>>>        >>>>>>>> -John
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> > >>>>>>>>>>>>>> J. Sax <
> > >>>>>>>>>>>>>>        >>>>> matthias@confluent.io
> > >>>>>>>>>>>>>> <ma...@confluent.io>
> > >>>>>>>>>>>>>>        >>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> Thanks for updating the KIP and your
> > >>>>>>>>>>>>>> answers.
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> this is to make the name similar to
> > >>>>>>>>>>>>>> String#split
> > >>>>>>>>>>>>>>        >>>>>>>>>>> that also returns an array, right?
> > >>>>>>>>>>>>>>        >>>>>>>>> The intend was to avoid name duplication.
> > >>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>> return type
> > >>>>>>>>>>>>>>        should
> > >>>>>>>>>>>>>>        >>>>>> _not_
> > >>>>>>>>>>>>>>        >>>>>>>>> be an array.
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> The current proposal is
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> stream.branch()
> > >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> > >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> > >>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> IMHO, this reads a little odd, because
> > >>>>>>>>>>>>>> the first
> > >>>>>>>>>>>>>>        `branch()` does
> > >>>>>>>>>>>>>>        >>>>> not
> > >>>>>>>>>>>>>>        >>>>>>>>> take any parameters and has different
> > >>>>>>>>>>>>>> semantics
> > >>>>>>>>>>>>>> than the
> > >>>>>>>>>>>>> later
> > >>>>>>>>>>>>>>        >>>>>>>>> `branch()` calls. Note, that from the code
> > >>>>>>>>>>>>>> snippet above,
> > >>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>        >>>>> hidden
> > >>>>>>>>>>>>>>        >>>>>>>>> that the first call is `KStream#branch()`
> > >>>>>>>>>>>>>> while
> > >>>>>>>>>>>>>> the others
> > >>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>        >>>>>>>>> `KBranchedStream#branch()` what makes
> > >>>>>>>>>>>>>> reading the
> > >>>>>>>>>>>>>> code
> > >>>>>>>>>>>>> harder.
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> Because I suggested to rename
> > >>>>>>>>>>>>>> `addBranch()` ->
> > >>>>>>>>>>>>>> `branch()`,
> > >>>>>>>>>>>>>>        I though
> > >>>>>>>>>>>>>>        >>>>>> it
> > >>>>>>>>>>>>>>        >>>>>>>>> might be better to also rename
> > >>>>>>>>>>>>>> `KStream#branch()`
> > >>>>>>>>>>>>>> to avoid
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>> naming
> > >>>>>>>>>>>>>>        >>>>>>>>> overlap that seems to be confusing. The
> > >>>>>>>>>>>>>> following
> > >>>>>>>>>>>>>> reads
> > >>>>>>>>>>>>> much
> > >>>>>>>>>>>>>>        >>>>> cleaner
> > >>>>>>>>>>>>>>        >>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>> me:
> > >>>>>>>>>>>>>>        >>>>>>>>> stream.split()
> > >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> > >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> > >>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> Maybe there is a better alternative to
> > >>>>>>>>>>>>>> `split()`
> > >>>>>>>>>>>>>> though to
> > >>>>>>>>>>>>>>        avoid
> > >>>>>>>>>>>>>>        >>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> naming overlap.
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> 'default' is, however, a reserved word, so
> > >>>>>>>>>>>>>> unfortunately
> > >>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>        >>>>> cannot
> > >>>>>>>>>>>>>>        >>>>>>> have
> > >>>>>>>>>>>>>>        >>>>>>>>> a method with such name :-)
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> Bummer. Didn't consider this. Maybe we
> > >>>>>>>>>>>>>> can still
> > >>>>>>>>>>>>>> come up
> > >>>>>>>>>>>>>>        with a
> > >>>>>>>>>>>>>>        >>>>> short
> > >>>>>>>>>>>>>>        >>>>>>>> name?
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> Can you add the interface
> > >>>>>>>>>>>>>> `KBranchedStream` to
> > >>>>>>>>>>>>>> the KIP
> > >>>>>>>>>>>>>>        with all
> > >>>>>>>>>>>>>>        >>>>> it's
> > >>>>>>>>>>>>>>        >>>>>>>>> methods? It will be part of public API and
> > >>>>>>>>>>>>>> should be
> > >>>>>>>>>>>>>>        contained in
> > >>>>>>>>>>>>>>        >>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> KIP. For example, it's unclear atm, what the
> > >>>>>>>>>>>>>> return type of
> > >>>>>>>>>>>>>>        >>>>>>>>> `defaultBranch()` is.
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> You did not comment on the idea to add a
> > >>>>>>>>>>>>>>        `KBranchedStream#get(int
> > >>>>>>>>>>>>>>        >>>>>>> index)
> > >>>>>>>>>>>>>>        >>>>>>>>> -> KStream` method to get the individually
> > >>>>>>>>>>>>>>        branched-KStreams. Would
> > >>>>>>>>>>>>>>        >>>>>> be
> > >>>>>>>>>>>>>>        >>>>>>>>> nice to get your feedback about it. It
> > >>>>>>>>>>>>>> seems you
> > >>>>>>>>>>>>>> suggest
> > >>>>>>>>>>>>>>        that users
> > >>>>>>>>>>>>>>        >>>>>>>>> would need to write custom utility code
> > >>>>>>>>>>>>>> otherwise, to
> > >>>>>>>>>>>>>>        access them.
> > >>>>>>>>>>>>>>        >>>>> We
> > >>>>>>>>>>>>>>        >>>>>>>>> should discuss the pros and cons of both
> > >>>>>>>>>>>>>> approaches. It
> > >>>>>>>>>>>>> feels
> > >>>>>>>>>>>>>>        >>>>>>>>> "incomplete" to me atm, if the API has no
> > >>>>>>>>>>>>>> built-in support
> > >>>>>>>>>>>>>>        to get
> > >>>>>>>>>>>>>>        >>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> branched-KStreams directly.
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>> Hi all!
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> I have updated the KIP-418 according to
> > >>>>>>>>>>>>>> the new
> > >>>>>>>>>>>>>> vision.
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> Matthias, thanks for your comment!
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KStream#branch() -> #split()
> > >>>>>>>>>>>>>>        >>>>>>>>>> I can see your point: this is to make
> > >>>>>>>>>>>>>> the name
> > >>>>>>>>>>>>>> similar to
> > >>>>>>>>>>>>>>        >>>>>>> String#split
> > >>>>>>>>>>>>>>        >>>>>>>>>> that also returns an array, right? But
> > >>>>>>>>>>>>>> is it
> > >>>>>>>>>>>>>> worth the
> > >>>>>>>>>>>>>>        loss of
> > >>>>>>>>>>>>>>        >>>>>>>> backwards
> > >>>>>>>>>>>>>>        >>>>>>>>>> compatibility? We can have overloaded
> > >>>>>>>>>>>>>> branch()
> > >>>>>>>>>>>>>> as well
> > >>>>>>>>>>>>>>        without
> > >>>>>>>>>>>>>>        >>>>>>>> affecting
> > >>>>>>>>>>>>>>        >>>>>>>>>> the existing code. Maybe the old
> > >>>>>>>>>>>>>> array-based
> > >>>>>>>>>>>>>> `branch`
> > >>>>>>>>>>>>> method
> > >>>>>>>>>>>>>>        >>>>> should
> > >>>>>>>>>>>>>>        >>>>>>> be
> > >>>>>>>>>>>>>>        >>>>>>>>>> deprecated, but this is a subject for
> > >>>>>>>>>>>>>> discussion.
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> > >>>>>>>>>>>>>>        >>>>> BranchingKStream#branch(),
> > >>>>>>>>>>>>>>        >>>>>>>>>> KBranchedStream#defaultBranch() ->
> > >>>>>>>>>>>>> BranchingKStream#default()
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> Totally agree with 'addBranch->branch'
> > >>>>>>>>>>>>>> rename.
> > >>>>>>>>>>>>>> 'default'
> > >>>>>>>>>>>>> is,
> > >>>>>>>>>>>>>>        >>>>>>> however, a
> > >>>>>>>>>>>>>>        >>>>>>>>>> reserved word, so unfortunately we
> > >>>>>>>>>>>>>> cannot have a
> > >>>>>>>>>>>>>> method
> > >>>>>>>>>>>>>>        with such
> > >>>>>>>>>>>>>>        >>>>>>> name
> > >>>>>>>>>>>>>>        >>>>>>>>> :-)
> > >>>>>>>>>>>>>>        >>>>>>>>>>> defaultBranch() does take an
> > >>>>>>>>>>>>>> `Predicate` as
> > >>>>>>>>>>>>>> argument,
> > >>>>>>>>>>>>> but I
> > >>>>>>>>>>>>>>        >>>>> think
> > >>>>>>>>>>>>>>        >>>>>>> that
> > >>>>>>>>>>>>>>        >>>>>>>>>> is not required?
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> Absolutely! I think that was just
> > >>>>>>>>>>>>>> copy-paste
> > >>>>>>>>>>>>>> error or
> > >>>>>>>>>>>>>>        something.
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> Dear colleagues,
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> please revise the new version of the KIP
> > >>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>> Paul's PR
> > >>>>>>>>>>>>>>        >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> Any new suggestions/objections?
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>> Thanks for driving the discussion of
> > >>>>>>>>>>>>>> this KIP.
> > >>>>>>>>>>>>>> It seems
> > >>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>        >>>>>>> everybody
> > >>>>>>>>>>>>>>        >>>>>>>>>>> agrees that the current branch() method
> > >>>>>>>>>>>>>> using
> > >>>>>>>>>>>>>> arrays is
> > >>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>        >>>>>> optimal.
> > >>>>>>>>>>>>>>        >>>>>>>>>>> I had a quick look into the PR and I
> > >>>>>>>>>>>>>> like the
> > >>>>>>>>>>>>>> overall
> > >>>>>>>>>>>>>>        proposal.
> > >>>>>>>>>>>>>>        >>>>>>> There
> > >>>>>>>>>>>>>>        >>>>>>>>>>> are some minor things we need to
> > >>>>>>>>>>>>>> consider. I
> > >>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>        recommend the
> > >>>>>>>>>>>>>>        >>>>>>>>>>> following renaming:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> KStream#branch() -> #split()
> > >>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#addBranch() ->
> > >>>>>>>>>>>>>> BranchingKStream#branch()
> > >>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> > >>>>>>>>>>>>>>        BranchingKStream#default()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> It's just a suggestion to get slightly
> > >>>>>>>>>>>>>> shorter
> > >>>>>>>>>>>>>> method
> > >>>>>>>>>>>>> names.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> In the current PR, defaultBranch() does
> > >>>>>>>>>>>>>> take an
> > >>>>>>>>>>>>>>        `Predicate` as
> > >>>>>>>>>>>>>>        >>>>>>>> argument,
> > >>>>>>>>>>>>>>        >>>>>>>>>>> but I think that is not required?
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> Also, we should consider KIP-307, that was
> > >>>>>>>>>>>>>> recently
> > >>>>>>>>>>>>>>        accepted and
> > >>>>>>>>>>>>>>        >>>>>> is
> > >>>>>>>>>>>>>>        >>>>>>>>>>> currently implemented:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> Ie, we should add overloads that
> > >>>>>>>>>>>>>> accepted a
> > >>>>>>>>>>>>>> `Named`
> > >>>>>>>>>>>>>>        parameter.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> For the issue that the created
> > >>>>>>>>>>>>>> `KStream` object
> > >>>>>>>>>>>>>> are in
> > >>>>>>>>>>>>>>        different
> > >>>>>>>>>>>>>>        >>>>>>>> scopes:
> > >>>>>>>>>>>>>>        >>>>>>>>>>> could we extend `KBranchedStream` with a
> > >>>>>>>>>>>>>> `get(int
> > >>>>>>>>>>>>>>        index)` method
> > >>>>>>>>>>>>>>        >>>>>>> that
> > >>>>>>>>>>>>>>        >>>>>>>>>>> returns the corresponding "branched"
> > >>>>>>>>>>>>>> result
> > >>>>>>>>>>>>>> `KStream`
> > >>>>>>>>>>>>>>        object?
> > >>>>>>>>>>>>>>        >>>>>> Maybe,
> > >>>>>>>>>>>>>>        >>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>> second argument of `addBranch()` should
> > >>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>> be a
> > >>>>>>>>>>>>>>        >>>>>> `Consumer<KStream>`
> > >>>>>>>>>>>>>>        >>>>>>>> but
> > >>>>>>>>>>>>>>        >>>>>>>>>>> a `Function<KStream,KStream>` and
> > >>>>>>>>>>>>>> `get()` could
> > >>>>>>>>>>>>>> return
> > >>>>>>>>>>>>>>        whatever
> > >>>>>>>>>>>>>>        >>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>> `Function` returns?
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> Finally, I would also suggest to update
> > >>>>>>>>>>>>>> the KIP
> > >>>>>>>>>>>>>> with the
> > >>>>>>>>>>>>>>        current
> > >>>>>>>>>>>>>>        >>>>>>>>>>> proposal. That makes it easier to review.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> Ivan,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> > >>>>>>>>>>>>>> think it
> > >>>>>>>>>>>>>>        makes sense
> > >>>>>>>>>>>>>>        >>>>>> for
> > >>>>>>>>>>>>>>        >>>>>>>> you
> > >>>>>>>>>>>>>>        >>>>>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> revise the KIP and continue the
> > >>>>>>>>>>>>>> discussion.
> > >>>>>>>>>>>>>> Obviously
> > >>>>>>>>>>>>>>        we'll
> > >>>>>>>>>>>>>>        >>>>> need
> > >>>>>>>>>>>>>>        >>>>>>>> some
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> buy-in from committers that have actual
> > >>>>>>>>>>>>>> binding votes on
> > >>>>>>>>>>>>>>        >>>>> whether
> > >>>>>>>>>>>>>>        >>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> KIP
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> could be adopted.  It would be great
> > >>>>>>>>>>>>>> to hear
> > >>>>>>>>>>>>>> if they
> > >>>>>>>>>>>>>>        think this
> > >>>>>>>>>>>>>>        >>>>>> is
> > >>>>>>>>>>>>>>        >>>>>>> a
> > >>>>>>>>>>>>>>        >>>>>>>>> good
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> idea overall.  I'm not sure if that
> > >>>>>>>>>>>>>> happens
> > >>>>>>>>>>>>>> just by
> > >>>>>>>>>>>>>>        starting a
> > >>>>>>>>>>>>>>        >>>>>>> vote,
> > >>>>>>>>>>>>>>        >>>>>>>>> or if
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> there is generally some indication of
> > >>>>>>>>>>>>>> interest
> > >>>>>>>>>>>>> beforehand.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> That being said, I'll continue the
> > >>>>>>>>>>>>>> discussion
> > >>>>>>>>>>>>>> a bit:
> > >>>>>>>>>>>>>>        assuming
> > >>>>>>>>>>>>>>        >>>>> we
> > >>>>>>>>>>>>>>        >>>>>> do
> > >>>>>>>>>>>>>>        >>>>>>>>> move
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> forward the solution of "stream.branch()
> > >>>>>>>>>>>>>> returns
> > >>>>>>>>>>>>>>        >>>>>> KBranchedStream",
> > >>>>>>>>>>>>>>        >>>>>>> do
> > >>>>>>>>>>>>>>        >>>>>>>>> we
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> deprecate "stream.branch(...) returns
> > >>>>>>>>>>>>>> KStream[]"?  I
> > >>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>        >>>>> favor
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> deprecating, since having two mutually
> > >>>>>>>>>>>>>> exclusive APIs
> > >>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>        >>>>>>> accomplish
> > >>>>>>>>>>>>>>        >>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> same thing is confusing, especially when
> > >>>>>>>>>>>>>> they're fairly
> > >>>>>>>>>>>>>>        similar
> > >>>>>>>>>>>>>>        >>>>>>>>> anyway.  We
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> just need to be sure we're not making
> > >>>>>>>>>>>>>> something
> > >>>>>>>>>>>>>>        >>>>>>> impossible/difficult
> > >>>>>>>>>>>>>>        >>>>>>>>> that
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> is currently possible/easy.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> Regarding my PR - I think the general
> > >>>>>>>>>>>>>> structure would
> > >>>>>>>>>>>>> work,
> > >>>>>>>>>>>>>>        >>>>> it's
> > >>>>>>>>>>>>>>        >>>>>>>> just a
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> little sloppy overall in terms of
> > >>>>>>>>>>>>>> naming and
> > >>>>>>>>>>>>>> clarity. In
> > >>>>>>>>>>>>>>        >>>>>>> particular,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> passing in the "predicates" and
> > >>>>>>>>>>>>>> "children"
> > >>>>>>>>>>>>>> lists which
> > >>>>>>>>>>>>> get
> > >>>>>>>>>>>>>>        >>>>>> modified
> > >>>>>>>>>>>>>>        >>>>>>>> in
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> KBranchedStream but read from all the way
> > >>>>>>>>>>>>>>        KStreamLazyBranch is
> > >>>>>>>>>>>>>>        >>>>> a
> > >>>>>>>>>>>>>>        >>>>>>> bit
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> complicated to follow.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> Paul
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> > >>>>>>>>>>>>>> Ponomarev <
> > >>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> > >>>>>>>>>>>>>>        >>>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> Hi Paul!
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> I read your code carefully and now I
> > >>>>>>>>>>>>>> am fully
> > >>>>>>>>>>>>>>        convinced: your
> > >>>>>>>>>>>>>>        >>>>>>>> proposal
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> looks better and should work. We just
> > >>>>>>>>>>>>>> have to
> > >>>>>>>>>>>>>> document
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>> crucial
> > >>>>>>>>>>>>>>        >>>>>>>>> fact
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> that KStream consumers are invoked as
> > >>>>>>>>>>>>>> they're
> > >>>>>>>>>>>>>> added.
> > >>>>>>>>>>>>>>        And then
> > >>>>>>>>>>>>>>        >>>>>> it's
> > >>>>>>>>>>>>>>        >>>>>>>> all
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> going to be very nice.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> What shall we do now? I should
> > >>>>>>>>>>>>>> re-write the
> > >>>>>>>>>>>>>> KIP and
> > >>>>>>>>>>>>>>        resume the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> discussion here, right?
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> Why are you telling that your PR
> > >>>>>>>>>>>>>> 'should not
> > >>>>>>>>>>>>>> be even a
> > >>>>>>>>>>>>>>        >>>>> starting
> > >>>>>>>>>>>>>>        >>>>>>>> point
> > >>>>>>>>>>>>>>        >>>>>>>>> if
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> we go in this direction'? To me it
> > >>>>>>>>>>>>>> looks like
> > >>>>>>>>>>>>>> a good
> > >>>>>>>>>>>>>>        starting
> > >>>>>>>>>>>>>>        >>>>>>> point.
> > >>>>>>>>>>>>>>        >>>>>>>>> But
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> as a novice in this project I might
> > >>>>>>>>>>>>>> miss some
> > >>>>>>>>>>>>>> important
> > >>>>>>>>>>>>>>        >>>>> details.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>> Ivan,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> > >>>>>>>>>>>>>> believe the
> > >>>>>>>>>>>>>>        >>>>> stream.branch()
> > >>>>>>>>>>>>>>        >>>>>>>>> solution
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> supports this. The couponIssuer::set*
> > >>>>>>>>>>>>>> consumers will be
> > >>>>>>>>>>>>>>        >>>>> invoked
> > >>>>>>>>>>>>>>        >>>>>> as
> > >>>>>>>>>>>>>>        >>>>>>>>> they’re
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> added, not during
> > >>>>>>>>>>>>>> streamsBuilder.build(). So
> > >>>>>>>>>>>>>> the user
> > >>>>>>>>>>>>>>        still
> > >>>>>>>>>>>>>>        >>>>>> ought
> > >>>>>>>>>>>>>>        >>>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>> be
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> able to call couponIssuer.coupons()
> > >>>>>>>>>>>>>> afterward
> > >>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>        depend on
> > >>>>>>>>>>>>>>        >>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> branched
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> streams having been set.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>> The issue I mean to point out is
> > >>>>>>>>>>>>>> that it is
> > >>>>>>>>>>>>>> hard to
> > >>>>>>>>>>>>>>        access
> > >>>>>>>>>>>>>>        >>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> branched
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> streams in the same scope as the
> > >>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>> stream (that
> > >>>>>>>>>>>>>>        is, not
> > >>>>>>>>>>>>>>        >>>>>>>> inside
> > >>>>>>>>>>>>>>        >>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> couponIssuer), which is a problem
> > >>>>>>>>>>>>>> with both
> > >>>>>>>>>>>>>> proposed
> > >>>>>>>>>>>>>>        >>>>> solutions.
> > >>>>>>>>>>>>>>        >>>>>> It
> > >>>>>>>>>>>>>>        >>>>>>>>> can be
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> worked around though.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>> [Also, great to hear additional
> > >>>>>>>>>>>>>> interest in
> > >>>>>>>>>>>>>> 401, I’m
> > >>>>>>>>>>>>>>        excited
> > >>>>>>>>>>>>>>        >>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>> hear
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> your thoughts!]
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>> Paul
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> > >>>>>>>>>>>>>> Ponomarev <
> > >>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> > >>>>>>>>>>>>>>        >>>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Hi Paul!
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> The idea to postpone the wiring of
> > >>>>>>>>>>>>>> branches
> > >>>>>>>>>>>>>> to the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> streamsBuilder.build() also looked
> > >>>>>>>>>>>>>> great for
> > >>>>>>>>>>>>>> me at
> > >>>>>>>>>>>>> first
> > >>>>>>>>>>>>>>        >>>>> glance,
> > >>>>>>>>>>>>>>        >>>>>>> but
> > >>>>>>>>>>>>>>        >>>>>>>>> ---
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> the newly branched streams are not
> > >>>>>>>>>>>>>> available in the
> > >>>>>>>>>>>>>>        same
> > >>>>>>>>>>>>>>        >>>>>> scope
> > >>>>>>>>>>>>>>        >>>>>>> as
> > >>>>>>>>>>>>>>        >>>>>>>>> each
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> other.  That is, if we wanted to merge
> > >>>>>>>>>>>>>> them back
> > >>>>>>>>>>>>> together
> > >>>>>>>>>>>>>>        >>>>> again
> > >>>>>>>>>>>>>>        >>>>>> I
> > >>>>>>>>>>>>>>        >>>>>>>>> don't see
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> a way to do that.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> You just took the words right out
> > >>>>>>>>>>>>>> of my
> > >>>>>>>>>>>>>> mouth, I was
> > >>>>>>>>>>>>>>        just
> > >>>>>>>>>>>>>>        >>>>>> going
> > >>>>>>>>>>>>>>        >>>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> write in details about this issue.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Consider the example from Bill's
> > >>>>>>>>>>>>>> book, p.
> > >>>>>>>>>>>>>> 101: say
> > >>>>>>>>>>>>>>        we need
> > >>>>>>>>>>>>>>        >>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>> identify
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> customers who have bought coffee and
> > >>>>>>>>>>>>>> made a
> > >>>>>>>>>>>>>> purchase
> > >>>>>>>>>>>>>>        in the
> > >>>>>>>>>>>>>>        >>>>>>>>> electronics
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> store to give them coupons.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> This is the code I usually write under
> > >>>>>>>>>>>>>> these
> > >>>>>>>>>>>>>>        circumstances
> > >>>>>>>>>>>>>>        >>>>>> using
> > >>>>>>>>>>>>>>        >>>>>>>> my
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> 'brancher' class:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> @Setter
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> class CouponIssuer{
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....>
> > >>>>>>>>>>>>>> coffePurchases;
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....>
> > >>>>>>>>>>>>>> electronicsPurchases;
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   KStream<...> coupons(){
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       return
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       /*In the real world the code
> > >>>>>>>>>>>>>> here
> > >>>>>>>>>>>>>> can be
> > >>>>>>>>>>>>>>        complex, so
> > >>>>>>>>>>>>>>        >>>>>>>>> creation of
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> a separate CouponIssuer class is fully
> > >>>>>>>>>>>>>> justified, in
> > >>>>>>>>>>>>>>        order to
> > >>>>>>>>>>>>>>        >>>>>>>> separate
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> classes' responsibilities.*/
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>  }
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> > >>>>>>>>>>>>>> CouponIssuer();
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate1,
> > >>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate2,
> > >>>>>>>>>>>>>>        >>>>>> couponIssuer::setElectronicsPurchases)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> > >>>>>>>>>>>>>> going to
> > >>>>>>>>>>>>>> wire up
> > >>>>>>>>>>>>>>        everything
> > >>>>>>>>>>>>>>        >>>>>>>> later,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> without the terminal operation!!!*/
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> couponIssuer.coupons()...
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Does this make sense?  In order to
> > >>>>>>>>>>>>>> properly
> > >>>>>>>>>>>>>>        initialize the
> > >>>>>>>>>>>>>>        >>>>>>>>> CouponIssuer
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> we need the terminal operation to be
> > >>>>>>>>>>>>>> called
> > >>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>        >>>>>>>>> streamsBuilder.build()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> is called.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> > >>>>>>>>>>>>>> KIP-401 is
> > >>>>>>>>>>>>>>        essentially
> > >>>>>>>>>>>>>>        >>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> next
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> KIP I was going to write here. I have
> > >>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>> thoughts
> > >>>>>>>>>>>>>>        based on
> > >>>>>>>>>>>>>>        >>>>> my
> > >>>>>>>>>>>>>>        >>>>>>>>> experience,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> so I will join the discussion on KIP-401
> > >>>>>>>>>>>>>> soon.]
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Ivan,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> > >>>>>>>>>>>>>> concept of a
> > >>>>>>>>>>>>>>        fluent
> > >>>>>>>>>>>>>>        >>>>> API
> > >>>>>>>>>>>>>>        >>>>>>>> based
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> off of
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> KStream here
> > >>>>>>>>>>>>>>        (https://github.com/apache/kafka/pull/6512),
> > >>>>>>>>>>>>>>        >>>>>> and
> > >>>>>>>>>>>>>>        >>>>>>> I
> > >>>>>>>>>>>>>>        >>>>>>>>> think
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> succeeded at removing both cons.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> > >>>>>>>>>>>>>> earlier about
> > >>>>>>>>>>>>>>        >>>>>>> compatibility
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> issues,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    there aren't any direct ones. 
> > >>>>>>>>>>>>>> I was
> > >>>>>>>>>>>>>> unaware
> > >>>>>>>>>>>>>>        that Java
> > >>>>>>>>>>>>>>        >>>>> is
> > >>>>>>>>>>>>>>        >>>>>>>> smart
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> enough to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    distinguish between a
> > >>>>>>>>>>>>>> branch(varargs...)
> > >>>>>>>>>>>>>>        returning one
> > >>>>>>>>>>>>>>        >>>>>>> thing
> > >>>>>>>>>>>>>>        >>>>>>>>> and
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> branch()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    with no arguments returning
> > >>>>>>>>>>>>>> another
> > >>>>>>>>>>>>>> thing.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> > >>>>>>>>>>>>>> We don't
> > >>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>        need
> > >>>>>>>>>>>>>>        >>>>> it.
> > >>>>>>>>>>>>>>        >>>>>>> We
> > >>>>>>>>>>>>>>        >>>>>>>>> can
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    build up the branches in the
> > >>>>>>>>>>>>>> KBranchedStream who
> > >>>>>>>>>>>>>>        shares
> > >>>>>>>>>>>>>>        >>>>>> its
> > >>>>>>>>>>>>>>        >>>>>>>>> state
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> with the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> > >>>>>>>>>>>>>> actually do
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        branching.
> > >>>>>>>>>>>>>>        >>>>>>> It's
> > >>>>>>>>>>>>>>        >>>>>>>>> not
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> terribly
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    pretty in its current form, but I
> > >>>>>>>>>>>>>> think it
> > >>>>>>>>>>>>>>        demonstrates
> > >>>>>>>>>>>>>>        >>>>>> its
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> feasibility.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> > >>>>>>>>>>>>>> request should
> > >>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>        >>>>> final
> > >>>>>>>>>>>>>>        >>>>>> or
> > >>>>>>>>>>>>>>        >>>>>>>>> even a
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> starting point if we go in this
> > >>>>>>>>>>>>>> direction,
> > >>>>>>>>>>>>>> I just
> > >>>>>>>>>>>>>>        wanted to
> > >>>>>>>>>>>>>>        >>>>>> see
> > >>>>>>>>>>>>>>        >>>>>>>> how
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> challenging it would be to get the
> > >>>>>>>>>>>>>> API
> > >>>>>>>>>>>>>> working.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I will say though, that I'm not
> > >>>>>>>>>>>>>> sure the
> > >>>>>>>>>>>>>> existing
> > >>>>>>>>>>>>>>        solution
> > >>>>>>>>>>>>>>        >>>>>>> could
> > >>>>>>>>>>>>>>        >>>>>>>> be
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecated in favor of this, which
> > >>>>>>>>>>>>>> I had
> > >>>>>>>>>>>>>> originally
> > >>>>>>>>>>>>>>        >>>>> suggested
> > >>>>>>>>>>>>>>        >>>>>>>> was a
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> possibility.  The reason is that
> > >>>>>>>>>>>>>> the newly
> > >>>>>>>>>>>>>> branched
> > >>>>>>>>>>>>>>        streams
> > >>>>>>>>>>>>>>        >>>>>> are
> > >>>>>>>>>>>>>>        >>>>>>>> not
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> available in the same scope as each
> > >>>>>>>>>>>>>> other.  That
> > >>>>>>>>>>>>>>        is, if we
> > >>>>>>>>>>>>>>        >>>>>>> wanted
> > >>>>>>>>>>>>>>        >>>>>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> merge
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> them back together again I don't
> > >>>>>>>>>>>>>> see a way
> > >>>>>>>>>>>>>> to do
> > >>>>>>>>>>>>>>        that.  The
> > >>>>>>>>>>>>>>        >>>>>> KIP
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> proposal
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> has the same issue, though - all this
> > >>>>>>>>>>>>>> means is that
> > >>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>        >>>>>> either
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> solution,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecating the existing
> > >>>>>>>>>>>>>> branch(...) is
> > >>>>>>>>>>>>>> not on the
> > >>>>>>>>>>>>>>        table.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Paul
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> > >>>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>> Ponomarev <
> > >>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru
> > >>>>>>>>>>>>>> <ma...@mail.ru>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> > >>>>>>>>>>>>>> discussed up to
> > >>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>        >>>>>> point.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> First, it seems that it's
> > >>>>>>>>>>>>>> commonly agreed
> > >>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>        branch API
> > >>>>>>>>>>>>>>        >>>>>>> needs
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> > >>>>>>>>>>>>>> the KIP.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> There are two potential ways to
> > >>>>>>>>>>>>>> do it:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>   
> > >>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> > >>>>>>>>>>>>>> //onTopOf
> > >>>>>>>>>>>>>>        returns
> > >>>>>>>>>>>>>>        >>>>>> its
> > >>>>>>>>>>>>>>        >>>>>>>>> argument
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> > >>>>>>>>>>>>>> compatible. 2)
> > >>>>>>>>>>>>>> The code
> > >>>>>>>>>>>>> won't
> > >>>>>>>>>>>>>>        >>>>> make
> > >>>>>>>>>>>>>>        >>>>>>>> sense
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> until
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> all the necessary ingredients are
> > >>>>>>>>>>>>>> provided.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: The need to create a
> > >>>>>>>>>>>>>> KafkaStreamsBrancher
> > >>>>>>>>>>>>>>        instance
> > >>>>>>>>>>>>>>        >>>>>>>>> contrasts the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> stream
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> > >>>>>>>>>>>>>> noDefault(). Both
> > >>>>>>>>>>>>>>        >>>>>>>>> defaultBranch(..)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault() return void
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> > >>>>>>>>>>>>>> KStreams
> > >>>>>>>>>>>>>> interface
> > >>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>        >>>>>> defined.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> > >>>>>>>>>>>>>> methods
> > >>>>>>>>>>>>>>        >>>>>>>> (defaultBranch(ks->)
> > >>>>>>>>>>>>>>        >>>>>>>>> and
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> > >>>>>>>>>>>>>> is very
> > >>>>>>>>>>>>>> easy to
> > >>>>>>>>>>>>>>        miss the
> > >>>>>>>>>>>>>>        >>>>>> fact
> > >>>>>>>>>>>>>>        >>>>>>>>> that one
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> of the terminal methods should be
> > >>>>>>>>>>>>>> called.
> > >>>>>>>>>>>>>> If these
> > >>>>>>>>>>>>>>        methods
> > >>>>>>>>>>>>>>        >>>>>> are
> > >>>>>>>>>>>>>>        >>>>>>>> not
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> called, we can throw an exception in
> > >>>>>>>>>>>>>> runtime.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Colleagues, what are your
> > >>>>>>>>>>>>>> thoughts? Can
> > >>>>>>>>>>>>>> we do
> > >>>>>>>>>>>>> better?
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> > >>>>>>>>>>>>>> пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> > >>>>>>>>>>>>>> пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Paul,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> I see your point when you are
> > >>>>>>>>>>>>>> talking
> > >>>>>>>>>>>>>> about
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> > >>>>>>>>>>>>>> not be
> > >>>>>>>>>>>>>>        implemented the
> > >>>>>>>>>>>>>>        >>>>>>> easy
> > >>>>>>>>>>>>>>        >>>>>>>>> way.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> > >>>>>>>>>>>>>> ideas.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> > >>>>>>>>>>>>>> method that
> > >>>>>>>>>>>>> assumes
> > >>>>>>>>>>>>>>        >>>>> nothing
> > >>>>>>>>>>>>>>        >>>>>>>> will
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> reach
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> the default branch,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> > >>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>> occurs.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> > >>>>>>>>>>>>>> not be
> > >>>>>>>>>>>>>> the only
> > >>>>>>>>>>>>> option
> > >>>>>>>>>>>>>>        >>>>>> besides
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> `default`, because there are
> > >>>>>>>>>>>>>> scenarios
> > >>>>>>>>>>>>>> when we
> > >>>>>>>>>>>>>>        want to
> > >>>>>>>>>>>>>>        >>>>>> just
> > >>>>>>>>>>>>>>        >>>>>>>>> silently
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> > >>>>>>>>>>>>>> match any
> > >>>>>>>>>>>>>>        predicate. 2)
> > >>>>>>>>>>>>>>        >>>>>>> Throwing
> > >>>>>>>>>>>>>>        >>>>>>>>> an
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> exception in the middle of data
> > >>>>>>>>>>>>>> flow
> > >>>>>>>>>>>>>> processing
> > >>>>>>>>>>>>>>        looks
> > >>>>>>>>>>>>>>        >>>>>> like a
> > >>>>>>>>>>>>>>        >>>>>>>> bad
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> idea.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> > >>>>>>>>>>>>>> I would
> > >>>>>>>>>>>>>> prefer to
> > >>>>>>>>>>>>>>        emit a
> > >>>>>>>>>>>>>>        >>>>>>>> special
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> > >>>>>>>>>>>>>> This is
> > >>>>>>>>>>>>>> exactly
> > >>>>>>>>>>>>> where
> > >>>>>>>>>>>>>>        >>>>>>> `default`
> > >>>>>>>>>>>>>>        >>>>>>>>> can
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> used.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> > >>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
> > >>>>>>>>>>>>>>        >>>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>> track
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> branches that haven't been
> > >>>>>>>>>>>>>> terminated
> > >>>>>>>>>>>>>> and raise
> > >>>>>>>>>>>>>>        a clear
> > >>>>>>>>>>>>>>        >>>>>>> error
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> becomes an issue.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> > >>>>>>>>>>>>>> when the
> > >>>>>>>>>>>>>> program is
> > >>>>>>>>>>>>>>        >>>>> compiled
> > >>>>>>>>>>>>>>        >>>>>>> and
> > >>>>>>>>>>>>>>        >>>>>>>>> run?
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> > >>>>>>>>>>>>>> simply
> > >>>>>>>>>>>>>> won't
> > >>>>>>>>>>>>>>        compile if
> > >>>>>>>>>>>>>>        >>>>> used
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> > >>>>>>>>>>>>>> API as a
> > >>>>>>>>>>>>>>        method chain
> > >>>>>>>>>>>>>>        >>>>>>>> starting
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> from
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> > >>>>>>>>>>>>>> cost
> > >>>>>>>>>>>>>> difference
> > >>>>>>>>>>>>>>        between
> > >>>>>>>>>>>>>>        >>>>>>>> runtime
> > >>>>>>>>>>>>>>        >>>>>>>>> and
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> > >>>>>>>>>>>>>> failure
> > >>>>>>>>>>>>>> uncovers
> > >>>>>>>>>>>>>>        >>>>> instantly
> > >>>>>>>>>>>>>>        >>>>>> on
> > >>>>>>>>>>>>>>        >>>>>>>>> unit
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> > >>>>>>>>>>>>>> project
> > >>>>>>>>>>>>>> than a
> > >>>>>>>>>>>>>>        compilation
> > >>>>>>>>>>>>>>        >>>>>>>> failure.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> > >>>>>>>>>>>>>> пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Ivan,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> > >>>>>>>>>>>>>> operation being
> > >>>>>>>>>>>>>>        required.
> > >>>>>>>>>>>>>>        >>>>>>> But
> > >>>>>>>>>>>>>>        >>>>>>>> is
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> really
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> > >>>>>>>>>>>>>> doesn't
> > >>>>>>>>>>>>>> want a
> > >>>>>>>>>>>>>>        >>>>>> defaultBranch
> > >>>>>>>>>>>>>>        >>>>>>>>> they
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> call
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> some other terminal method
> > >>>>>>>>>>>>>> (noDefaultBranch()?)
> > >>>>>>>>>>>>>>        just as
> > >>>>>>>>>>>>>>        >>>>>>>>> easily.  In
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> fact I
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> think it creates an
> > >>>>>>>>>>>>>> opportunity for a
> > >>>>>>>>>>>>>> nicer API
> > >>>>>>>>>>>>> - a
> > >>>>>>>>>>>>>>        >>>>> user
> > >>>>>>>>>>>>>>        >>>>>>>> could
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> specify
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> > >>>>>>>>>>>>>> nothing
> > >>>>>>>>>>>>>> will reach
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>> default
> > >>>>>>>>>>>>>>        >>>>>>>>> branch,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> > >>>>>>>>>>>>>> a case
> > >>>>>>>>>>>>>> occurs.
> > >>>>>>>>>>>>> That
> > >>>>>>>>>>>>>>        >>>>> seems
> > >>>>>>>>>>>>>>        >>>>>>> like
> > >>>>>>>>>>>>>>        >>>>>>>>> an
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> improvement over the current
> > >>>>>>>>>>>>>> branch()
> > >>>>>>>>>>>>>> API,
> > >>>>>>>>>>>>>>        which allows
> > >>>>>>>>>>>>>>        >>>>>> for
> > >>>>>>>>>>>>>>        >>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> more
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> subtle
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> > >>>>>>>>>>>>>> getting
> > >>>>>>>>>>>>> dropped.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> > >>>>>>>>>>>>>> certainly has
> > >>>>>>>>>>>>>>        to be
> > >>>>>>>>>>>>>>        >>>>>> well
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> documented, but
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> > >>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
> > >>>>>>>>>>>>>>        >>>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>> track
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> branches that haven't been
> > >>>>>>>>>>>>>> terminated
> > >>>>>>>>>>>>>> and raise
> > >>>>>>>>>>>>>>        a clear
> > >>>>>>>>>>>>>>        >>>>>>> error
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> > >>>>>>>>>>>>>> now that
> > >>>>>>>>>>>>>> there is
> > >>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>        >>>>> "build
> > >>>>>>>>>>>>>>        >>>>>>>> step"
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> where
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> > >>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>        >>>>>> StreamsBuilder.build()
> > >>>>>>>>>>>>>>        >>>>>>> is
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> called.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> > >>>>>>>>>>>>>> argument, I
> > >>>>>>>>>>>>> agree
> > >>>>>>>>>>>>>>        >>>>> that
> > >>>>>>>>>>>>>>        >>>>>>> it's
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> critical to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> allow users to do other
> > >>>>>>>>>>>>>> operations on
> > >>>>>>>>>>>>>> the input
> > >>>>>>>>>>>>>>        stream.
> > >>>>>>>>>>>>>>        >>>>>>> With
> > >>>>>>>>>>>>>>        >>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluent
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> > >>>>>>>>>>>>>> same
> > >>>>>>>>>>>>>> way all
> > >>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>        >>>>>>> operations
> > >>>>>>>>>>>>>>        >>>>>>>>> do -
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> want to process off the original
> > >>>>>>>>>>>>>> KStream
> > >>>>>>>>>>>>> multiple
> > >>>>>>>>>>>>>>        >>>>> times,
> > >>>>>>>>>>>>>>        >>>>>>> you
> > >>>>>>>>>>>>>>        >>>>>>>>> just
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> need the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> > >>>>>>>>>>>>>> can call
> > >>>>>>>>>>>>>> as many
> > >>>>>>>>>>>>>>        operations
> > >>>>>>>>>>>>>>        >>>>>> on
> > >>>>>>>>>>>>>>        >>>>>>> it
> > >>>>>>>>>>>>>>        >>>>>>>>> as
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> desire.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Thoughts?
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Paul
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> > >>>>>>>>>>>>>> PM Ivan
> > >>>>>>>>>>>>>> Ponomarev <
> > >>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru
> > >>>>>>>>>>>>>> <ma...@mail.ru>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> > >>>>>>>>>>>>>> because we
> > >>>>>>>>>>>>>> do not
> > >>>>>>>>>>>>>>        always need
> > >>>>>>>>>>>>>>        >>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> > >>>>>>>>>>>>>> terminal
> > >>>>>>>>>>>>> operation we
> > >>>>>>>>>>>>>>        >>>>> don't
> > >>>>>>>>>>>>>>        >>>>>>>> know
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> when to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> > >>>>>>>>>>>>>> switch'.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> > >>>>>>>>>>>>>> returns its
> > >>>>>>>>>>>>>> argument,
> > >>>>>>>>>>>>>>        so we
> > >>>>>>>>>>>>>>        >>>>> can
> > >>>>>>>>>>>>>>        >>>>>> do
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> something
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> more with the original branch
> > >>>>>>>>>>>>>> after
> > >>>>>>>>>>>>>> branching.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> > >>>>>>>>>>>>>> need of
> > >>>>>>>>>>>>> special
> > >>>>>>>>>>>>>>        >>>>> object
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> construction
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> > >>>>>>>>>>>>>> KStream
> > >>>>>>>>>>>>>> methods.
> > >>>>>>>>>>>>> But
> > >>>>>>>>>>>>>>        >>>>> here
> > >>>>>>>>>>>>>>        >>>>>> we
> > >>>>>>>>>>>>>>        >>>>>>>>> have a
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> special case: we build the
> > >>>>>>>>>>>>>> switch to
> > >>>>>>>>>>>>>> split the
> > >>>>>>>>>>>>>>        flow,
> > >>>>>>>>>>>>>>        >>>>> so
> > >>>>>>>>>>>>>>        >>>>>> I
> > >>>>>>>>>>>>>>        >>>>>>>>> think
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> > >>>>>>>>>>>>>> пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Ivan,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> > >>>>>>>>>>>>>> improve
> > >>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>        API, but I
> > >>>>>>>>>>>>>>        >>>>>> find
> > >>>>>>>>>>>>>>        >>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> > >>>>>>>>>>>>>> since it
> > >>>>>>>>>>>>>>        contrasts the
> > >>>>>>>>>>>>>>        >>>>>>> fluency
> > >>>>>>>>>>>>>>        >>>>>>>>> of
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> KStream method calls. 
> > >>>>>>>>>>>>>> Ideally I'd
> > >>>>>>>>>>>>>> like to
> > >>>>>>>>>>>>>>        just call
> > >>>>>>>>>>>>>>        >>>>> a
> > >>>>>>>>>>>>>>        >>>>>>>>> method on
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> stream
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> > >>>>>>>>>>>>>> bottom if
> > >>>>>>>>>>>>>> the branch
> > >>>>>>>>>>>>>>        cases
> > >>>>>>>>>>>>>>        >>>>> are
> > >>>>>>>>>>>>>>        >>>>>>>>> defined
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> fluently.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think the
> > >>>>>>>>>>>>>> addBranch(predicate,
> > >>>>>>>>>>>>>> handleCase)
> > >>>>>>>>>>>>>>        is very
> > >>>>>>>>>>>>>>        >>>>>> nice
> > >>>>>>>>>>>>>>        >>>>>>>>> and the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> right
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> way
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> > >>>>>>>>>>>>>> flipped
> > >>>>>>>>>>>>>> around
> > >>>>>>>>>>>>>>        how we
> > >>>>>>>>>>>>>>        >>>>>>> specify
> > >>>>>>>>>>>>>>        >>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> source
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Like:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>          
> > >>>>>>>>>>>>>> .addBranch(predicate1,
> > >>>>>>>>>>>>> this::handle1)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>          
> > >>>>>>>>>>>>>> .addBranch(predicate2,
> > >>>>>>>>>>>>> this::handle2)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> > >>>>>>>>>>>>>> KBranchedStreams or
> > >>>>>>>>>>>>>>        >>>>>>>> KStreamBrancher
> > >>>>>>>>>>>>>>        >>>>>>>>> or
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> something,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> which is added to by
> > >>>>>>>>>>>>>> addBranch() and
> > >>>>>>>>>>>>>>        terminated by
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> defaultBranch()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> (which
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> > >>>>>>>>>>>>>> obviously
> > >>>>>>>>>>>>>>        incompatible with
> > >>>>>>>>>>>>>>        >>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> current
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> API, so
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> > >>>>>>>>>>>>>> have to
> > >>>>>>>>>>>>>> have a
> > >>>>>>>>>>>>>>        different
> > >>>>>>>>>>>>>>        >>>>>> name,
> > >>>>>>>>>>>>>>        >>>>>>>> but
> > >>>>>>>>>>>>>>        >>>>>>>>> that
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> seems
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> > >>>>>>>>>>>>>> - we
> > >>>>>>>>>>>>>> could call it
> > >>>>>>>>>>>>>>        >>>>>> something
> > >>>>>>>>>>>>>>        >>>>>>>> like
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branched()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> > >>>>>>>>>>>>>> deprecate the
> > >>>>>>>>>>>>>> old API.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> > >>>>>>>>>>>>>> motivations of
> > >>>>>>>>>>>>>> your
> > >>>>>>>>>>>>>>        KIP?  It
> > >>>>>>>>>>>>>>        >>>>>> seems
> > >>>>>>>>>>>>>>        >>>>>>>>> like it
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> does to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> > >>>>>>>>>>>>>> branching
> > >>>>>>>>>>>>>>        while also
> > >>>>>>>>>>>>>>        >>>>>>>> allowing
> > >>>>>>>>>>>>>>        >>>>>>>>> you
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> > >>>>>>>>>>>>>> branches off of
> > >>>>>>>>>>>>>>        KBranchedStreams
> > >>>>>>>>>>>>>>        >>>>>> if
> > >>>>>>>>>>>>>>        >>>>>>>>> desired.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Paul
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> > >>>>>>>>>>>>>> PM Ivan
> > >>>>>>>>>>>>>> Ponomarev
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
> > >>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> > >>>>>>>>>>>>>>        ks){
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
> > >>>>>>>>>>>>>> handleSecondCase(KStream<String,
> > >>>>>>>>>>>>>>        String> ks){
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> ......
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> > >>>>>>>>>>>>>> String>()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> > >>>>>>>>>>>>>>        this::handleFirstCase)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> > >>>>>>>>>>>>>>        this::handleSecondCase)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> > >>>>>>>>>>>>>> Bejeck пишет:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> > >>>>>>>>>>>>> KafkaStreamsBrancher
> > >>>>>>>>>>>>>>        >>>>> takes a
> > >>>>>>>>>>>>>>        >>>>>>>>> Consumer
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> as a
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> second
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> > >>>>>>>>>>>>>> nothing,
> > >>>>>>>>>>>>>> and the
> > >>>>>>>>>>>>>>        example in
> > >>>>>>>>>>>>>>        >>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>> KIP
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> shows
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> each
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> > >>>>>>>>>>>>>> using a
> > >>>>>>>>>>>>>> terminal node
> > >>>>>>>>>>>>>>        >>>>>>>>> (KafkaStreams#to()
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> in this
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> case).
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> > >>>>>>>>>>>>>> something, but
> > >>>>>>>>>>>>>> how would
> > >>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>        >>>>> handle
> > >>>>>>>>>>>>>>        >>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>> case
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> where the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> > >>>>>>>>>>>>>> wants to
> > >>>>>>>>>>>>> continue
> > >>>>>>>>>>>>>>        >>>>>>>> processing
> > >>>>>>>>>>>>>>        >>>>>>>>> and
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> > >>>>>>>>>>>>>> node on
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>> branched
> > >>>>>>>>>>>>>>        >>>>>> stream
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> immediately?
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> > >>>>>>>>>>>>>> logic
> > >>>>>>>>>>>>>> as is if
> > >>>>>>>>>>>>>>        we had
> > >>>>>>>>>>>>>>        >>>>>>>> something
> > >>>>>>>>>>>>>>        >>>>>>>>> like
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> this:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> > >>>>>>>>>>>>>> branches =
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> originalStream.branch(predicate1,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Bill
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> > >>>>>>>>>>>>>> 6:15 PM
> > >>>>>>>>>>>>>> Bill Bejeck
> > >>>>>>>>>>>>> <
> > >>>>>>>>>>>>>>        >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> All,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> > >>>>>>>>>>>>>> discussion for
> > >>>>>>>>>>>>> KIP-
> > >>>>>>>>>>>>>>        >>>>> 418.
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> > >>>>>>>>>>>>>> discussion
> > >>>>>>>>>>>>>> about
> > >>>>>>>>>>>>> KIP-418.
> > >>>>>>>>>>>>>>        >>>>> Please
> > >>>>>>>>>>>>>>        >>>>>>>> take
> > >>>>>>>>>>>>>>        >>>>>>>>> a
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>> look
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> > >>>>>>>>>>>>>> appreciate any
> > >>>>>>>>>>>>>>        feedback :)
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> > >>>>>>>>>>>>>>        >>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> > >>>>>>>>>>>>>>        >>>>> https://github.com/apache/kafka/pull/6164
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>        >>>>>>>>>
> > >>>>>>>>>>>>>>        >
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>>
> > >>
> > > 
> > 
> > 
> > Attachments:
> > * signature.asc
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Thanks for this thought, Matthias,

Your idea has a few aspects I find attractive:
1. There’s no ambiguity at all about what will be in the map, because there’s only one thing that could be there, which is whatever is returned from the chain function.
2. We keep the API smaller. Thanks to the extensible way this KIP is designed, it would be trivially easy to add the “terminal” chain later. As you say, fewer concepts leads to an API that is easier to learn.
3. We get to side-step the naming of this method. Although I didn’t complain about withJavaConsumer, it was only because I couldn’t think of a better name. Still, it’s somewhat unsatisfying to name a method after its argument type, since this provides no information at all about what the method does. I was willing to accept it because I didn’t have an alternative, but I would be happy to skip this method for now to avoid the problem until we have more inspiration. 

The only con I see is that it makes the code a little less ergonomic to write when you don’t want to return the result of the chain (such as when the chain is terminal), since I’m your example, you have to declare a block with a return statement at the end. It’s not ideal, but it doesn’t seem too bad to me. 

Lastly, on the null question, I’d be fine with allowing a null result, which would just remove the branch from the returned map. It seems nicer than forcing people to pick a stream to return when their chain is terminal and they don’t want to use the result later.

Thanks again for sharing the idea,
John

On Wed, May 20, 2020, at 18:17, Matthias J. Sax wrote:
> Thanks for updating the KIP!
> 
> I guess the only open question is about `Branched.withJavaConsumer` and
> its relationship to the returned `Map`.
> 
> Originally, we discussed two main patterns:
> 
>  (1) split a stream and return the substreams for futher processing
>  (2) split a stream and modify the substreams with in-place method chaining
> 
> To combine both patterns we wanted to allow for
> 
>   -> split a stream, modify the substreams, and return the _modified_
> substreams for further processing
> 
> > But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there
> 
> That is of course possible. However, it introduces some "hidded" semantics:
> 
>  - using `withChain` I get the modified sub-stream
>  - using `withJavaConsumer` I get the unmodifed sub-stream
> 
> This seems to be quite subtle to me.
> 
> 
> 
> From my understanding the original idea of `withJavaConsumer` was to
> model a terminal operation, ie, it should be similar to:
> 
> Branched.withChain(s -> {
>   s.to();
>   return null;
> })
> 
> However, I am not sure if we should even allow `withChain()` to return
> `null`? IMHO, we should throw an exception for this case to avoid a `key
> -> null` entry in the returned Map.
> 
> Following this train of through, and if we want to allow the "return
> null" pattern in general, we need `withJavaConsumer` that does not add
> an entry to the Map.
> 
> Following your proposal, the semantics of `withJavaConsumer` could also
> be achieved with `withChain`:
> 
> Branched.withChain(s -> {
>   s.to();
>   return s;
> })
> 
> Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
> while for the first proposal it adds new functionality (if `return null`
> is not allowed, using `withChain()` is not possible to "hide a
> sub-stream in the result). Furthermore, we might need to allow `return
> null` in your prosal to allow uses to "hide" a sub-stream in the Map.
> 
> 
> 
> I guess I can be convinced either way. However, if we follow your
> proposal, I am wondering if we need `withJavaConsumer` at all? Its
> benefit seems to be small? Also, having a reduced API is usually
> preferable as it's simpler to learn.
> 
> 
> 
> -Matthias
> 
> 
> 
> 
> On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> > Hello, John, hello Matthias!
> > 
> > Thank you very much for your detailed feedback!
> > 
> > -----------------------------------------
> > 
> > John,
> > 
> >> It looks like you missed my reply on Apr 23rd.
> > 
> > For some unknown reason it didn't reach my inbox, fortunately we have
> > all the emails on the web.
> > 
> >> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> > method?
> > 
> > Done, in "Compatibility, Deprecation, and Migration Plan" section.
> > 
> >> 2. [Explain why 'branch' operator is superior to branching directly
> > off of the parent KStream for the needs of dynamic branching]
> > 
> > Done, see an ugly counterexample in 'Dynamic Branching' section.
> > 
> >> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> > 
> > As Mathhias noted, `withSink` can also be confusing. I renamed this
> > method to `withJavaConsumer` per Matthias' suggestion.
> > 
> >> 4. ...It seems like there are two disjoint use cases: EITHER using
> > chain and the result map OR using just the sink
> > 
> > This is discussed below.
> > 
> > ----------------------------------------------
> > 
> > Mathhias,
> > 
> >> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> > 
> > Done.
> > 
> >> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> > method]
> > 
> > Fixed.
> > 
> > 
> >> 3. Overview of newly added methods/interfaces
> > 
> > Done in `Proposed Changes` section.
> > 
> > 
> >> 4. [Concerning John's note] > I don't think that using both
> > `withChain()` and `withConsumer()` is the
> > issue, as the KIP clearly states that the result of `withChain()` will
> > be given to the `Consumer`.
> > 
> > Yes, I agree!
> > 
> >> The issue is really with the `Consumer` and the returned `Map` of
> > `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> > implementation would be to not add the "branch" to the result map if
> > `withConsumer` is used?
> > 
> > But is it also an issue? With Kafka Streams, we can split the topology
> > graph at any point. Technically, it's OK to do both: feed the KStream to
> > a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> > stream in the Map, one simply does not extract it from there :-)
> > 
> > In the current version of KIP it is assumed that the returned map
> > contains ALL the branches, either tagged with IDs explicitly set by the
> > programmer, or with some default auto-generated ids. Dealing with this
> > map is the user's responsibility.
> > 
> > What seems to me to be an issue is introducing exclusions to this
> > general rule, like 'swallowing' some streams by provided
> > [Java]Consumers. This can make things complicated. What if a user
> > provides both the name of the branch and a [Java]Consumer? What do they
> > mean in this case? Should we 'swallow' the stream or save it to the map?
> > There's no point in 'saving the space' in this map, so maybe just leave
> > it as it is?
> > 
> > ----
> > 
> > I rewrote the KIP and also fixed a couple of typos.
> > 
> > Looking forward for your feedback again!
> > 
> > Regards,
> > 
> > Ivan.
> > 
> > 
> > 
> > 08.05.2020 22:55, Matthias J. Sax пишет:
> >> Thanks for updating the KIP!
> >>
> >> I also have some minor comment:
> >>
> >>
> >>
> >> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> >>
> >> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> >> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> >> and `KGroupedKTable` that we cannot rename without a breaking change...
> >> so we just keep them.)
> >>
> >>
> >>
> >> (2) Quote:
> >>
> >>> Both branch and defaultBranch operations also have overloaded
> >>> parameterless alternatives.
> >>
> >> I think `branch()` always needs to take a `Predicate` and assume you
> >> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> >> as `branch()` would not be "parameterless".
> >>
> >>
> >>
> >> (3) Can you maybe add an overview in the "Public Interface" section) of
> >> newly added and deprecated methods/classes (cf.
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> >>
> >>
> >>
> >>
> >> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> >> and the finally returned `Map<String, KStream>`. This related to John's
> >> 4th comment:
> >>
> >>> It seems like there are really two disjoint use cases: EITHER using
> >>> chain and the result map OR using just the sink.
> >>
> >> I don't think that using both `withChain()` and `withConsumer()` is the
> >> issue though, as the KIP clearly states that the result of `withChain()`
> >> will be given to the `Consumer`. The issue is really with the `Consumer`
> >> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> >>
> >> Maybe a reasonable implementation would be to not add the "branch" to
> >> the result map if `withConsumer` is used? As long as we clearly document
> >> it in the JavaDocs, this might be fine?
> >>
> >>
> >>
> >> (5) Reply to John's comments:
> >>
> >>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
> >>> were talking about the kafka Consumer interface (which doesn’t make
> >>> sense, of course). I get that you were referring to the java Consumer
> >>> interface, but we should still probably to to avoid the ambiguity.
> >>> Just throwing out a suggestion, how about ‘withSink’?
> >>
> >> IMHO, `withSink` has the issue that it might be confused with a "sink
> >> node", ie., writing the KStream to a topic.
> >>
> >> Maybe `withJavaConsumer` would make it less ambiguous?
> >>
> >>
> >>
> >>
> >> -Matthias
> >>
> >>
> >>
> >>
> >> On 5/8/20 7:13 AM, John Roesler wrote:
> >>> Hi Ivan,
> >>>
> >>> It looks like you missed my reply on Apr 23rd. I think it’s close,
> >>> but I had a few last comments.
> >>>
> >>> Thanks,
> >>> John
> >>>
> >>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> >>>> Hello everyone,
> >>>>
> >>>> will someone please take a look at the reworked KIP?
> >>>>
> >>>> I believe that now it follows design principles and takes into account
> >>>> all the arguments discussed here.
> >>>>
> >>>>
> >>>> Regards,
> >>>>
> >>>> Ivan
> >>>>
> >>>>
> >>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
> >>>>> Hi,
> >>>>>
> >>>>> I have read the John's "DSL design principles" and have completely
> >>>>> rewritten the KIP, see
> >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> This version includes all the previous discussion results and follows
> >>>>> the design principles, with one exception.
> >>>>>
> >>>>> The exception is
> >>>>>
> >>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> >>>>>
> >>>>> which formally violates 'no more than one parameter' rule, but I think
> >>>>> here it is justified.
> >>>>>
> >>>>> We must provide a predicate for a branch and don't need to provide one
> >>>>> for the default branch. Thus for both operations we may use a single
> >>>>> Branched parameter class, with an extra method parameter for `branch`.
> >>>>>
> >>>>> Since predicate is a natural, necessary part of a branch, no
> >>>>> 'proliferation of overloads, deprecations, etc.' is expected here
> >>>>> as it
> >>>>> is said in the rationale for the 'single parameter rule'.
> >>>>>
> >>>>> WDYT, is this KIP mature enough to begin voting?
> >>>>>
> >>>>> Regards,
> >>>>>
> >>>>> Ivan
> >>>>>
> >>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
> >>>>>> Ivan,
> >>>>>>
> >>>>>> no worries about getting side tracked. Glad to have you back!
> >>>>>>
> >>>>>> The DSL improved further in the meantime and we already have a
> >>>>>> `Named`
> >>>>>> config object to name operators. It seems reasonable to me to
> >>>>>> build on
> >>>>>> this.
> >>>>>>
> >>>>>> Furthermore, John did a writeup about "DSL design principles" that we
> >>>>>> want to follow:
> >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> >>>>>>
> >>>>>>
> >>>>>> -- might be worth to checkout.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>>
> >>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>>>>>> Hi everyone!
> >>>>>>>
> >>>>>>> Let me revive the discussion of this KIP.
> >>>>>>>
> >>>>>>> I'm very sorry for stopping my participation in the discussion in
> >>>>>>> June
> >>>>>>> 2019. My project work was very intensive then and it didn't leave me
> >>>>>>> spare time. But I think I must finish this, because we invested
> >>>>>>> substantial effort into this discussion and I'm not feel entitled to
> >>>>>>> propose other things before this one is finalized.
> >>>>>>>
> >>>>>>> During these months I proceeded with writing and reviewing Kafka
> >>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>>>>>> KafkaStreamBrancher class of my invention (the original idea for
> >>>>>>> this
> >>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
> >>>>>>> the
> >>>>>>> KIP forward. When I was coming across the problem with the scope of
> >>>>>>> branches, I worked around it this way:
> >>>>>>>
> >>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>>>>>> new KafkaStreamBrancher<....>()
> >>>>>>>       .branch(....)
> >>>>>>>       .defaultBranch(result::set)
> >>>>>>>       .onTopOf(someStream);
> >>>>>>> result.get()...
> >>>>>>>
> >>>>>>>
> >>>>>>> And yes, of course I don't feel very happy with this approach.
> >>>>>>>
> >>>>>>> I think that Matthias came up with a bright solution in his post
> >>>>>>> from
> >>>>>>> May, 24th 2019. Let me quote it:
> >>>>>>>
> >>>>>>> KStream#split() -> KBranchedStream
> >>>>>>> // branch is not easily accessible in current scope
> >>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>     -> KBranchedStream
> >>>>>>> // assign a name to the branch and
> >>>>>>> // return the sub-stream to the current scope later
> >>>>>>> //
> >>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>>>>>     -> KBranchedStream
> >>>>>>> // default branch is not easily accessible
> >>>>>>> // return map of all named sub-stream into current scope
> >>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>     -> Map<String,KStream>
> >>>>>>> // assign custom name to default-branch
> >>>>>>> // return map of all named sub-stream into current scope
> >>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>     -> Map<String,KStream>
> >>>>>>> // assign a default name for default
> >>>>>>> // return map of all named sub-stream into current scope
> >>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>     -> Map<String,KStream>
> >>>>>>> // return map of all names sub-stream into current scope
> >>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>     -> Map<String,KStream>
> >>>>>>>
> >>>>>>> I believe this would satisfy everyone. Optional names seems to be
> >>>>>>> a good
> >>>>>>> idea: when you don't need to have the branches in the same scope,
> >>>>>>> you
> >>>>>>> just don't use names and you don't risk making your code brittle.
> >>>>>>> Or,
> >>>>>>> you might want to add names just for debugging purposes. Or,
> >>>>>>> finally,
> >>>>>>> you might use the returned Map to have the named branches in the
> >>>>>>> original scope.
> >>>>>>>
> >>>>>>> There also was an input from John Roesler on June 4th, 2019, who
> >>>>>>> suggested using Named class. I can't comment on this. The idea seems
> >>>>>>> reasonable, but in this matter I'd rather trust people who are more
> >>>>>>> familiar with Streams API design principles than me.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>>
> >>>>>>> Ivan
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
> >>>>>>>> the
> >>>>>>>> KIP
> >>>>>>>> at any point.
> >>>>>>>>
> >>>>>>>> If anybody else is interested in picking up this KIP, feel free to
> >>>>>>>> do so.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>>>>>> Ivan,
> >>>>>>>>>
> >>>>>>>>> did you see my last reply? What do you think about my proposal
> >>>>>>>>> to mix
> >>>>>>>>> both approaches and try to get best-of-both worlds?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -Matthias
> >>>>>>>>>
> >>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>>>>>> Thanks for the input John!
> >>>>>>>>>>
> >>>>>>>>>>> under your suggestion, it seems that the name is required
> >>>>>>>>>>
> >>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
> >>>>>>>>>> using a
> >>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
> >>>>>>>>>> using a
> >>>>>>>>>> `Consumer`, no.
> >>>>>>>>>>
> >>>>>>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>>>>>
> >>>>>>>>>> I wanted to sketch out a high level proposal to merge both
> >>>>>>>>>> patterns
> >>>>>>>>>> only. Your suggestions to align the new API with the existing API
> >>>>>>>>>> make
> >>>>>>>>>> totally sense.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>>>>>
> >>>>>>>>>> If both are mandatory, what do we gain by it? The returned
> >>>>>>>>>> `Map` only
> >>>>>>>>>> contains the corresponding branches, so why should we prefix
> >>>>>>>>>> all of
> >>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>>>>>> `split()`, the same question raises?
> >>>>>>>>>>
> >>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
> >>>>>>>>>> `Named` is
> >>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
> >>>>>>>>>> counter
> >>>>>>>>>> for
> >>>>>>>>>> different branch name. However, this might lead to the problem of
> >>>>>>>>>> changing names if branches are added/removed. Also, how would the
> >>>>>>>>>> names
> >>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>>>>>> returned in the `Map`).
> >>>>>>>>>>
> >>>>>>>>>> If `Named` is optional for both, it could happen that a user
> >>>>>>>>>> misses to
> >>>>>>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Hence, I am actually in favor to not allow a default name but
> >>>>>>>>>> keep
> >>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
> >>>>>>>>>> required
> >>>>>>>>>> if a
> >>>>>>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>>>>>> specifying a
> >>>>>>>>>> name is required if a `Function` is used.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> About
> >>>>>>>>>>
> >>>>>>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>
> >>>>>>>>>> I don't think that the branching predicate is a configuration and
> >>>>>>>>>> hence
> >>>>>>>>>> would not include it in a configuration object.
> >>>>>>>>>>
> >>>>>>>>>>>        withChain(...);
> >>>>>>>>>>
> >>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
> >>>>>>>>>> does not
> >>>>>>>>>> seem to be a configuration. We can also not prevent a user to
> >>>>>>>>>> call
> >>>>>>>>>> `withName()` in combination of `withChain()` what does not
> >>>>>>>>>> make sense
> >>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
> >>>>>>>>>> check
> >>>>>>>>>> seems less appealing. Also, it could happen that neither
> >>>>>>>>>> `withChain()`
> >>>>>>>>>> not `withName()` is called and the branch is missing in the
> >>>>>>>>>> returned
> >>>>>>>>>> `Map` what lead to runtime issues, too.
> >>>>>>>>>>
> >>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
> >>>>>>>>>> object
> >>>>>>>>>> is helpful if each configuration can be set independently of all
> >>>>>>>>>> others,
> >>>>>>>>>> but this seems not to be the case here. If we add new
> >>>>>>>>>> configuration
> >>>>>>>>>> later, we can also just move forward by deprecating the
> >>>>>>>>>> methods that
> >>>>>>>>>> accept `Named` and add new methods that accepted
> >>>>>>>>>> `BranchConfig` (that
> >>>>>>>>>> would of course implement `Named`).
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Thoughts?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> @Ivan, what do you think about the general idea to blend the
> >>>>>>>>>> two main
> >>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> -Matthias
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
> >>>>>>>>>>> satisfy
> >>>>>>>>>>> everyone. Returning the map from the terminal operations also
> >>>>>>>>>>> solves
> >>>>>>>>>>> the problem of merging/joining the branched streams, if we want
> >>>>>>>>>>> to add
> >>>>>>>>>>> support for the compliment later on.
> >>>>>>>>>>>
> >>>>>>>>>>> Under your suggestion, it seems that the name is required.
> >>>>>>>>>>> Otherwise,
> >>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
> >>>>>>>>>>> actually
> >>>>>>>>>>> not too bad, since experience has taught us that, although
> >>>>>>>>>>> names for
> >>>>>>>>>>> operations are not required to define stream processing
> >>>>>>>>>>> logic, it
> >>>>>>>>>>> does
> >>>>>>>>>>> significantly improve the operational experience when you can
> >>>>>>>>>>> map
> >>>>>>>>>>> the
> >>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>>>>>> wouldn't (have to) reference the name to chain extra
> >>>>>>>>>>> processing onto
> >>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>>>>>
> >>>>>>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>>>>>> operator itself, and then all the branches get index-suffixed
> >>>>>>>>>>> names
> >>>>>>>>>>> built from the branch operator name. I guess under this
> >>>>>>>>>>> proposal, we
> >>>>>>>>>>> could naturally append the branch name to the branching operator
> >>>>>>>>>>> name,
> >>>>>>>>>>> like this:
> >>>>>>>>>>>
> >>>>>>>>>>>       stream.split(Named.withName("mysplit")) //creates node
> >>>>>>>>>>> "mysplit"
> >>>>>>>>>>>                  .branch(..., ..., "abranch") // creates node
> >>>>>>>>>>> "mysplit-abranch"
> >>>>>>>>>>>                  .defaultBranch(...) // creates node
> >>>>>>>>>>> "mysplit-default"
> >>>>>>>>>>>
> >>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>>>>>
> >>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
> >>>>>>>>>>> debate
> >>>>>>>>>>> the "best" syntax in the context of each operation, but in
> >>>>>>>>>>> general,
> >>>>>>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>>>>>
> >>>>>>>>>>>        operator(function, config_object?) OR
> >>>>>>>>>>> operator(config_object)
> >>>>>>>>>>>
> >>>>>>>>>>> where config_object is often just Named in the "function"
> >>>>>>>>>>> variant.
> >>>>>>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>>>>>
> >>>>>>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>>>>>
> >>>>>>>>>>>      operator(function, function, string)
> >>>>>>>>>>>
> >>>>>>>>>>> where the string is the name.
> >>>>>>>>>>> My first question is whether the name should instead be
> >>>>>>>>>>> specified
> >>>>>>>>>>> with
> >>>>>>>>>>> the NamedOperation interface.
> >>>>>>>>>>>
> >>>>>>>>>>> My second question is whether we should just roll all these
> >>>>>>>>>>> arguments
> >>>>>>>>>>> up into a config object like:
> >>>>>>>>>>>
> >>>>>>>>>>>       KBranchedStream#branch(BranchConfig)
> >>>>>>>>>>>
> >>>>>>>>>>>       interface BranchConfig extends NamedOperation {
> >>>>>>>>>>>        withPredicate(...);
> >>>>>>>>>>>        withChain(...);
> >>>>>>>>>>>        withName(...);
> >>>>>>>>>>>      }
> >>>>>>>>>>>
> >>>>>>>>>>> Although I guess we'd like to call BranchConfig something
> >>>>>>>>>>> more like
> >>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>>>>>
> >>>>>>>>>>> This makes the source code a little noisier, but it also
> >>>>>>>>>>> makes us
> >>>>>>>>>>> more
> >>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
> >>>>>>>>>>> purely
> >>>>>>>>>>> in the config interface, and never have to deal with adding
> >>>>>>>>>>> overloads
> >>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>>>>>
> >>>>>>>>>>> WDYT?
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> -John
> >>>>>>>>>>>
> >>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
> >>>>>>>>>>>> view.
> >>>>>>>>>>>> Good
> >>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Interesting discussion.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
> >>>>>>>>>>>>> approaches:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>>>>>      -> KBranchedStream
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // assign a name to the branch and
> >>>>>>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>>>>>> //
> >>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
> >>>>>>>>>>>>> String)
> >>>>>>>>>>>>>      -> KBranchedStream
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // default branch is not easily accessible
> >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // assign custom name to default-branch
> >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // assign a default name for default
> >>>>>>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>>>>>      -> Map<String,KStream>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>> return
> >>>>>>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>>>>>> implementation can
> >>>>>>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>>>>>> returned
> >>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>>>>>> dynamic case.
> >>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>>>>>> necessity.
> >>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
> >>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Separately, I'm interested to see where the present
> >>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>> leads.
> >>>>>>>>>>>>>> I've written enough Javascript code in my life to be
> >>>>>>>>>>>>>> suspicious of
> >>>>>>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>>>>>> references (or
> >>>>>>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>>>>>> that this
> >>>>>>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>>>>>> logic when
> >>>>>>>>>>>>>> their nested closure situation got out of hand.
> >>>>>>>>>>>>>> Unfortunately,
> >>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
> >>>>>>>>>>>>>> code
> >>>>>>>>>>>>>> readability (but in different ways for different reasons). In
> >>>>>>>>>>>>>> other
> >>>>>>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>>>>>> solution if
> >>>>>>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>>>>>> references aren't
> >>>>>>>>>>>>>> the end of the story either, and you can see that by
> >>>>>>>>>>>>>> observing
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>> there have been two follow-on eras, as they continue
> >>>>>>>>>>>>>> trying to
> >>>>>>>>>>>>>> cope with
> >>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>>>>>> First, you
> >>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
> >>>>>>>>>>>>>> nested
> >>>>>>>>>>>>>> code to
> >>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>>>>>> this).
> >>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
> >>>>>>>>>>>>>> apply
> >>>>>>>>>>>>>> language
> >>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
> >>>>>>>>>>>>>> "flattest"
> >>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>> programming style to solve the problem (because you get
> >>>>>>>>>>>>>> back to
> >>>>>>>>>>>>>> just one
> >>>>>>>>>>>>>> code block per functional unit).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>>>>>> nowhere
> >>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
> >>>>>>>>>>>>>> take
> >>>>>>>>>>>>>> the JS
> >>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>>>>>> valuable
> >>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>>>>>> bringing this
> >>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
> >>>>>>>>>>>>>> just
> >>>>>>>>>>>>>> like JS,
> >>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
> >>>>>>>>>>>>>> (and
> >>>>>>>>>>>>>> I'd also
> >>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
> >>>>>>>>>>>>>> punt on
> >>>>>>>>>>>>>> it, by
> >>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>>>>>> there a DSL
> >>>>>>>>>>>>>> way to do it?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
> >>>>>>>>>>>>>> <pgwhalen@gmail.com
> >>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        Ivan, I’ll definitely forfeit my point on the
> >>>>>>>>>>>>>> clumsiness of
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        branch(predicate, consumer) solution, I don’t see
> >>>>>>>>>>>>>> any real
> >>>>>>>>>>>>>> drawbacks
> >>>>>>>>>>>>>>        for the dynamic case.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        IMO the one trade off to consider at this point is the
> >>>>>>>>>>>>>> scope
> >>>>>>>>>>>>>>        question. I don’t know if I totally agree that “we
> >>>>>>>>>>>>>> rarely
> >>>>>>>>>>>>>> need them
> >>>>>>>>>>>>>>        in the same scope” since merging the branches back
> >>>>>>>>>>>>>> together
> >>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>        seems like a perfectly plausible use case that can
> >>>>>>>>>>>>>> be a lot
> >>>>>>>>>>>>>> nicer
> >>>>>>>>>>>>>>        when the branched streams are in the same scope.
> >>>>>>>>>>>>>> That being
> >>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>        for the reasons Ivan listed, I think it is overall the
> >>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>        solution - working around the scope thing is easy
> >>>>>>>>>>>>>> enough if
> >>>>>>>>>>>>>> you need
> >>>>>>>>>>>>>>        to.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>>>>>        <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > Hello everyone, thank you all for joining the
> >>>>>>>>>>>>>> discussion!
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > Well, I don't think the idea of named branches,
> >>>>>>>>>>>>>> be it a
> >>>>>>>>>>>>>>        LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>>>>>> definition
> >>>>>>>>>>>>>>        matters) or `branch` method  taking name and Consumer
> >>>>>>>>>>>>>> has more
> >>>>>>>>>>>>>>        advantages than drawbacks.
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > In my opinion, the only real positive outcome from
> >>>>>>>>>>>>>> Michael's
> >>>>>>>>>>>>>>        proposal is that all the returned branches are in
> >>>>>>>>>>>>>> the same
> >>>>>>>>>>>>>> scope.
> >>>>>>>>>>>>>>        But 1) we rarely need them in the same scope 2)
> >>>>>>>>>>>>>> there is a
> >>>>>>>>>>>>>>        workaround for the scope problem, described in the
> >>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > 'Inlining the complex logic' is not a problem,
> >>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>> can use
> >>>>>>>>>>>>>>        method references instead of lambdas. In real world
> >>>>>>>>>>>>>> scenarios you
> >>>>>>>>>>>>>>        tend to split the complex logic to methods anyway,
> >>>>>>>>>>>>>> so the
> >>>>>>>>>>>>>> code is
> >>>>>>>>>>>>>>        going to be clean.
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > The drawbacks are strong. The cohesion between
> >>>>>>>>>>>>>> predicates
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>        handlers is lost. We have to define predicates in one
> >>>>>>>>>>>>>> place, and
> >>>>>>>>>>>>>>        handlers in another. This opens the door for bugs:
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > - what if we forget to define a handler for a
> >>>>>>>>>>>>>> name? or a
> >>>>>>>>>>>>>> name for
> >>>>>>>>>>>>>>        a handler?
> >>>>>>>>>>>>>>        > - what if we misspell a name?
> >>>>>>>>>>>>>>        > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > What Michael propose would have been totally OK
> >>>>>>>>>>>>>> if we had
> >>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>        writing the API in Lua, Ruby or Python. In those
> >>>>>>>>>>>>>> languages the
> >>>>>>>>>>>>>>        "dynamic naming" approach would have looked most
> >>>>>>>>>>>>>> concise
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>        beautiful. But in Java we expect all the problems
> >>>>>>>>>>>>>> related to
> >>>>>>>>>>>>>>        identifiers to be eliminated in compile time.
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > And if we do, what advantage are we supposed to get
> >>>>>>>>>>>>>> besides having
> >>>>>>>>>>>>>>        all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>>>>>> missing your
> >>>>>>>>>>>>>>        point?
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > ---
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > Earlier in this discussion John Roesler also
> >>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>        without "start branching" operator, and later Paul
> >>>>>>>>>>>>>> mentioned that in
> >>>>>>>>>>>>>>        the case when we have to add a dynamic number of
> >>>>>>>>>>>>>> branches, the
> >>>>>>>>>>>>>>        current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>>>>>> solution. Let
> >>>>>>>>>>>>>>        me address both comments here.
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > 1) "Start branching" operator (I think that
> >>>>>>>>>>>>>> *split* is a
> >>>>>>>>>>>>>> good name
> >>>>>>>>>>>>>>        for it indeed) is critical when we need to do a
> >>>>>>>>>>>>>> dynamic
> >>>>>>>>>>>>>> branching,
> >>>>>>>>>>>>>>        see example below.
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > 2) No, dynamic branching in current KIP is not
> >>>>>>>>>>>>>> clumsy at
> >>>>>>>>>>>>>> all.
> >>>>>>>>>>>>>>        Imagine a real-world scenario when you need one
> >>>>>>>>>>>>>> branch per
> >>>>>>>>>>>>>> enum
> >>>>>>>>>>>>>>        value (say, RecordType). You can have something
> >>>>>>>>>>>>>> like this:
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > /*John:if we had to start with stream.branch(...)
> >>>>>>>>>>>>>> here,
> >>>>>>>>>>>>>> it would
> >>>>>>>>>>>>>>        have been much messier.*/
> >>>>>>>>>>>>>>        > KBranchedStream branched = stream.split();
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > /*Not clumsy at all :-)*/
> >>>>>>>>>>>>>>        > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>>>>>        >             branched = branched.branch((k, v) ->
> >>>>>>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>>>>>        recordType,
> >>>>>>>>>>>>>>        >                     recordType::processRecords);
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > Regards,
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > Ivan
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>        > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>        >> I also agree with Michael's observation about
> >>>>>>>>>>>>>> the core
> >>>>>>>>>>>>>> problem of
> >>>>>>>>>>>>>>        >> current `branch()` implementation.
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>>>>>> object. My
> >>>>>>>>>>>>>>        thinking
> >>>>>>>>>>>>>>        >> was more aligned with Paul's proposal to just
> >>>>>>>>>>>>>> add a name
> >>>>>>>>>>>>>> to each
> >>>>>>>>>>>>>>        >> `branch()` statement and return a
> >>>>>>>>>>>>>> `Map<String,KStream>`.
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >> It makes the code easier to read, and also make the
> >>>>>>>>>>>>>> order of
> >>>>>>>>>>>>>>        >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >>>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>        >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>        >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>        >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>>>>>        >> An open question is the case for which no
> >>>>>>>>>>>>>> defaultBranch() should
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>        >> specified. Atm, `split()` and `branch()` would
> >>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>        `BranchedKStream`
> >>>>>>>>>>>>>>        >> and the call to `defaultBranch()` that returns the
> >>>>>>>>>>>>>> `Map` is
> >>>>>>>>>>>>> mandatory
> >>>>>>>>>>>>>>        >> (what is not the case atm). Or is this actually
> >>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>> real
> >>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>        >> because users can just ignore the branch
> >>>>>>>>>>>>>> returned by
> >>>>>>>>>>>>>>        `defaultBranch()`
> >>>>>>>>>>>>>>        >> in the result `Map` ?
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >> About "inlining": So far, it seems to be a
> >>>>>>>>>>>>>> matter of
> >>>>>>>>>>>>>> personal
> >>>>>>>>>>>>>>        >> preference. I can see arguments for both, but no
> >>>>>>>>>>>>>> "killer
> >>>>>>>>>>>>>>        argument" yet
> >>>>>>>>>>>>>>        >> that clearly make the case for one or the other.
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >> -Matthias
> >>>>>>>>>>>>>>        >>
> >>>>>>>>>>>>>>        >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>        >>> Perhaps inlining is the wrong terminology. It
> >>>>>>>>>>>>>> doesn’t
> >>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>        that a lambda with the full downstream topology be
> >>>>>>>>>>>>>> defined
> >>>>>>>>>>>>>> inline -
> >>>>>>>>>>>>>>        it can be a method reference as with Ivan’s original
> >>>>>>>>>>>>>> suggestion.
> >>>>>>>>>>>>>>        The advantage of putting the predicate and its
> >>>>>>>>>>>>>> downstream
> >>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>>        (Consumer) together in branch() is that they are
> >>>>>>>>>>>>>> required
> >>>>>>>>>>>>>> to be near
> >>>>>>>>>>>>>>        to each other.
> >>>>>>>>>>>>>>        >>>
> >>>>>>>>>>>>>>        >>> Ultimately the downstream code has to live
> >>>>>>>>>>>>>> somewhere,
> >>>>>>>>>>>>>> and deep
> >>>>>>>>>>>>>>        branch trees will be hard to read regardless.
> >>>>>>>>>>>>>>        >>>
> >>>>>>>>>>>>>>        >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>>>>>        <michael.drogalis@confluent.io
> >>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>        >>>>
> >>>>>>>>>>>>>>        >>>> I'm less enthusiastic about inlining the
> >>>>>>>>>>>>>> branch logic
> >>>>>>>>>>>>>> with its
> >>>>>>>>>>>>>>        downstream
> >>>>>>>>>>>>>>        >>>> functionality. Programs that have deep branch
> >>>>>>>>>>>>>> trees
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>        quickly become
> >>>>>>>>>>>>>>        >>>> harder to read as a single unit.
> >>>>>>>>>>>>>>        >>>>
> >>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>>>>>        <pgwhalen@gmail.com <ma...@gmail.com>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Also +1 on the issues/goals as Michael
> >>>>>>>>>>>>>> outlined them,
> >>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>>        that sets a
> >>>>>>>>>>>>>>        >>>>> great framework for the discussion.
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Regarding the SortedMap solution, my
> >>>>>>>>>>>>>> understanding is
> >>>>>>>>>>>>>> that the
> >>>>>>>>>>>>>>        current
> >>>>>>>>>>>>>>        >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>>>>>> (pending naming
> >>>>>>>>>>>>>>        decisions) is
> >>>>>>>>>>>>>>        >>>>> roughly this:
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> stream.split()
> >>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>,
> >>>>>>>>>>>>>> Consumer<KStream<K, V>>)
> >>>>>>>>>>>>>>        >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Obviously some ordering is necessary, since
> >>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>        construct
> >>>>>>>>>>>>>>        >>>>> doesn't work without it, but this solution seems
> >>>>>>>>>>>>>> like it
> >>>>>>>>>>>>>>        provides as much
> >>>>>>>>>>>>>>        >>>>> associativity as the SortedMap solution,
> >>>>>>>>>>>>>> because each
> >>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>        call
> >>>>>>>>>>>>>>        >>>>> directly associates the "conditional" with
> >>>>>>>>>>>>>> the "code
> >>>>>>>>>>>>>> block."
> >>>>>>>>>>>>>>        The value it
> >>>>>>>>>>>>>>        >>>>> provides over the KIP solution is the
> >>>>>>>>>>>>>> accessing of
> >>>>>>>>>>>>>> streams in
> >>>>>>>>>>>>>>        the same
> >>>>>>>>>>>>>>        >>>>> scope.
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> The KIP solution is less "dynamic" than the
> >>>>>>>>>>>>>> SortedMap
> >>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>        in the sense
> >>>>>>>>>>>>>>        >>>>> that it is slightly clumsier to add a dynamic
> >>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>        branches, but it is
> >>>>>>>>>>>>>>        >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>>>>>> should favor
> >>>>>>>>>>>>>>        the "static"
> >>>>>>>>>>>>>>        >>>>> case anyway, and should make it simple and
> >>>>>>>>>>>>>> readable to
> >>>>>>>>>>>>>>        fluently declare and
> >>>>>>>>>>>>>>        >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>>>>>> impossible to
> >>>>>>>>>>>>>>        ignore a
> >>>>>>>>>>>>>>        >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>>>>>> identical
> >>>>>>>>>>>>>>        SortedMap
> >>>>>>>>>>>>>>        >>>>> solution on top of it.
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> I could also see a middle ground where
> >>>>>>>>>>>>>> instead of
> >>>>>>>>>>>>>> a raw
> >>>>>>>>>>>>>>        SortedMap being
> >>>>>>>>>>>>>>        >>>>> taken in, branch() takes a name and not a
> >>>>>>>>>>>>>> Consumer.
> >>>>>>>>>>>>>> Something
> >>>>>>>>>>>>>>        like this:
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Map<String, KStream<K, V>> branches =
> >>>>>>>>>>>>>> stream.split()
> >>>>>>>>>>>>>>        >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>>>>>        >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>>>>>        >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Pros for that solution:
> >>>>>>>>>>>>>>        >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>>>>>        >>>>> - no double brace initialization, hopefully
> >>>>>>>>>>>>>> slightly
> >>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>        readable than
> >>>>>>>>>>>>>>        >>>>> SortedMap
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Cons
> >>>>>>>>>>>>>>        >>>>> - downstream branch logic cannot be specified
> >>>>>>>>>>>>>> inline
> >>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>        makes it harder
> >>>>>>>>>>>>>>        >>>>> to read top to bottom (like existing API and
> >>>>>>>>>>>>>> SortedMap, but
> >>>>>>>>>>>>>>        unlike the KIP)
> >>>>>>>>>>>>>>        >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>>>>>> streams (like
> >>>>>>>>>>>>>>        existing
> >>>>>>>>>>>>>>        >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> (KBranchedStreams could even work *both* ways
> >>>>>>>>>>>>>> but
> >>>>>>>>>>>>>> perhaps
> >>>>>>>>>>>>>>        that's overdoing
> >>>>>>>>>>>>>>        >>>>> it).
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Overall I'm curious how important it is to be
> >>>>>>>>>>>>>> able to
> >>>>>>>>>>>>>> easily
> >>>>>>>>>>>>>>        access the
> >>>>>>>>>>>>>>        >>>>> branched KStream in the same scope as the
> >>>>>>>>>>>>>> original.
> >>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>        possible that it
> >>>>>>>>>>>>>>        >>>>> doesn't need to be handled directly by the
> >>>>>>>>>>>>>> API, but
> >>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>        left up to the
> >>>>>>>>>>>>>>        >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> Paul
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
> >>>>>>>>>>>>>> Blee-Goldman
> >>>>>>>>>>>>>>        <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>        >>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>        >>>>>> I'd like to +1 what Michael said about the
> >>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>        existing
> >>>>>>>>>>>>>>        >>>>> branch
> >>>>>>>>>>>>>>        >>>>>> method, I agree with what he's outlined and
> >>>>>>>>>>>>>> I think
> >>>>>>>>>>>>>> we should
> >>>>>>>>>>>>>>        proceed by
> >>>>>>>>>>>>>>        >>>>>> trying to alleviate these problems.
> >>>>>>>>>>>>>> Specifically it
> >>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>        important to be
> >>>>>>>>>>>>>>        >>>>>> able to cleanly access the individual
> >>>>>>>>>>>>>> branches (eg
> >>>>>>>>>>>>>> by mapping
> >>>>>>>>>>>>>>        >>>>>> name->stream), which I thought was the original
> >>>>>>>>>>>>>> intention of
> >>>>>>>>>>>>>>        this KIP.
> >>>>>>>>>>>>>>        >>>>>>
> >>>>>>>>>>>>>>        >>>>>> That said, I don't think we should so easily
> >>>>>>>>>>>>>> give in
> >>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>        double brace
> >>>>>>>>>>>>>>        >>>>>> anti-pattern or force ours users into it if
> >>>>>>>>>>>>>> at all
> >>>>>>>>>>>>>> possible to
> >>>>>>>>>>>>>>        >>>>> avoid...just
> >>>>>>>>>>>>>>        >>>>>> my two cents.
> >>>>>>>>>>>>>>        >>>>>>
> >>>>>>>>>>>>>>        >>>>>> Cheers,
> >>>>>>>>>>>>>>        >>>>>> Sophie
> >>>>>>>>>>>>>>        >>>>>>
> >>>>>>>>>>>>>>        >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
> >>>>>>>>>>>>>> Drogalis <
> >>>>>>>>>>>>>>        >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
> >>>>>>>>>>>>>>        >>>>>>
> >>>>>>>>>>>>>>        >>>>>>> I’d like to propose a different way of
> >>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>> about this.
> >>>>>>>>>>>>>>        To me,
> >>>>>>>>>>>>>>        >>>>> there
> >>>>>>>>>>>>>>        >>>>>>> are three problems with the existing branch
> >>>>>>>>>>>>>> signature:
> >>>>>>>>>>>>>>        >>>>>>>
> >>>>>>>>>>>>>>        >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>>>>>> raises unsafe
> >>>>>>>>>>>>> type
> >>>>>>>>>>>>>>        >>>>>> warnings.
> >>>>>>>>>>>>>>        >>>>>>> 2. The way in which you use the stream
> >>>>>>>>>>>>>> branches is
> >>>>>>>>>>>>>>        positionally coupled
> >>>>>>>>>>>>>>        >>>>>> to
> >>>>>>>>>>>>>>        >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>>>>>        >>>>>>> 3. It is brittle to extend existing branch
> >>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>        additional code
> >>>>>>>>>>>>>>        >>>>>>> paths.
> >>>>>>>>>>>>>>        >>>>>>>
> >>>>>>>>>>>>>>        >>>>>>> Using associative constructs instead of
> >>>>>>>>>>>>>> relying on
> >>>>>>>>>>>>>> ordered
> >>>>>>>>>>>>>>        constructs
> >>>>>>>>>>>>>>        >>>>>> would
> >>>>>>>>>>>>>>        >>>>>>> be a stronger approach. Consider a
> >>>>>>>>>>>>>> signature that
> >>>>>>>>>>>>>> instead
> >>>>>>>>>>>>>>        looks like
> >>>>>>>>>>>>>>        >>>>>> this:
> >>>>>>>>>>>>>>        >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>>>>>        Predicate<?
> >>>>>>>>>>>>>>        >>>>>>> super K,? super V>>);
> >>>>>>>>>>>>>>        >>>>>>>
> >>>>>>>>>>>>>>        >>>>>>> Branches are given names in a map, and as a
> >>>>>>>>>>>>>> result,
> >>>>>>>>>>>>>> the API
> >>>>>>>>>>>>>>        returns a
> >>>>>>>>>>>>>>        >>>>>>> mapping of names to streams. The ordering
> >>>>>>>>>>>>>> of the
> >>>>>>>>>>>>> conditionals is
> >>>>>>>>>>>>>>        >>>>>> maintained
> >>>>>>>>>>>>>>        >>>>>>> because it’s a sorted map. Insert order
> >>>>>>>>>>>>>> determines
> >>>>>>>>>>>>>> the order
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>        >>>>>> evaluation.
> >>>>>>>>>>>>>>        >>>>>>> This solves problem 1 because there are no
> >>>>>>>>>>>>>> more
> >>>>>>>>>>>>>> varargs. It
> >>>>>>>>>>>>>>        solves
> >>>>>>>>>>>>>>        >>>>>> problem
> >>>>>>>>>>>>>>        >>>>>>> 2 because you no longer lean on ordering to
> >>>>>>>>>>>>>> access the
> >>>>>>>>>>>>>>        branch you’re
> >>>>>>>>>>>>>>        >>>>>>> interested in. It solves problem 3 because
> >>>>>>>>>>>>>> you can
> >>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>        another
> >>>>>>>>>>>>>>        >>>>>>> conditional by simply attaching another
> >>>>>>>>>>>>>> name to the
> >>>>>>>>>>>>>>        structure, rather
> >>>>>>>>>>>>>>        >>>>>> than
> >>>>>>>>>>>>>>        >>>>>>> messing with the existing indices.
> >>>>>>>>>>>>>>        >>>>>>>
> >>>>>>>>>>>>>>        >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>>>>>> inline is
> >>>>>>>>>>>>>>        historically
> >>>>>>>>>>>>>>        >>>>>>> awkward in Java. I know it’s an
> >>>>>>>>>>>>>> anti-pattern to use
> >>>>>>>>>>>>>>        voluminously, but
> >>>>>>>>>>>>>>        >>>>>>> double brace initialization would clean up the
> >>>>>>>>>>>>>> aesthetics.
> >>>>>>>>>>>>>>        >>>>>>>
> >>>>>>>>>>>>>>        >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>>>>>        <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>>>>>        >>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>> Thanks for the update.
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>> FWIW, I agree with Matthias that the current
> >>>>>>>>>>>>>> "start
> >>>>>>>>>>>>> branching"
> >>>>>>>>>>>>>>        >>>>> operator
> >>>>>>>>>>>>>>        >>>>>>> is
> >>>>>>>>>>>>>>        >>>>>>>> confusing when named the same way as the
> >>>>>>>>>>>>>> actual
> >>>>>>>>>>>>>> branches.
> >>>>>>>>>>>>>>        "Split"
> >>>>>>>>>>>>>>        >>>>> seems
> >>>>>>>>>>>>>>        >>>>>>>> like a good name. Alternatively, we can do
> >>>>>>>>>>>>>> without
> >>>>>>>>>>>>>> a "start
> >>>>>>>>>>>>>>        >>>>> branching"
> >>>>>>>>>>>>>>        >>>>>>>> operator at all, and just do:
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>> stream
> >>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>        >>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>> Tentatively, I think that this branching
> >>>>>>>>>>>>>> operation
> >>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>        >>>>> terminal.
> >>>>>>>>>>>>>>        >>>>>>> That
> >>>>>>>>>>>>>>        >>>>>>>> way, we don't create ambiguity about how
> >>>>>>>>>>>>>> to use
> >>>>>>>>>>>>>> it. That
> >>>>>>>>>>>>>>        is, `branch`
> >>>>>>>>>>>>>>        >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>>>>>        `void`, to
> >>>>>>>>>>>>>>        >>>>>>>> enforce that it comes last, and that there
> >>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>> one
> >>>>>>>>>>>>>>        definition of
> >>>>>>>>>>>>>>        >>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>>>>>> warning if
> >>>>>>>>>>>>>>        there's no
> >>>>>>>>>>>>>>        >>>>>>> default,
> >>>>>>>>>>>>>>        >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>>>>>> exception) if a
> >>>>>>>>>>>>>>        record
> >>>>>>>>>>>>>>        >>>>>> falls
> >>>>>>>>>>>>>>        >>>>>>>> though with no default.
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>> Thoughts?
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>> Thanks,
> >>>>>>>>>>>>>>        >>>>>>>> -John
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
> >>>>>>>>>>>>>> J. Sax <
> >>>>>>>>>>>>>>        >>>>> matthias@confluent.io
> >>>>>>>>>>>>>> <ma...@confluent.io>
> >>>>>>>>>>>>>>        >>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> Thanks for updating the KIP and your
> >>>>>>>>>>>>>> answers.
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> this is to make the name similar to
> >>>>>>>>>>>>>> String#split
> >>>>>>>>>>>>>>        >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>>>>>        >>>>>>>>> The intend was to avoid name duplication.
> >>>>>>>>>>>>>> The
> >>>>>>>>>>>>>> return type
> >>>>>>>>>>>>>>        should
> >>>>>>>>>>>>>>        >>>>>> _not_
> >>>>>>>>>>>>>>        >>>>>>>>> be an array.
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> The current proposal is
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> IMHO, this reads a little odd, because
> >>>>>>>>>>>>>> the first
> >>>>>>>>>>>>>>        `branch()` does
> >>>>>>>>>>>>>>        >>>>> not
> >>>>>>>>>>>>>>        >>>>>>>>> take any parameters and has different
> >>>>>>>>>>>>>> semantics
> >>>>>>>>>>>>>> than the
> >>>>>>>>>>>>> later
> >>>>>>>>>>>>>>        >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>>>>>> snippet above,
> >>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>        >>>>> hidden
> >>>>>>>>>>>>>>        >>>>>>>>> that the first call is `KStream#branch()`
> >>>>>>>>>>>>>> while
> >>>>>>>>>>>>>> the others
> >>>>>>>>>>>>> are
> >>>>>>>>>>>>>>        >>>>>>>>> `KBranchedStream#branch()` what makes
> >>>>>>>>>>>>>> reading the
> >>>>>>>>>>>>>> code
> >>>>>>>>>>>>> harder.
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> Because I suggested to rename
> >>>>>>>>>>>>>> `addBranch()` ->
> >>>>>>>>>>>>>> `branch()`,
> >>>>>>>>>>>>>>        I though
> >>>>>>>>>>>>>>        >>>>>> it
> >>>>>>>>>>>>>>        >>>>>>>>> might be better to also rename
> >>>>>>>>>>>>>> `KStream#branch()`
> >>>>>>>>>>>>>> to avoid
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>> naming
> >>>>>>>>>>>>>>        >>>>>>>>> overlap that seems to be confusing. The
> >>>>>>>>>>>>>> following
> >>>>>>>>>>>>>> reads
> >>>>>>>>>>>>> much
> >>>>>>>>>>>>>>        >>>>> cleaner
> >>>>>>>>>>>>>>        >>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>> me:
> >>>>>>>>>>>>>>        >>>>>>>>> stream.split()
> >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> Maybe there is a better alternative to
> >>>>>>>>>>>>>> `split()`
> >>>>>>>>>>>>>> though to
> >>>>>>>>>>>>>>        avoid
> >>>>>>>>>>>>>>        >>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> naming overlap.
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>>>>>> unfortunately
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>        >>>>> cannot
> >>>>>>>>>>>>>>        >>>>>>> have
> >>>>>>>>>>>>>>        >>>>>>>>> a method with such name :-)
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> Bummer. Didn't consider this. Maybe we
> >>>>>>>>>>>>>> can still
> >>>>>>>>>>>>>> come up
> >>>>>>>>>>>>>>        with a
> >>>>>>>>>>>>>>        >>>>> short
> >>>>>>>>>>>>>>        >>>>>>>> name?
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> Can you add the interface
> >>>>>>>>>>>>>> `KBranchedStream` to
> >>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>>        with all
> >>>>>>>>>>>>>>        >>>>> it's
> >>>>>>>>>>>>>>        >>>>>>>>> methods? It will be part of public API and
> >>>>>>>>>>>>>> should be
> >>>>>>>>>>>>>>        contained in
> >>>>>>>>>>>>>>        >>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>>>>>> return type of
> >>>>>>>>>>>>>>        >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>>>>>        `KBranchedStream#get(int
> >>>>>>>>>>>>>>        >>>>>>> index)
> >>>>>>>>>>>>>>        >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>>>>>        branched-KStreams. Would
> >>>>>>>>>>>>>>        >>>>>> be
> >>>>>>>>>>>>>>        >>>>>>>>> nice to get your feedback about it. It
> >>>>>>>>>>>>>> seems you
> >>>>>>>>>>>>>> suggest
> >>>>>>>>>>>>>>        that users
> >>>>>>>>>>>>>>        >>>>>>>>> would need to write custom utility code
> >>>>>>>>>>>>>> otherwise, to
> >>>>>>>>>>>>>>        access them.
> >>>>>>>>>>>>>>        >>>>> We
> >>>>>>>>>>>>>>        >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>>>>>> approaches. It
> >>>>>>>>>>>>> feels
> >>>>>>>>>>>>>>        >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>>>>>> built-in support
> >>>>>>>>>>>>>>        to get
> >>>>>>>>>>>>>>        >>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>> -Matthias
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>> Hi all!
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> I have updated the KIP-418 according to
> >>>>>>>>>>>>>> the new
> >>>>>>>>>>>>>> vision.
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>>>>>        >>>>>>>>>> I can see your point: this is to make
> >>>>>>>>>>>>>> the name
> >>>>>>>>>>>>>> similar to
> >>>>>>>>>>>>>>        >>>>>>> String#split
> >>>>>>>>>>>>>>        >>>>>>>>>> that also returns an array, right? But
> >>>>>>>>>>>>>> is it
> >>>>>>>>>>>>>> worth the
> >>>>>>>>>>>>>>        loss of
> >>>>>>>>>>>>>>        >>>>>>>> backwards
> >>>>>>>>>>>>>>        >>>>>>>>>> compatibility? We can have overloaded
> >>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>> as well
> >>>>>>>>>>>>>>        without
> >>>>>>>>>>>>>>        >>>>>>>> affecting
> >>>>>>>>>>>>>>        >>>>>>>>>> the existing code. Maybe the old
> >>>>>>>>>>>>>> array-based
> >>>>>>>>>>>>>> `branch`
> >>>>>>>>>>>>> method
> >>>>>>>>>>>>>>        >>>>> should
> >>>>>>>>>>>>>>        >>>>>>> be
> >>>>>>>>>>>>>>        >>>>>>>>>> deprecated, but this is a subject for
> >>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>>        >>>>> BranchingKStream#branch(),
> >>>>>>>>>>>>>>        >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> Totally agree with 'addBranch->branch'
> >>>>>>>>>>>>>> rename.
> >>>>>>>>>>>>>> 'default'
> >>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>        >>>>>>> however, a
> >>>>>>>>>>>>>>        >>>>>>>>>> reserved word, so unfortunately we
> >>>>>>>>>>>>>> cannot have a
> >>>>>>>>>>>>>> method
> >>>>>>>>>>>>>>        with such
> >>>>>>>>>>>>>>        >>>>>>> name
> >>>>>>>>>>>>>>        >>>>>>>>> :-)
> >>>>>>>>>>>>>>        >>>>>>>>>>> defaultBranch() does take an
> >>>>>>>>>>>>>> `Predicate` as
> >>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>        >>>>> think
> >>>>>>>>>>>>>>        >>>>>>> that
> >>>>>>>>>>>>>>        >>>>>>>>>> is not required?
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> Absolutely! I think that was just
> >>>>>>>>>>>>>> copy-paste
> >>>>>>>>>>>>>> error or
> >>>>>>>>>>>>>>        something.
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> please revise the new version of the KIP
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>> Paul's PR
> >>>>>>>>>>>>>>        >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> Ivan
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>> Thanks for driving the discussion of
> >>>>>>>>>>>>>> this KIP.
> >>>>>>>>>>>>>> It seems
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>        >>>>>>> everybody
> >>>>>>>>>>>>>>        >>>>>>>>>>> agrees that the current branch() method
> >>>>>>>>>>>>>> using
> >>>>>>>>>>>>>> arrays is
> >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>        >>>>>> optimal.
> >>>>>>>>>>>>>>        >>>>>>>>>>> I had a quick look into the PR and I
> >>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>        proposal.
> >>>>>>>>>>>>>>        >>>>>>> There
> >>>>>>>>>>>>>>        >>>>>>>>>>> are some minor things we need to
> >>>>>>>>>>>>>> consider. I
> >>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>        recommend the
> >>>>>>>>>>>>>>        >>>>>>>>>>> following renaming:
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>>>>>        BranchingKStream#default()
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> It's just a suggestion to get slightly
> >>>>>>>>>>>>>> shorter
> >>>>>>>>>>>>>> method
> >>>>>>>>>>>>> names.
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> In the current PR, defaultBranch() does
> >>>>>>>>>>>>>> take an
> >>>>>>>>>>>>>>        `Predicate` as
> >>>>>>>>>>>>>>        >>>>>>>> argument,
> >>>>>>>>>>>>>>        >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>>>>>> recently
> >>>>>>>>>>>>>>        accepted and
> >>>>>>>>>>>>>>        >>>>>> is
> >>>>>>>>>>>>>>        >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> Ie, we should add overloads that
> >>>>>>>>>>>>>> accepted a
> >>>>>>>>>>>>>> `Named`
> >>>>>>>>>>>>>>        parameter.
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> For the issue that the created
> >>>>>>>>>>>>>> `KStream` object
> >>>>>>>>>>>>>> are in
> >>>>>>>>>>>>>>        different
> >>>>>>>>>>>>>>        >>>>>>>> scopes:
> >>>>>>>>>>>>>>        >>>>>>>>>>> could we extend `KBranchedStream` with a
> >>>>>>>>>>>>>> `get(int
> >>>>>>>>>>>>>>        index)` method
> >>>>>>>>>>>>>>        >>>>>>> that
> >>>>>>>>>>>>>>        >>>>>>>>>>> returns the corresponding "branched"
> >>>>>>>>>>>>>> result
> >>>>>>>>>>>>>> `KStream`
> >>>>>>>>>>>>>>        object?
> >>>>>>>>>>>>>>        >>>>>> Maybe,
> >>>>>>>>>>>>>>        >>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>> second argument of `addBranch()` should
> >>>>>>>>>>>>>> not
> >>>>>>>>>>>>>> be a
> >>>>>>>>>>>>>>        >>>>>> `Consumer<KStream>`
> >>>>>>>>>>>>>>        >>>>>>>> but
> >>>>>>>>>>>>>>        >>>>>>>>>>> a `Function<KStream,KStream>` and
> >>>>>>>>>>>>>> `get()` could
> >>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>        whatever
> >>>>>>>>>>>>>>        >>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> Finally, I would also suggest to update
> >>>>>>>>>>>>>> the KIP
> >>>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>        current
> >>>>>>>>>>>>>>        >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>        makes sense
> >>>>>>>>>>>>>>        >>>>>> for
> >>>>>>>>>>>>>>        >>>>>>>> you
> >>>>>>>>>>>>>>        >>>>>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>>>>> revise the KIP and continue the
> >>>>>>>>>>>>>> discussion.
> >>>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>        we'll
> >>>>>>>>>>>>>>        >>>>> need
> >>>>>>>>>>>>>>        >>>>>>>> some
> >>>>>>>>>>>>>>        >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>>>>>> binding votes on
> >>>>>>>>>>>>>>        >>>>> whether
> >>>>>>>>>>>>>>        >>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> KIP
> >>>>>>>>>>>>>>        >>>>>>>>>>>> could be adopted.  It would be great
> >>>>>>>>>>>>>> to hear
> >>>>>>>>>>>>>> if they
> >>>>>>>>>>>>>>        think this
> >>>>>>>>>>>>>>        >>>>>> is
> >>>>>>>>>>>>>>        >>>>>>> a
> >>>>>>>>>>>>>>        >>>>>>>>> good
> >>>>>>>>>>>>>>        >>>>>>>>>>>> idea overall.  I'm not sure if that
> >>>>>>>>>>>>>> happens
> >>>>>>>>>>>>>> just by
> >>>>>>>>>>>>>>        starting a
> >>>>>>>>>>>>>>        >>>>>>> vote,
> >>>>>>>>>>>>>>        >>>>>>>>> or if
> >>>>>>>>>>>>>>        >>>>>>>>>>>> there is generally some indication of
> >>>>>>>>>>>>>> interest
> >>>>>>>>>>>>> beforehand.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>> That being said, I'll continue the
> >>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>> a bit:
> >>>>>>>>>>>>>>        assuming
> >>>>>>>>>>>>>>        >>>>> we
> >>>>>>>>>>>>>>        >>>>>> do
> >>>>>>>>>>>>>>        >>>>>>>>> move
> >>>>>>>>>>>>>>        >>>>>>>>>>>> forward the solution of "stream.branch()
> >>>>>>>>>>>>>> returns
> >>>>>>>>>>>>>>        >>>>>> KBranchedStream",
> >>>>>>>>>>>>>>        >>>>>>> do
> >>>>>>>>>>>>>>        >>>>>>>>> we
> >>>>>>>>>>>>>>        >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>>>>>> KStream[]"?  I
> >>>>>>>>>>>>> would
> >>>>>>>>>>>>>>        >>>>> favor
> >>>>>>>>>>>>>>        >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>>>>>> exclusive APIs
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>        >>>>>>> accomplish
> >>>>>>>>>>>>>>        >>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>>>>>> they're fairly
> >>>>>>>>>>>>>>        similar
> >>>>>>>>>>>>>>        >>>>>>>>> anyway.  We
> >>>>>>>>>>>>>>        >>>>>>>>>>>> just need to be sure we're not making
> >>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>        >>>>>>> impossible/difficult
> >>>>>>>>>>>>>>        >>>>>>>>> that
> >>>>>>>>>>>>>>        >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>>>>>> structure would
> >>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>        >>>>> it's
> >>>>>>>>>>>>>>        >>>>>>>> just a
> >>>>>>>>>>>>>>        >>>>>>>>>>>> little sloppy overall in terms of
> >>>>>>>>>>>>>> naming and
> >>>>>>>>>>>>>> clarity. In
> >>>>>>>>>>>>>>        >>>>>>> particular,
> >>>>>>>>>>>>>>        >>>>>>>>>>>> passing in the "predicates" and
> >>>>>>>>>>>>>> "children"
> >>>>>>>>>>>>>> lists which
> >>>>>>>>>>>>> get
> >>>>>>>>>>>>>>        >>>>>> modified
> >>>>>>>>>>>>>>        >>>>>>>> in
> >>>>>>>>>>>>>>        >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>>>>>        KStreamLazyBranch is
> >>>>>>>>>>>>>>        >>>>> a
> >>>>>>>>>>>>>>        >>>>>>> bit
> >>>>>>>>>>>>>>        >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>        >>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>        >>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
> >>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>        >>>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> I read your code carefully and now I
> >>>>>>>>>>>>>> am fully
> >>>>>>>>>>>>>>        convinced: your
> >>>>>>>>>>>>>>        >>>>>>>> proposal
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> looks better and should work. We just
> >>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>> document
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>> crucial
> >>>>>>>>>>>>>>        >>>>>>>>> fact
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> that KStream consumers are invoked as
> >>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>> added.
> >>>>>>>>>>>>>>        And then
> >>>>>>>>>>>>>>        >>>>>> it's
> >>>>>>>>>>>>>>        >>>>>>>> all
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> What shall we do now? I should
> >>>>>>>>>>>>>> re-write the
> >>>>>>>>>>>>>> KIP and
> >>>>>>>>>>>>>>        resume the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> Why are you telling that your PR
> >>>>>>>>>>>>>> 'should not
> >>>>>>>>>>>>>> be even a
> >>>>>>>>>>>>>>        >>>>> starting
> >>>>>>>>>>>>>>        >>>>>>>> point
> >>>>>>>>>>>>>>        >>>>>>>>> if
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> we go in this direction'? To me it
> >>>>>>>>>>>>>> looks like
> >>>>>>>>>>>>>> a good
> >>>>>>>>>>>>>>        starting
> >>>>>>>>>>>>>>        >>>>>>> point.
> >>>>>>>>>>>>>>        >>>>>>>>> But
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> as a novice in this project I might
> >>>>>>>>>>>>>> miss some
> >>>>>>>>>>>>>> important
> >>>>>>>>>>>>>>        >>>>> details.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>> Maybe I’m missing the point, but I
> >>>>>>>>>>>>>> believe the
> >>>>>>>>>>>>>>        >>>>> stream.branch()
> >>>>>>>>>>>>>>        >>>>>>>>> solution
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>>>>>> consumers will be
> >>>>>>>>>>>>>>        >>>>> invoked
> >>>>>>>>>>>>>>        >>>>>> as
> >>>>>>>>>>>>>>        >>>>>>>>> they’re
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> added, not during
> >>>>>>>>>>>>>> streamsBuilder.build(). So
> >>>>>>>>>>>>>> the user
> >>>>>>>>>>>>>>        still
> >>>>>>>>>>>>>>        >>>>>> ought
> >>>>>>>>>>>>>>        >>>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>> be
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> able to call couponIssuer.coupons()
> >>>>>>>>>>>>>> afterward
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>        depend on
> >>>>>>>>>>>>>>        >>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> branched
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>> The issue I mean to point out is
> >>>>>>>>>>>>>> that it is
> >>>>>>>>>>>>>> hard to
> >>>>>>>>>>>>>>        access
> >>>>>>>>>>>>>>        >>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> branched
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> streams in the same scope as the
> >>>>>>>>>>>>>> original
> >>>>>>>>>>>>>> stream (that
> >>>>>>>>>>>>>>        is, not
> >>>>>>>>>>>>>>        >>>>>>>> inside
> >>>>>>>>>>>>>>        >>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> couponIssuer), which is a problem
> >>>>>>>>>>>>>> with both
> >>>>>>>>>>>>>> proposed
> >>>>>>>>>>>>>>        >>>>> solutions.
> >>>>>>>>>>>>>>        >>>>>> It
> >>>>>>>>>>>>>>        >>>>>>>>> can be
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>> [Also, great to hear additional
> >>>>>>>>>>>>>> interest in
> >>>>>>>>>>>>>> 401, I’m
> >>>>>>>>>>>>>>        excited
> >>>>>>>>>>>>>>        >>>>> to
> >>>>>>>>>>>>>>        >>>>>>>> hear
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
> >>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>>>>>        >>>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> The idea to postpone the wiring of
> >>>>>>>>>>>>>> branches
> >>>>>>>>>>>>>> to the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> streamsBuilder.build() also looked
> >>>>>>>>>>>>>> great for
> >>>>>>>>>>>>>> me at
> >>>>>>>>>>>>> first
> >>>>>>>>>>>>>>        >>>>> glance,
> >>>>>>>>>>>>>>        >>>>>>> but
> >>>>>>>>>>>>>>        >>>>>>>>> ---
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>>>>>> available in the
> >>>>>>>>>>>>>>        same
> >>>>>>>>>>>>>>        >>>>>> scope
> >>>>>>>>>>>>>>        >>>>>>> as
> >>>>>>>>>>>>>>        >>>>>>>>> each
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> other.  That is, if we wanted to merge
> >>>>>>>>>>>>>> them back
> >>>>>>>>>>>>> together
> >>>>>>>>>>>>>>        >>>>> again
> >>>>>>>>>>>>>>        >>>>>> I
> >>>>>>>>>>>>>>        >>>>>>>>> don't see
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> You just took the words right out
> >>>>>>>>>>>>>> of my
> >>>>>>>>>>>>>> mouth, I was
> >>>>>>>>>>>>>>        just
> >>>>>>>>>>>>>>        >>>>>> going
> >>>>>>>>>>>>>>        >>>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Consider the example from Bill's
> >>>>>>>>>>>>>> book, p.
> >>>>>>>>>>>>>> 101: say
> >>>>>>>>>>>>>>        we need
> >>>>>>>>>>>>>>        >>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>> identify
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> customers who have bought coffee and
> >>>>>>>>>>>>>> made a
> >>>>>>>>>>>>>> purchase
> >>>>>>>>>>>>>>        in the
> >>>>>>>>>>>>>>        >>>>>>>>> electronics
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> This is the code I usually write under
> >>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>        circumstances
> >>>>>>>>>>>>>>        >>>>>> using
> >>>>>>>>>>>>>>        >>>>>>>> my
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>> coffePurchases;
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....>
> >>>>>>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       return
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       /*In the real world the code
> >>>>>>>>>>>>>> here
> >>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>        complex, so
> >>>>>>>>>>>>>>        >>>>>>>>> creation of
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>>>>>> justified, in
> >>>>>>>>>>>>>>        order to
> >>>>>>>>>>>>>>        >>>>>>>> separate
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>>>>>> CouponIssuer();
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>>>>>        >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> /*Alas, this won't work if we're
> >>>>>>>>>>>>>> going to
> >>>>>>>>>>>>>> wire up
> >>>>>>>>>>>>>>        everything
> >>>>>>>>>>>>>>        >>>>>>>> later,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Does this make sense?  In order to
> >>>>>>>>>>>>>> properly
> >>>>>>>>>>>>>>        initialize the
> >>>>>>>>>>>>>>        >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> we need the terminal operation to be
> >>>>>>>>>>>>>> called
> >>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>        >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>>>>>> KIP-401 is
> >>>>>>>>>>>>>>        essentially
> >>>>>>>>>>>>>>        >>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> next
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> KIP I was going to write here. I have
> >>>>>>>>>>>>>> some
> >>>>>>>>>>>>>> thoughts
> >>>>>>>>>>>>>>        based on
> >>>>>>>>>>>>>>        >>>>> my
> >>>>>>>>>>>>>>        >>>>>>>>> experience,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> so I will join the discussion on KIP-401
> >>>>>>>>>>>>>> soon.]
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>>>>>> concept of a
> >>>>>>>>>>>>>>        fluent
> >>>>>>>>>>>>>>        >>>>> API
> >>>>>>>>>>>>>>        >>>>>>>> based
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>>>>>        (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>>>>>        >>>>>> and
> >>>>>>>>>>>>>>        >>>>>>> I
> >>>>>>>>>>>>>>        >>>>>>>>> think
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>>>>>> earlier about
> >>>>>>>>>>>>>>        >>>>>>> compatibility
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    there aren't any direct ones. 
> >>>>>>>>>>>>>> I was
> >>>>>>>>>>>>>> unaware
> >>>>>>>>>>>>>>        that Java
> >>>>>>>>>>>>>>        >>>>> is
> >>>>>>>>>>>>>>        >>>>>>>> smart
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    distinguish between a
> >>>>>>>>>>>>>> branch(varargs...)
> >>>>>>>>>>>>>>        returning one
> >>>>>>>>>>>>>>        >>>>>>> thing
> >>>>>>>>>>>>>>        >>>>>>>>> and
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    with no arguments returning
> >>>>>>>>>>>>>> another
> >>>>>>>>>>>>>> thing.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Requiring a terminal method:
> >>>>>>>>>>>>>> We don't
> >>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>        need
> >>>>>>>>>>>>>>        >>>>> it.
> >>>>>>>>>>>>>>        >>>>>>> We
> >>>>>>>>>>>>>>        >>>>>>>>> can
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>>>>>> KBranchedStream who
> >>>>>>>>>>>>>>        shares
> >>>>>>>>>>>>>>        >>>>>> its
> >>>>>>>>>>>>>>        >>>>>>>>> state
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    ProcessorSupplier that will
> >>>>>>>>>>>>>> actually do
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        branching.
> >>>>>>>>>>>>>>        >>>>>>> It's
> >>>>>>>>>>>>>>        >>>>>>>>> not
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    pretty in its current form, but I
> >>>>>>>>>>>>>> think it
> >>>>>>>>>>>>>>        demonstrates
> >>>>>>>>>>>>>>        >>>>>> its
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>>>>>> request should
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>        >>>>> final
> >>>>>>>>>>>>>>        >>>>>> or
> >>>>>>>>>>>>>>        >>>>>>>>> even a
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> starting point if we go in this
> >>>>>>>>>>>>>> direction,
> >>>>>>>>>>>>>> I just
> >>>>>>>>>>>>>>        wanted to
> >>>>>>>>>>>>>>        >>>>>> see
> >>>>>>>>>>>>>>        >>>>>>>> how
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> challenging it would be to get the
> >>>>>>>>>>>>>> API
> >>>>>>>>>>>>>> working.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I will say though, that I'm not
> >>>>>>>>>>>>>> sure the
> >>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>        solution
> >>>>>>>>>>>>>>        >>>>>>> could
> >>>>>>>>>>>>>>        >>>>>>>> be
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecated in favor of this, which
> >>>>>>>>>>>>>> I had
> >>>>>>>>>>>>>> originally
> >>>>>>>>>>>>>>        >>>>> suggested
> >>>>>>>>>>>>>>        >>>>>>>> was a
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> possibility.  The reason is that
> >>>>>>>>>>>>>> the newly
> >>>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>        streams
> >>>>>>>>>>>>>>        >>>>>> are
> >>>>>>>>>>>>>>        >>>>>>>> not
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>>>>>> other.  That
> >>>>>>>>>>>>>>        is, if we
> >>>>>>>>>>>>>>        >>>>>>> wanted
> >>>>>>>>>>>>>>        >>>>>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> them back together again I don't
> >>>>>>>>>>>>>> see a way
> >>>>>>>>>>>>>> to do
> >>>>>>>>>>>>>>        that.  The
> >>>>>>>>>>>>>>        >>>>>> KIP
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>>>>>> means is that
> >>>>>>>>>>>>> for
> >>>>>>>>>>>>>>        >>>>>> either
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecating the existing
> >>>>>>>>>>>>>> branch(...) is
> >>>>>>>>>>>>>> not on the
> >>>>>>>>>>>>>>        table.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
> >>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>> <ma...@mail.ru>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>>>>>> discussed up to
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>        >>>>>> point.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> First, it seems that it's
> >>>>>>>>>>>>>> commonly agreed
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>        branch API
> >>>>>>>>>>>>>>        >>>>>>> needs
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> improvement. Motivation is given in
> >>>>>>>>>>>>>> the KIP.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> There are two potential ways to
> >>>>>>>>>>>>>> do it:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>   
> >>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
> >>>>>>>>>>>>>> //onTopOf
> >>>>>>>>>>>>>>        returns
> >>>>>>>>>>>>>>        >>>>>> its
> >>>>>>>>>>>>>>        >>>>>>>>> argument
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
> >>>>>>>>>>>>>> compatible. 2)
> >>>>>>>>>>>>>> The code
> >>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>        >>>>> make
> >>>>>>>>>>>>>>        >>>>>>>> sense
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> all the necessary ingredients are
> >>>>>>>>>>>>>> provided.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>        instance
> >>>>>>>>>>>>>>        >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>>>>>> noDefault(). Both
> >>>>>>>>>>>>>>        >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: Generally follows the way
> >>>>>>>>>>>>>> KStreams
> >>>>>>>>>>>>>> interface
> >>>>>>>>>>>>> is
> >>>>>>>>>>>>>>        >>>>>> defined.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
> >>>>>>>>>>>>>> methods
> >>>>>>>>>>>>>>        >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>>>>>        >>>>>>>>> and
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault()). And for a user it
> >>>>>>>>>>>>>> is very
> >>>>>>>>>>>>>> easy to
> >>>>>>>>>>>>>>        miss the
> >>>>>>>>>>>>>>        >>>>>> fact
> >>>>>>>>>>>>>>        >>>>>>>>> that one
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> of the terminal methods should be
> >>>>>>>>>>>>>> called.
> >>>>>>>>>>>>>> If these
> >>>>>>>>>>>>>>        methods
> >>>>>>>>>>>>>>        >>>>>> are
> >>>>>>>>>>>>>>        >>>>>>>> not
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>>>>>> runtime.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Colleagues, what are your
> >>>>>>>>>>>>>> thoughts? Can
> >>>>>>>>>>>>>> we do
> >>>>>>>>>>>>> better?
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
> >>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
> >>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> I see your point when you are
> >>>>>>>>>>>>>> talking
> >>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
> >>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>>        implemented the
> >>>>>>>>>>>>>>        >>>>>>> easy
> >>>>>>>>>>>>>>        >>>>>>>>> way.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Let me comment on two of your
> >>>>>>>>>>>>>> ideas.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> user could specify a terminal
> >>>>>>>>>>>>>> method that
> >>>>>>>>>>>>> assumes
> >>>>>>>>>>>>>>        >>>>> nothing
> >>>>>>>>>>>>>>        >>>>>>>> will
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> throwing an exception if such a
> >>>>>>>>>>>>>> case
> >>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
> >>>>>>>>>>>>>> not be
> >>>>>>>>>>>>>> the only
> >>>>>>>>>>>>> option
> >>>>>>>>>>>>>>        >>>>>> besides
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> `default`, because there are
> >>>>>>>>>>>>>> scenarios
> >>>>>>>>>>>>>> when we
> >>>>>>>>>>>>>>        want to
> >>>>>>>>>>>>>>        >>>>>> just
> >>>>>>>>>>>>>>        >>>>>>>>> silently
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> drop the messages that didn't
> >>>>>>>>>>>>>> match any
> >>>>>>>>>>>>>>        predicate. 2)
> >>>>>>>>>>>>>>        >>>>>>> Throwing
> >>>>>>>>>>>>>>        >>>>>>>>> an
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> exception in the middle of data
> >>>>>>>>>>>>>> flow
> >>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>        looks
> >>>>>>>>>>>>>>        >>>>>> like a
> >>>>>>>>>>>>>>        >>>>>>>> bad
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
> >>>>>>>>>>>>>> I would
> >>>>>>>>>>>>>> prefer to
> >>>>>>>>>>>>>>        emit a
> >>>>>>>>>>>>>>        >>>>>>>> special
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
> >>>>>>>>>>>>>> This is
> >>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>        >>>>>>> `default`
> >>>>>>>>>>>>>>        >>>>>>>>> can
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>        >>>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>> track
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>        a clear
> >>>>>>>>>>>>>>        >>>>>>> error
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
> >>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>> program is
> >>>>>>>>>>>>>>        >>>>> compiled
> >>>>>>>>>>>>>>        >>>>>>> and
> >>>>>>>>>>>>>>        >>>>>>>>> run?
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
> >>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>> won't
> >>>>>>>>>>>>>>        compile if
> >>>>>>>>>>>>>>        >>>>> used
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
> >>>>>>>>>>>>>> API as a
> >>>>>>>>>>>>>>        method chain
> >>>>>>>>>>>>>>        >>>>>>>> starting
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
> >>>>>>>>>>>>>> cost
> >>>>>>>>>>>>>> difference
> >>>>>>>>>>>>>>        between
> >>>>>>>>>>>>>>        >>>>>>>> runtime
> >>>>>>>>>>>>>>        >>>>>>>>> and
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
> >>>>>>>>>>>>>> failure
> >>>>>>>>>>>>>> uncovers
> >>>>>>>>>>>>>>        >>>>> instantly
> >>>>>>>>>>>>>>        >>>>>> on
> >>>>>>>>>>>>>>        >>>>>>>>> unit
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> tests, it costs more for the
> >>>>>>>>>>>>>> project
> >>>>>>>>>>>>>> than a
> >>>>>>>>>>>>>>        compilation
> >>>>>>>>>>>>>>        >>>>>>>> failure.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
> >>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>>>>>> operation being
> >>>>>>>>>>>>>>        required.
> >>>>>>>>>>>>>>        >>>>>>> But
> >>>>>>>>>>>>>>        >>>>>>>> is
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
> >>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>> want a
> >>>>>>>>>>>>>>        >>>>>> defaultBranch
> >>>>>>>>>>>>>>        >>>>>>>>> they
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>>>>>        just as
> >>>>>>>>>>>>>>        >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> think it creates an
> >>>>>>>>>>>>>> opportunity for a
> >>>>>>>>>>>>>> nicer API
> >>>>>>>>>>>>> - a
> >>>>>>>>>>>>>>        >>>>> user
> >>>>>>>>>>>>>>        >>>>>>>> could
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> terminal method that assumes
> >>>>>>>>>>>>>> nothing
> >>>>>>>>>>>>>> will reach
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>> default
> >>>>>>>>>>>>>>        >>>>>>>>> branch,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> throwing an exception if such
> >>>>>>>>>>>>>> a case
> >>>>>>>>>>>>>> occurs.
> >>>>>>>>>>>>> That
> >>>>>>>>>>>>>>        >>>>> seems
> >>>>>>>>>>>>>>        >>>>>>> like
> >>>>>>>>>>>>>>        >>>>>>>>> an
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> improvement over the current
> >>>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>        which allows
> >>>>>>>>>>>>>>        >>>>>> for
> >>>>>>>>>>>>>>        >>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
> >>>>>>>>>>>>>> getting
> >>>>>>>>>>>>> dropped.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>>>>>> certainly has
> >>>>>>>>>>>>>>        to be
> >>>>>>>>>>>>>>        >>>>>> well
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
> >>>>>>>>>>>>>>        >>>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>> track
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> branches that haven't been
> >>>>>>>>>>>>>> terminated
> >>>>>>>>>>>>>> and raise
> >>>>>>>>>>>>>>        a clear
> >>>>>>>>>>>>>>        >>>>>>> error
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
> >>>>>>>>>>>>>> now that
> >>>>>>>>>>>>>> there is
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>        >>>>> "build
> >>>>>>>>>>>>>>        >>>>>>>> step"
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
> >>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>        >>>>>> StreamsBuilder.build()
> >>>>>>>>>>>>>>        >>>>>>> is
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>>>>>> argument, I
> >>>>>>>>>>>>> agree
> >>>>>>>>>>>>>>        >>>>> that
> >>>>>>>>>>>>>>        >>>>>>> it's
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> allow users to do other
> >>>>>>>>>>>>>> operations on
> >>>>>>>>>>>>>> the input
> >>>>>>>>>>>>>>        stream.
> >>>>>>>>>>>>>>        >>>>>>> With
> >>>>>>>>>>>>>>        >>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
> >>>>>>>>>>>>>> same
> >>>>>>>>>>>>>> way all
> >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>        >>>>>>> operations
> >>>>>>>>>>>>>>        >>>>>>>>> do -
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> want to process off the original
> >>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>> multiple
> >>>>>>>>>>>>>>        >>>>> times,
> >>>>>>>>>>>>>>        >>>>>>> you
> >>>>>>>>>>>>>>        >>>>>>>>> just
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> stream as a variable so you
> >>>>>>>>>>>>>> can call
> >>>>>>>>>>>>>> as many
> >>>>>>>>>>>>>>        operations
> >>>>>>>>>>>>>>        >>>>>> on
> >>>>>>>>>>>>>>        >>>>>>> it
> >>>>>>>>>>>>>>        >>>>>>>>> as
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
> >>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>> Ponomarev <
> >>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>> <ma...@mail.ru>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
> >>>>>>>>>>>>>> because we
> >>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>        always need
> >>>>>>>>>>>>>>        >>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
> >>>>>>>>>>>>>> terminal
> >>>>>>>>>>>>> operation we
> >>>>>>>>>>>>>>        >>>>> don't
> >>>>>>>>>>>>>>        >>>>>>>> know
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
> >>>>>>>>>>>>>> switch'.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
> >>>>>>>>>>>>>> returns its
> >>>>>>>>>>>>>> argument,
> >>>>>>>>>>>>>>        so we
> >>>>>>>>>>>>>>        >>>>> can
> >>>>>>>>>>>>>>        >>>>>> do
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> more with the original branch
> >>>>>>>>>>>>>> after
> >>>>>>>>>>>>>> branching.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I understand your point that the
> >>>>>>>>>>>>>> need of
> >>>>>>>>>>>>> special
> >>>>>>>>>>>>>>        >>>>> object
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
> >>>>>>>>>>>>>> KStream
> >>>>>>>>>>>>>> methods.
> >>>>>>>>>>>>> But
> >>>>>>>>>>>>>>        >>>>> here
> >>>>>>>>>>>>>>        >>>>>> we
> >>>>>>>>>>>>>>        >>>>>>>>> have a
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> special case: we build the
> >>>>>>>>>>>>>> switch to
> >>>>>>>>>>>>>> split the
> >>>>>>>>>>>>>>        flow,
> >>>>>>>>>>>>>>        >>>>> so
> >>>>>>>>>>>>>>        >>>>>> I
> >>>>>>>>>>>>>>        >>>>>>>>> think
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
> >>>>>>>>>>>>>> пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
> >>>>>>>>>>>>>> improve
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>        API, but I
> >>>>>>>>>>>>>>        >>>>>> find
> >>>>>>>>>>>>>>        >>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
> >>>>>>>>>>>>>> since it
> >>>>>>>>>>>>>>        contrasts the
> >>>>>>>>>>>>>>        >>>>>>> fluency
> >>>>>>>>>>>>>>        >>>>>>>>> of
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> KStream method calls. 
> >>>>>>>>>>>>>> Ideally I'd
> >>>>>>>>>>>>>> like to
> >>>>>>>>>>>>>>        just call
> >>>>>>>>>>>>>>        >>>>> a
> >>>>>>>>>>>>>>        >>>>>>>>> method on
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
> >>>>>>>>>>>>>> bottom if
> >>>>>>>>>>>>>> the branch
> >>>>>>>>>>>>>>        cases
> >>>>>>>>>>>>>>        >>>>> are
> >>>>>>>>>>>>>>        >>>>>>>>> defined
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think the
> >>>>>>>>>>>>>> addBranch(predicate,
> >>>>>>>>>>>>>> handleCase)
> >>>>>>>>>>>>>>        is very
> >>>>>>>>>>>>>>        >>>>>> nice
> >>>>>>>>>>>>>>        >>>>>>>>> and the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
> >>>>>>>>>>>>>> flipped
> >>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>        how we
> >>>>>>>>>>>>>>        >>>>>>> specify
> >>>>>>>>>>>>>>        >>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>          
> >>>>>>>>>>>>>> .addBranch(predicate1,
> >>>>>>>>>>>>> this::handle1)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>          
> >>>>>>>>>>>>>> .addBranch(predicate2,
> >>>>>>>>>>>>> this::handle2)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>>>>>        >>>>>>>> KStreamBrancher
> >>>>>>>>>>>>>>        >>>>>>>>> or
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> which is added to by
> >>>>>>>>>>>>>> addBranch() and
> >>>>>>>>>>>>>>        terminated by
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
> >>>>>>>>>>>>>> obviously
> >>>>>>>>>>>>>>        incompatible with
> >>>>>>>>>>>>>>        >>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> current
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
> >>>>>>>>>>>>>> have to
> >>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>        different
> >>>>>>>>>>>>>>        >>>>>> name,
> >>>>>>>>>>>>>>        >>>>>>>> but
> >>>>>>>>>>>>>>        >>>>>>>>> that
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
> >>>>>>>>>>>>>> - we
> >>>>>>>>>>>>>> could call it
> >>>>>>>>>>>>>>        >>>>>> something
> >>>>>>>>>>>>>>        >>>>>>>> like
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
> >>>>>>>>>>>>>> deprecate the
> >>>>>>>>>>>>>> old API.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
> >>>>>>>>>>>>>> motivations of
> >>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>        KIP?  It
> >>>>>>>>>>>>>>        >>>>>> seems
> >>>>>>>>>>>>>>        >>>>>>>>> like it
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>>>>>> branching
> >>>>>>>>>>>>>>        while also
> >>>>>>>>>>>>>>        >>>>>>>> allowing
> >>>>>>>>>>>>>>        >>>>>>>>> you
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> dynamically build of
> >>>>>>>>>>>>>> branches off of
> >>>>>>>>>>>>>>        KBranchedStreams
> >>>>>>>>>>>>>>        >>>>>> if
> >>>>>>>>>>>>>>        >>>>>>>>> desired.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
> >>>>>>>>>>>>>> PM Ivan
> >>>>>>>>>>>>>> Ponomarev
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>>>>>        ks){
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>>>>>        String> ks){
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>> KafkaStreamsBrancher<String,
> >>>>>>>>>>>>>> String>()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>>>>>        this::handleFirstCase)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>>>>>        this::handleSecondCase)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
> >>>>>>>>>>>>>> Bejeck пишет:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>>>>>        >>>>> takes a
> >>>>>>>>>>>>>>        >>>>>>>>> Consumer
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
> >>>>>>>>>>>>>> nothing,
> >>>>>>>>>>>>>> and the
> >>>>>>>>>>>>>>        example in
> >>>>>>>>>>>>>>        >>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>> KIP
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
> >>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>> terminal node
> >>>>>>>>>>>>>>        >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
> >>>>>>>>>>>>>> something, but
> >>>>>>>>>>>>>> how would
> >>>>>>>>>>>>> we
> >>>>>>>>>>>>>>        >>>>> handle
> >>>>>>>>>>>>>>        >>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>> case
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>>>>>> wants to
> >>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>        >>>>>>>> processing
> >>>>>>>>>>>>>>        >>>>>>>>> and
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
> >>>>>>>>>>>>>> node on
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>> branched
> >>>>>>>>>>>>>>        >>>>>> stream
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
> >>>>>>>>>>>>>> logic
> >>>>>>>>>>>>>> as is if
> >>>>>>>>>>>>>>        we had
> >>>>>>>>>>>>>>        >>>>>>>> something
> >>>>>>>>>>>>>>        >>>>>>>>> like
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
> >>>>>>>>>>>>>> branches =
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
> >>>>>>>>>>>>>> 6:15 PM
> >>>>>>>>>>>>>> Bill Bejeck
> >>>>>>>>>>>>> <
> >>>>>>>>>>>>>>        >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>>>>>> discussion for
> >>>>>>>>>>>>> KIP-
> >>>>>>>>>>>>>>        >>>>> 418.
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
> >>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>> about
> >>>>>>>>>>>>> KIP-418.
> >>>>>>>>>>>>>>        >>>>> Please
> >>>>>>>>>>>>>>        >>>>>>>> take
> >>>>>>>>>>>>>>        >>>>>>>>> a
> >>>>>>>>>>>>>>        >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>>>>>> appreciate any
> >>>>>>>>>>>>>>        feedback :)
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>>>>>        >>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>
> >>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>>>>>        >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>        >>>>>>>>>
> >>>>>>>>>>>>>>        >
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>>
> >>
> > 
> 
> 
> Attachments:
> * signature.asc

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <mj...@apache.org>.
Thanks for updating the KIP!

I guess the only open question is about `Branched.withJavaConsumer` and
its relationship to the returned `Map`.

Originally, we discussed two main patterns:

 (1) split a stream and return the substreams for futher processing
 (2) split a stream and modify the substreams with in-place method chaining

To combine both patterns we wanted to allow for

  -> split a stream, modify the substreams, and return the _modified_
substreams for further processing

> But is it also an issue? With Kafka Streams, we can split the topology graph at any point. Technically, it's OK to do both: feed the KStream to a [Java]Consumer AND save it in resulting Map. If one doesn't need the stream in the Map, one simply does not extract it from there

That is of course possible. However, it introduces some "hidded" semantics:

 - using `withChain` I get the modified sub-stream
 - using `withJavaConsumer` I get the unmodifed sub-stream

This seems to be quite subtle to me.



From my understanding the original idea of `withJavaConsumer` was to
model a terminal operation, ie, it should be similar to:

Branched.withChain(s -> {
  s.to();
  return null;
})

However, I am not sure if we should even allow `withChain()` to return
`null`? IMHO, we should throw an exception for this case to avoid a `key
-> null` entry in the returned Map.

Following this train of through, and if we want to allow the "return
null" pattern in general, we need `withJavaConsumer` that does not add
an entry to the Map.

Following your proposal, the semantics of `withJavaConsumer` could also
be achieved with `withChain`:

Branched.withChain(s -> {
  s.to();
  return s;
})

Thus, for you proposal `withJavaConumer` is purely syntactic sugar,
while for the first proposal it adds new functionality (if `return null`
is not allowed, using `withChain()` is not possible to "hide a
sub-stream in the result). Furthermore, we might need to allow `return
null` in your prosal to allow uses to "hide" a sub-stream in the Map.



I guess I can be convinced either way. However, if we follow your
proposal, I am wondering if we need `withJavaConsumer` at all? Its
benefit seems to be small? Also, having a reduced API is usually
preferable as it's simpler to learn.



-Matthias




On 5/15/20 3:12 PM, Ivan Ponomarev wrote:
> Hello, John, hello Matthias!
> 
> Thank you very much for your detailed feedback!
> 
> -----------------------------------------
> 
> John,
> 
>> It looks like you missed my reply on Apr 23rd.
> 
> For some unknown reason it didn't reach my inbox, fortunately we have
> all the emails on the web.
> 
>> 1. Can you propose to deprecate (but not remove) the existing ‘branch’
> method?
> 
> Done, in "Compatibility, Deprecation, and Migration Plan" section.
> 
>> 2. [Explain why 'branch' operator is superior to branching directly
> off of the parent KStream for the needs of dynamic branching]
> 
> Done, see an ugly counterexample in 'Dynamic Branching' section.
> 
>> 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]
> 
> As Mathhias noted, `withSink` can also be confusing. I renamed this
> method to `withJavaConsumer` per Matthias' suggestion.
> 
>> 4. ...It seems like there are two disjoint use cases: EITHER using
> chain and the result map OR using just the sink
> 
> This is discussed below.
> 
> ----------------------------------------------
> 
> Mathhias,
> 
>> 1. [We should rename `KBranchedStream` -> `BranchedKStream`]
> 
> Done.
> 
>> 2. [Ambiguous phrase about 'parameterless' version of the `branch`
> method]
> 
> Fixed.
> 
> 
>> 3. Overview of newly added methods/interfaces
> 
> Done in `Proposed Changes` section.
> 
> 
>> 4. [Concerning John's note] > I don't think that using both
> `withChain()` and `withConsumer()` is the
> issue, as the KIP clearly states that the result of `withChain()` will
> be given to the `Consumer`.
> 
> Yes, I agree!
> 
>> The issue is really with the `Consumer` and the returned `Map` of
> `defautBranch()` and `noDefaultBranch()`. Maybe a reasonable
> implementation would be to not add the "branch" to the result map if
> `withConsumer` is used?
> 
> But is it also an issue? With Kafka Streams, we can split the topology
> graph at any point. Technically, it's OK to do both: feed the KStream to
> a [Java]Consumer AND save it in resulting Map. If one doesn't need the
> stream in the Map, one simply does not extract it from there :-)
> 
> In the current version of KIP it is assumed that the returned map
> contains ALL the branches, either tagged with IDs explicitly set by the
> programmer, or with some default auto-generated ids. Dealing with this
> map is the user's responsibility.
> 
> What seems to me to be an issue is introducing exclusions to this
> general rule, like 'swallowing' some streams by provided
> [Java]Consumers. This can make things complicated. What if a user
> provides both the name of the branch and a [Java]Consumer? What do they
> mean in this case? Should we 'swallow' the stream or save it to the map?
> There's no point in 'saving the space' in this map, so maybe just leave
> it as it is?
> 
> ----
> 
> I rewrote the KIP and also fixed a couple of typos.
> 
> Looking forward for your feedback again!
> 
> Regards,
> 
> Ivan.
> 
> 
> 
> 08.05.2020 22:55, Matthias J. Sax пишет:
>> Thanks for updating the KIP!
>>
>> I also have some minor comment:
>>
>>
>>
>> (1) We should rename `KBranchedStream` -> `BranchedKStream`
>>
>> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
>> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
>> and `KGroupedKTable` that we cannot rename without a breaking change...
>> so we just keep them.)
>>
>>
>>
>> (2) Quote:
>>
>>> Both branch and defaultBranch operations also have overloaded
>>> parameterless alternatives.
>>
>> I think `branch()` always needs to take a `Predicate` and assume you
>> meant that `Branched` is optional. Can you maybe rephrase it accordingly
>> as `branch()` would not be "parameterless".
>>
>>
>>
>> (3) Can you maybe add an overview in the "Public Interface" section) of
>> newly added and deprecated methods/classes (cf.
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
>>
>>
>>
>>
>> (4) What is unclear from the KIP is the interaction of `withConsumer()`
>> and the finally returned `Map<String, KStream>`. This related to John's
>> 4th comment:
>>
>>> It seems like there are really two disjoint use cases: EITHER using
>>> chain and the result map OR using just the sink.
>>
>> I don't think that using both `withChain()` and `withConsumer()` is the
>> issue though, as the KIP clearly states that the result of `withChain()`
>> will be given to the `Consumer`. The issue is really with the `Consumer`
>> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
>>
>> Maybe a reasonable implementation would be to not add the "branch" to
>> the result map if `withConsumer` is used? As long as we clearly document
>> it in the JavaDocs, this might be fine?
>>
>>
>>
>> (5) Reply to John's comments:
>>
>>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you
>>> were talking about the kafka Consumer interface (which doesn’t make
>>> sense, of course). I get that you were referring to the java Consumer
>>> interface, but we should still probably to to avoid the ambiguity.
>>> Just throwing out a suggestion, how about ‘withSink’?
>>
>> IMHO, `withSink` has the issue that it might be confused with a "sink
>> node", ie., writing the KStream to a topic.
>>
>> Maybe `withJavaConsumer` would make it less ambiguous?
>>
>>
>>
>>
>> -Matthias
>>
>>
>>
>>
>> On 5/8/20 7:13 AM, John Roesler wrote:
>>> Hi Ivan,
>>>
>>> It looks like you missed my reply on Apr 23rd. I think it’s close,
>>> but I had a few last comments.
>>>
>>> Thanks,
>>> John
>>>
>>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>>> Hello everyone,
>>>>
>>>> will someone please take a look at the reworked KIP?
>>>>
>>>> I believe that now it follows design principles and takes into account
>>>> all the arguments discussed here.
>>>>
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>>
>>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>>> Hi,
>>>>>
>>>>> I have read the John's "DSL design principles" and have completely
>>>>> rewritten the KIP, see
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> This version includes all the previous discussion results and follows
>>>>> the design principles, with one exception.
>>>>>
>>>>> The exception is
>>>>>
>>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>>
>>>>> which formally violates 'no more than one parameter' rule, but I think
>>>>> here it is justified.
>>>>>
>>>>> We must provide a predicate for a branch and don't need to provide one
>>>>> for the default branch. Thus for both operations we may use a single
>>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>>
>>>>> Since predicate is a natural, necessary part of a branch, no
>>>>> 'proliferation of overloads, deprecations, etc.' is expected here
>>>>> as it
>>>>> is said in the rationale for the 'single parameter rule'.
>>>>>
>>>>> WDYT, is this KIP mature enough to begin voting?
>>>>>
>>>>> Regards,
>>>>>
>>>>> Ivan
>>>>>
>>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>>> Ivan,
>>>>>>
>>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>>
>>>>>> The DSL improved further in the meantime and we already have a
>>>>>> `Named`
>>>>>> config object to name operators. It seems reasonable to me to
>>>>>> build on
>>>>>> this.
>>>>>>
>>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>>> want to follow:
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>>
>>>>>>
>>>>>> -- might be worth to checkout.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>>> Hi everyone!
>>>>>>>
>>>>>>> Let me revive the discussion of this KIP.
>>>>>>>
>>>>>>> I'm very sorry for stopping my participation in the discussion in
>>>>>>> June
>>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>>> spare time. But I think I must finish this, because we invested
>>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>>> propose other things before this one is finalized.
>>>>>>>
>>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>>> KafkaStreamBrancher class of my invention (the original idea for
>>>>>>> this
>>>>>>> KIP) worked for me -- that's another reason why I gave up pushing
>>>>>>> the
>>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>>> branches, I worked around it this way:
>>>>>>>
>>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>>> new KafkaStreamBrancher<....>()
>>>>>>>       .branch(....)
>>>>>>>       .defaultBranch(result::set)
>>>>>>>       .onTopOf(someStream);
>>>>>>> result.get()...
>>>>>>>
>>>>>>>
>>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>>
>>>>>>> I think that Matthias came up with a bright solution in his post
>>>>>>> from
>>>>>>> May, 24th 2019. Let me quote it:
>>>>>>>
>>>>>>> KStream#split() -> KBranchedStream
>>>>>>> // branch is not easily accessible in current scope
>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>     -> KBranchedStream
>>>>>>> // assign a name to the branch and
>>>>>>> // return the sub-stream to the current scope later
>>>>>>> //
>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>     -> KBranchedStream
>>>>>>> // default branch is not easily accessible
>>>>>>> // return map of all named sub-stream into current scope
>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>     -> Map<String,KStream>
>>>>>>> // assign custom name to default-branch
>>>>>>> // return map of all named sub-stream into current scope
>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>     -> Map<String,KStream>
>>>>>>> // assign a default name for default
>>>>>>> // return map of all named sub-stream into current scope
>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>     -> Map<String,KStream>
>>>>>>> // return map of all names sub-stream into current scope
>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>     -> Map<String,KStream>
>>>>>>>
>>>>>>> I believe this would satisfy everyone. Optional names seems to be
>>>>>>> a good
>>>>>>> idea: when you don't need to have the branches in the same scope,
>>>>>>> you
>>>>>>> just don't use names and you don't risk making your code brittle.
>>>>>>> Or,
>>>>>>> you might want to add names just for debugging purposes. Or,
>>>>>>> finally,
>>>>>>> you might use the returned Map to have the named branches in the
>>>>>>> original scope.
>>>>>>>
>>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>>> familiar with Streams API design principles than me.
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>>> I am moving this KIP into "inactive status". Feel free to resume
>>>>>>>> the
>>>>>>>> KIP
>>>>>>>> at any point.
>>>>>>>>
>>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>>> do so.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>>> Ivan,
>>>>>>>>>
>>>>>>>>> did you see my last reply? What do you think about my proposal
>>>>>>>>> to mix
>>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>>> Thanks for the input John!
>>>>>>>>>>
>>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>>
>>>>>>>>>> If you want to get the `KStream` as part of the `Map` back
>>>>>>>>>> using a
>>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>>> using a
>>>>>>>>>> `Consumer`, no.
>>>>>>>>>>
>>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>>
>>>>>>>>>> I wanted to sketch out a high level proposal to merge both
>>>>>>>>>> patterns
>>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>>> make
>>>>>>>>>> totally sense.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>>
>>>>>>>>>> If both are mandatory, what do we gain by it? The returned
>>>>>>>>>> `Map` only
>>>>>>>>>> contains the corresponding branches, so why should we prefix
>>>>>>>>>> all of
>>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>>
>>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>>> `Named` is
>>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a
>>>>>>>>>> counter
>>>>>>>>>> for
>>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>>> names
>>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>>> returned in the `Map`).
>>>>>>>>>>
>>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>>> misses to
>>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Hence, I am actually in favor to not allow a default name but
>>>>>>>>>> keep
>>>>>>>>>> `split()` without parameter and make `Named` in `branch()`
>>>>>>>>>> required
>>>>>>>>>> if a
>>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>>> specifying a
>>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> About
>>>>>>>>>>
>>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>>
>>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>>> hence
>>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>>
>>>>>>>>>>>        withChain(...);
>>>>>>>>>>
>>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?)
>>>>>>>>>> does not
>>>>>>>>>> seem to be a configuration. We can also not prevent a user to
>>>>>>>>>> call
>>>>>>>>>> `withName()` in combination of `withChain()` what does not
>>>>>>>>>> make sense
>>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>>> check
>>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>>> `withChain()`
>>>>>>>>>> not `withName()` is called and the branch is missing in the
>>>>>>>>>> returned
>>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>>
>>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>>> object
>>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>>> others,
>>>>>>>>>> but this seems not to be the case here. If we add new
>>>>>>>>>> configuration
>>>>>>>>>> later, we can also just move forward by deprecating the
>>>>>>>>>> methods that
>>>>>>>>>> accept `Named` and add new methods that accepted
>>>>>>>>>> `BranchConfig` (that
>>>>>>>>>> would of course implement `Named`).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thoughts?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> @Ivan, what do you think about the general idea to blend the
>>>>>>>>>> two main
>>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would
>>>>>>>>>>> satisfy
>>>>>>>>>>> everyone. Returning the map from the terminal operations also
>>>>>>>>>>> solves
>>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>>> to add
>>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>>
>>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>>> Otherwise,
>>>>>>>>>>> we wouldn't have keys for the map to return. I this this is
>>>>>>>>>>> actually
>>>>>>>>>>> not too bad, since experience has taught us that, although
>>>>>>>>>>> names for
>>>>>>>>>>> operations are not required to define stream processing
>>>>>>>>>>> logic, it
>>>>>>>>>>> does
>>>>>>>>>>> significantly improve the operational experience when you can
>>>>>>>>>>> map
>>>>>>>>>>> the
>>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>>> wouldn't (have to) reference the name to chain extra
>>>>>>>>>>> processing onto
>>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>>
>>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>>> operator itself, and then all the branches get index-suffixed
>>>>>>>>>>> names
>>>>>>>>>>> built from the branch operator name. I guess under this
>>>>>>>>>>> proposal, we
>>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>>> name,
>>>>>>>>>>> like this:
>>>>>>>>>>>
>>>>>>>>>>>       stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>>> "mysplit"
>>>>>>>>>>>                  .branch(..., ..., "abranch") // creates node
>>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>>                  .defaultBranch(...) // creates node
>>>>>>>>>>> "mysplit-default"
>>>>>>>>>>>
>>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>>
>>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to
>>>>>>>>>>> debate
>>>>>>>>>>> the "best" syntax in the context of each operation, but in
>>>>>>>>>>> general,
>>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>>
>>>>>>>>>>>        operator(function, config_object?) OR
>>>>>>>>>>> operator(config_object)
>>>>>>>>>>>
>>>>>>>>>>> where config_object is often just Named in the "function"
>>>>>>>>>>> variant.
>>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>>
>>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>>
>>>>>>>>>>>      operator(function, function, string)
>>>>>>>>>>>
>>>>>>>>>>> where the string is the name.
>>>>>>>>>>> My first question is whether the name should instead be
>>>>>>>>>>> specified
>>>>>>>>>>> with
>>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>>
>>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>>> arguments
>>>>>>>>>>> up into a config object like:
>>>>>>>>>>>
>>>>>>>>>>>       KBranchedStream#branch(BranchConfig)
>>>>>>>>>>>
>>>>>>>>>>>       interface BranchConfig extends NamedOperation {
>>>>>>>>>>>        withPredicate(...);
>>>>>>>>>>>        withChain(...);
>>>>>>>>>>>        withName(...);
>>>>>>>>>>>      }
>>>>>>>>>>>
>>>>>>>>>>> Although I guess we'd like to call BranchConfig something
>>>>>>>>>>> more like
>>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>>
>>>>>>>>>>> This makes the source code a little noisier, but it also
>>>>>>>>>>> makes us
>>>>>>>>>>> more
>>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>>> purely
>>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>>> overloads
>>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>>
>>>>>>>>>>> WDYT?
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> -John
>>>>>>>>>>>
>>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of
>>>>>>>>>>>> view.
>>>>>>>>>>>> Good
>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>>> approaches:
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>>
>>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>>      -> KBranchedStream
>>>>>>>>>>>>>
>>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>>> //
>>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>>> String)
>>>>>>>>>>>>>      -> KBranchedStream
>>>>>>>>>>>>>
>>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>>
>>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>>
>>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>>
>>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name
>>>>>>>>>>>>> and
>>>>>>>>>>>>> return
>>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>>> implementation can
>>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>>> returned
>>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>>
>>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor
>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Separately, I'm interested to see where the present
>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>> leads.
>>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>>> references (or
>>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>>> that this
>>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>>> logic when
>>>>>>>>>>>>>> their nested closure situation got out of hand.
>>>>>>>>>>>>>> Unfortunately,
>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt
>>>>>>>>>>>>>> code
>>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>>> other
>>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>>> solution if
>>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>>> the end of the story either, and you can see that by
>>>>>>>>>>>>>> observing
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>> there have been two follow-on eras, as they continue
>>>>>>>>>>>>>> trying to
>>>>>>>>>>>>>> cope with
>>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>>> First, you
>>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert
>>>>>>>>>>>>>> nested
>>>>>>>>>>>>>> code to
>>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>>> this).
>>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to
>>>>>>>>>>>>>> apply
>>>>>>>>>>>>>> language
>>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the
>>>>>>>>>>>>>> "flattest"
>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>> programming style to solve the problem (because you get
>>>>>>>>>>>>>> back to
>>>>>>>>>>>>>> just one
>>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to
>>>>>>>>>>>>>> take
>>>>>>>>>>>>>> the JS
>>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>>> valuable
>>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time,
>>>>>>>>>>>>>> just
>>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge
>>>>>>>>>>>>>> (and
>>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly
>>>>>>>>>>>>>> punt on
>>>>>>>>>>>>>> it, by
>>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen
>>>>>>>>>>>>>> <pgwhalen@gmail.com
>>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        Ivan, I’ll definitely forfeit my point on the
>>>>>>>>>>>>>> clumsiness of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        branch(predicate, consumer) solution, I don’t see
>>>>>>>>>>>>>> any real
>>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>>        for the dynamic case.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        IMO the one trade off to consider at this point is the
>>>>>>>>>>>>>> scope
>>>>>>>>>>>>>>        question. I don’t know if I totally agree that “we
>>>>>>>>>>>>>> rarely
>>>>>>>>>>>>>> need them
>>>>>>>>>>>>>>        in the same scope” since merging the branches back
>>>>>>>>>>>>>> together
>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>        seems like a perfectly plausible use case that can
>>>>>>>>>>>>>> be a lot
>>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>>        when the branched streams are in the same scope.
>>>>>>>>>>>>>> That being
>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>        for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>        solution - working around the scope thing is easy
>>>>>>>>>>>>>> enough if
>>>>>>>>>>>>>> you need
>>>>>>>>>>>>>>        to.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>>        <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > Hello everyone, thank you all for joining the
>>>>>>>>>>>>>> discussion!
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > Well, I don't think the idea of named branches,
>>>>>>>>>>>>>> be it a
>>>>>>>>>>>>>>        LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>>> definition
>>>>>>>>>>>>>>        matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>>> has more
>>>>>>>>>>>>>>        advantages than drawbacks.
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > In my opinion, the only real positive outcome from
>>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>>        proposal is that all the returned branches are in
>>>>>>>>>>>>>> the same
>>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>>        But 1) we rarely need them in the same scope 2)
>>>>>>>>>>>>>> there is a
>>>>>>>>>>>>>>        workaround for the scope problem, described in the
>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > 'Inlining the complex logic' is not a problem,
>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>> can use
>>>>>>>>>>>>>>        method references instead of lambdas. In real world
>>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>>        tend to split the complex logic to methods anyway,
>>>>>>>>>>>>>> so the
>>>>>>>>>>>>>> code is
>>>>>>>>>>>>>>        going to be clean.
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > The drawbacks are strong. The cohesion between
>>>>>>>>>>>>>> predicates
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>        handlers is lost. We have to define predicates in one
>>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>>        handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > - what if we forget to define a handler for a
>>>>>>>>>>>>>> name? or a
>>>>>>>>>>>>>> name for
>>>>>>>>>>>>>>        a handler?
>>>>>>>>>>>>>>        > - what if we misspell a name?
>>>>>>>>>>>>>>        > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > What Michael propose would have been totally OK
>>>>>>>>>>>>>> if we had
>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>        writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>>        "dynamic naming" approach would have looked most
>>>>>>>>>>>>>> concise
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>        beautiful. But in Java we expect all the problems
>>>>>>>>>>>>>> related to
>>>>>>>>>>>>>>        identifiers to be eliminated in compile time.
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>>        all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>>        point?
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > ---
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > Earlier in this discussion John Roesler also
>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>        without "start branching" operator, and later Paul
>>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>>        the case when we have to add a dynamic number of
>>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>>        current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>>        me address both comments here.
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > 1) "Start branching" operator (I think that
>>>>>>>>>>>>>> *split* is a
>>>>>>>>>>>>>> good name
>>>>>>>>>>>>>>        for it indeed) is critical when we need to do a
>>>>>>>>>>>>>> dynamic
>>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>>        see example below.
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > 2) No, dynamic branching in current KIP is not
>>>>>>>>>>>>>> clumsy at
>>>>>>>>>>>>>> all.
>>>>>>>>>>>>>>        Imagine a real-world scenario when you need one
>>>>>>>>>>>>>> branch per
>>>>>>>>>>>>>> enum
>>>>>>>>>>>>>>        value (say, RecordType). You can have something
>>>>>>>>>>>>>> like this:
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > /*John:if we had to start with stream.branch(...)
>>>>>>>>>>>>>> here,
>>>>>>>>>>>>>> it would
>>>>>>>>>>>>>>        have been much messier.*/
>>>>>>>>>>>>>>        > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>>        > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>>        >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>>        recordType,
>>>>>>>>>>>>>>        >                     recordType::processRecords);
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > Regards,
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > Ivan
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>        > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>>        >> I also agree with Michael's observation about
>>>>>>>>>>>>>> the core
>>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>>        >> current `branch()` implementation.
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>>        thinking
>>>>>>>>>>>>>>        >> was more aligned with Paul's proposal to just
>>>>>>>>>>>>>> add a name
>>>>>>>>>>>>>> to each
>>>>>>>>>>>>>>        >> `branch()` statement and return a
>>>>>>>>>>>>>> `Map<String,KStream>`.
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >> It makes the code easier to read, and also make the
>>>>>>>>>>>>>> order of
>>>>>>>>>>>>>>        >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >>>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>        >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>        >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>        >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>>        >> An open question is the case for which no
>>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>        >> specified. Atm, `split()` and `branch()` would
>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>        `BranchedKStream`
>>>>>>>>>>>>>>        >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>>        >> (what is not the case atm). Or is this actually
>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>> real
>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>        >> because users can just ignore the branch
>>>>>>>>>>>>>> returned by
>>>>>>>>>>>>>>        `defaultBranch()`
>>>>>>>>>>>>>>        >> in the result `Map` ?
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >> About "inlining": So far, it seems to be a
>>>>>>>>>>>>>> matter of
>>>>>>>>>>>>>> personal
>>>>>>>>>>>>>>        >> preference. I can see arguments for both, but no
>>>>>>>>>>>>>> "killer
>>>>>>>>>>>>>>        argument" yet
>>>>>>>>>>>>>>        >> that clearly make the case for one or the other.
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >> -Matthias
>>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>>        >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>        >>> Perhaps inlining is the wrong terminology. It
>>>>>>>>>>>>>> doesn’t
>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>        that a lambda with the full downstream topology be
>>>>>>>>>>>>>> defined
>>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>>        it can be a method reference as with Ivan’s original
>>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>>        The advantage of putting the predicate and its
>>>>>>>>>>>>>> downstream
>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>        (Consumer) together in branch() is that they are
>>>>>>>>>>>>>> required
>>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>>        to each other.
>>>>>>>>>>>>>>        >>>
>>>>>>>>>>>>>>        >>> Ultimately the downstream code has to live
>>>>>>>>>>>>>> somewhere,
>>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>>        branch trees will be hard to read regardless.
>>>>>>>>>>>>>>        >>>
>>>>>>>>>>>>>>        >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>>        <michael.drogalis@confluent.io
>>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>        >>>>
>>>>>>>>>>>>>>        >>>> I'm less enthusiastic about inlining the
>>>>>>>>>>>>>> branch logic
>>>>>>>>>>>>>> with its
>>>>>>>>>>>>>>        downstream
>>>>>>>>>>>>>>        >>>> functionality. Programs that have deep branch
>>>>>>>>>>>>>> trees
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>        quickly become
>>>>>>>>>>>>>>        >>>> harder to read as a single unit.
>>>>>>>>>>>>>>        >>>>
>>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>>        <pgwhalen@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Also +1 on the issues/goals as Michael
>>>>>>>>>>>>>> outlined them,
>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>>        that sets a
>>>>>>>>>>>>>>        >>>>> great framework for the discussion.
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Regarding the SortedMap solution, my
>>>>>>>>>>>>>> understanding is
>>>>>>>>>>>>>> that the
>>>>>>>>>>>>>>        current
>>>>>>>>>>>>>>        >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>>        decisions) is
>>>>>>>>>>>>>>        >>>>> roughly this:
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> stream.split()
>>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>,
>>>>>>>>>>>>>> Consumer<KStream<K, V>>)
>>>>>>>>>>>>>>        >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Obviously some ordering is necessary, since
>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>        construct
>>>>>>>>>>>>>>        >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>>> like it
>>>>>>>>>>>>>>        provides as much
>>>>>>>>>>>>>>        >>>>> associativity as the SortedMap solution,
>>>>>>>>>>>>>> because each
>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>        call
>>>>>>>>>>>>>>        >>>>> directly associates the "conditional" with
>>>>>>>>>>>>>> the "code
>>>>>>>>>>>>>> block."
>>>>>>>>>>>>>>        The value it
>>>>>>>>>>>>>>        >>>>> provides over the KIP solution is the
>>>>>>>>>>>>>> accessing of
>>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>>        the same
>>>>>>>>>>>>>>        >>>>> scope.
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> The KIP solution is less "dynamic" than the
>>>>>>>>>>>>>> SortedMap
>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>        in the sense
>>>>>>>>>>>>>>        >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>        branches, but it is
>>>>>>>>>>>>>>        >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>>        the "static"
>>>>>>>>>>>>>>        >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>>        fluently declare and
>>>>>>>>>>>>>>        >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>>        ignore a
>>>>>>>>>>>>>>        >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>>> identical
>>>>>>>>>>>>>>        SortedMap
>>>>>>>>>>>>>>        >>>>> solution on top of it.
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> I could also see a middle ground where
>>>>>>>>>>>>>> instead of
>>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>>        SortedMap being
>>>>>>>>>>>>>>        >>>>> taken in, branch() takes a name and not a
>>>>>>>>>>>>>> Consumer.
>>>>>>>>>>>>>> Something
>>>>>>>>>>>>>>        like this:
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Map<String, KStream<K, V>> branches =
>>>>>>>>>>>>>> stream.split()
>>>>>>>>>>>>>>        >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>>        >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>>        >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Pros for that solution:
>>>>>>>>>>>>>>        >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>>        >>>>> - no double brace initialization, hopefully
>>>>>>>>>>>>>> slightly
>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>        readable than
>>>>>>>>>>>>>>        >>>>> SortedMap
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Cons
>>>>>>>>>>>>>>        >>>>> - downstream branch logic cannot be specified
>>>>>>>>>>>>>> inline
>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>        makes it harder
>>>>>>>>>>>>>>        >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>>        unlike the KIP)
>>>>>>>>>>>>>>        >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>>        existing
>>>>>>>>>>>>>>        >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> (KBranchedStreams could even work *both* ways
>>>>>>>>>>>>>> but
>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>        that's overdoing
>>>>>>>>>>>>>>        >>>>> it).
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Overall I'm curious how important it is to be
>>>>>>>>>>>>>> able to
>>>>>>>>>>>>>> easily
>>>>>>>>>>>>>>        access the
>>>>>>>>>>>>>>        >>>>> branched KStream in the same scope as the
>>>>>>>>>>>>>> original.
>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>        possible that it
>>>>>>>>>>>>>>        >>>>> doesn't need to be handled directly by the
>>>>>>>>>>>>>> API, but
>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>        left up to the
>>>>>>>>>>>>>>        >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> Paul
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie
>>>>>>>>>>>>>> Blee-Goldman
>>>>>>>>>>>>>>        <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>        >>>>> wrote:
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>        >>>>>> I'd like to +1 what Michael said about the
>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>        existing
>>>>>>>>>>>>>>        >>>>> branch
>>>>>>>>>>>>>>        >>>>>> method, I agree with what he's outlined and
>>>>>>>>>>>>>> I think
>>>>>>>>>>>>>> we should
>>>>>>>>>>>>>>        proceed by
>>>>>>>>>>>>>>        >>>>>> trying to alleviate these problems.
>>>>>>>>>>>>>> Specifically it
>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>        important to be
>>>>>>>>>>>>>>        >>>>>> able to cleanly access the individual
>>>>>>>>>>>>>> branches (eg
>>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>>        >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>>        this KIP.
>>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>>        >>>>>> That said, I don't think we should so easily
>>>>>>>>>>>>>> give in
>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>        double brace
>>>>>>>>>>>>>>        >>>>>> anti-pattern or force ours users into it if
>>>>>>>>>>>>>> at all
>>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>>        >>>>> avoid...just
>>>>>>>>>>>>>>        >>>>>> my two cents.
>>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>>        >>>>>> Cheers,
>>>>>>>>>>>>>>        >>>>>> Sophie
>>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>>        >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael
>>>>>>>>>>>>>> Drogalis <
>>>>>>>>>>>>>>        >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>>        >>>>>>> I’d like to propose a different way of
>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>>        To me,
>>>>>>>>>>>>>>        >>>>> there
>>>>>>>>>>>>>>        >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>>        >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>>> type
>>>>>>>>>>>>>>        >>>>>> warnings.
>>>>>>>>>>>>>>        >>>>>>> 2. The way in which you use the stream
>>>>>>>>>>>>>> branches is
>>>>>>>>>>>>>>        positionally coupled
>>>>>>>>>>>>>>        >>>>>> to
>>>>>>>>>>>>>>        >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>>        >>>>>>> 3. It is brittle to extend existing branch
>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>        additional code
>>>>>>>>>>>>>>        >>>>>>> paths.
>>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>>        >>>>>>> Using associative constructs instead of
>>>>>>>>>>>>>> relying on
>>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>>        constructs
>>>>>>>>>>>>>>        >>>>>> would
>>>>>>>>>>>>>>        >>>>>>> be a stronger approach. Consider a
>>>>>>>>>>>>>> signature that
>>>>>>>>>>>>>> instead
>>>>>>>>>>>>>>        looks like
>>>>>>>>>>>>>>        >>>>>> this:
>>>>>>>>>>>>>>        >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>>        Predicate<?
>>>>>>>>>>>>>>        >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>>        >>>>>>> Branches are given names in a map, and as a
>>>>>>>>>>>>>> result,
>>>>>>>>>>>>>> the API
>>>>>>>>>>>>>>        returns a
>>>>>>>>>>>>>>        >>>>>>> mapping of names to streams. The ordering
>>>>>>>>>>>>>> of the
>>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>>        >>>>>> maintained
>>>>>>>>>>>>>>        >>>>>>> because it’s a sorted map. Insert order
>>>>>>>>>>>>>> determines
>>>>>>>>>>>>>> the order
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>        >>>>>> evaluation.
>>>>>>>>>>>>>>        >>>>>>> This solves problem 1 because there are no
>>>>>>>>>>>>>> more
>>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>>        solves
>>>>>>>>>>>>>>        >>>>>> problem
>>>>>>>>>>>>>>        >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>>> access the
>>>>>>>>>>>>>>        branch you’re
>>>>>>>>>>>>>>        >>>>>>> interested in. It solves problem 3 because
>>>>>>>>>>>>>> you can
>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>        another
>>>>>>>>>>>>>>        >>>>>>> conditional by simply attaching another
>>>>>>>>>>>>>> name to the
>>>>>>>>>>>>>>        structure, rather
>>>>>>>>>>>>>>        >>>>>> than
>>>>>>>>>>>>>>        >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>>        >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>>        historically
>>>>>>>>>>>>>>        >>>>>>> awkward in Java. I know it’s an
>>>>>>>>>>>>>> anti-pattern to use
>>>>>>>>>>>>>>        voluminously, but
>>>>>>>>>>>>>>        >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>>        >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>>        <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>        >>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>>> "start
>>>>>>>>>>>>> branching"
>>>>>>>>>>>>>>        >>>>> operator
>>>>>>>>>>>>>>        >>>>>>> is
>>>>>>>>>>>>>>        >>>>>>>> confusing when named the same way as the
>>>>>>>>>>>>>> actual
>>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>>        "Split"
>>>>>>>>>>>>>>        >>>>> seems
>>>>>>>>>>>>>>        >>>>>>>> like a good name. Alternatively, we can do
>>>>>>>>>>>>>> without
>>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>>        >>>>> branching"
>>>>>>>>>>>>>>        >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>> stream
>>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>        >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>> Tentatively, I think that this branching
>>>>>>>>>>>>>> operation
>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>        >>>>> terminal.
>>>>>>>>>>>>>>        >>>>>>> That
>>>>>>>>>>>>>>        >>>>>>>> way, we don't create ambiguity about how
>>>>>>>>>>>>>> to use
>>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>>        is, `branch`
>>>>>>>>>>>>>>        >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>>        `void`, to
>>>>>>>>>>>>>>        >>>>>>>> enforce that it comes last, and that there
>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>> one
>>>>>>>>>>>>>>        definition of
>>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>>        >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>>        there's no
>>>>>>>>>>>>>>        >>>>>>> default,
>>>>>>>>>>>>>>        >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>>        record
>>>>>>>>>>>>>>        >>>>>> falls
>>>>>>>>>>>>>>        >>>>>>>> though with no default.
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>> Thoughts?
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>> Thanks,
>>>>>>>>>>>>>>        >>>>>>>> -John
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias
>>>>>>>>>>>>>> J. Sax <
>>>>>>>>>>>>>>        >>>>> matthias@confluent.io
>>>>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>>>>>        >>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> Thanks for updating the KIP and your
>>>>>>>>>>>>>> answers.
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> this is to make the name similar to
>>>>>>>>>>>>>> String#split
>>>>>>>>>>>>>>        >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>>        >>>>>>>>> The intend was to avoid name duplication.
>>>>>>>>>>>>>> The
>>>>>>>>>>>>>> return type
>>>>>>>>>>>>>>        should
>>>>>>>>>>>>>>        >>>>>> _not_
>>>>>>>>>>>>>>        >>>>>>>>> be an array.
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> The current proposal is
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> stream.branch()
>>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> IMHO, this reads a little odd, because
>>>>>>>>>>>>>> the first
>>>>>>>>>>>>>>        `branch()` does
>>>>>>>>>>>>>>        >>>>> not
>>>>>>>>>>>>>>        >>>>>>>>> take any parameters and has different
>>>>>>>>>>>>>> semantics
>>>>>>>>>>>>>> than the
>>>>>>>>>>>>> later
>>>>>>>>>>>>>>        >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>        >>>>> hidden
>>>>>>>>>>>>>>        >>>>>>>>> that the first call is `KStream#branch()`
>>>>>>>>>>>>>> while
>>>>>>>>>>>>>> the others
>>>>>>>>>>>>> are
>>>>>>>>>>>>>>        >>>>>>>>> `KBranchedStream#branch()` what makes
>>>>>>>>>>>>>> reading the
>>>>>>>>>>>>>> code
>>>>>>>>>>>>> harder.
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> Because I suggested to rename
>>>>>>>>>>>>>> `addBranch()` ->
>>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>>        I though
>>>>>>>>>>>>>>        >>>>>> it
>>>>>>>>>>>>>>        >>>>>>>>> might be better to also rename
>>>>>>>>>>>>>> `KStream#branch()`
>>>>>>>>>>>>>> to avoid
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        >>>>> naming
>>>>>>>>>>>>>>        >>>>>>>>> overlap that seems to be confusing. The
>>>>>>>>>>>>>> following
>>>>>>>>>>>>>> reads
>>>>>>>>>>>>> much
>>>>>>>>>>>>>>        >>>>> cleaner
>>>>>>>>>>>>>>        >>>>>> to
>>>>>>>>>>>>>>        >>>>>>>> me:
>>>>>>>>>>>>>>        >>>>>>>>> stream.split()
>>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> Maybe there is a better alternative to
>>>>>>>>>>>>>> `split()`
>>>>>>>>>>>>>> though to
>>>>>>>>>>>>>>        avoid
>>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> naming overlap.
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>        >>>>> cannot
>>>>>>>>>>>>>>        >>>>>>> have
>>>>>>>>>>>>>>        >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> Bummer. Didn't consider this. Maybe we
>>>>>>>>>>>>>> can still
>>>>>>>>>>>>>> come up
>>>>>>>>>>>>>>        with a
>>>>>>>>>>>>>>        >>>>> short
>>>>>>>>>>>>>>        >>>>>>>> name?
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> Can you add the interface
>>>>>>>>>>>>>> `KBranchedStream` to
>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>>        with all
>>>>>>>>>>>>>>        >>>>> it's
>>>>>>>>>>>>>>        >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>>> should be
>>>>>>>>>>>>>>        contained in
>>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>>        >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>>        `KBranchedStream#get(int
>>>>>>>>>>>>>>        >>>>>>> index)
>>>>>>>>>>>>>>        >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>>        branched-KStreams. Would
>>>>>>>>>>>>>>        >>>>>> be
>>>>>>>>>>>>>>        >>>>>>>>> nice to get your feedback about it. It
>>>>>>>>>>>>>> seems you
>>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>>        that users
>>>>>>>>>>>>>>        >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>>        access them.
>>>>>>>>>>>>>>        >>>>> We
>>>>>>>>>>>>>>        >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>>> feels
>>>>>>>>>>>>>>        >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>>        to get
>>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>> -Matthias
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>>        >>>>>>>>>> Hi all!
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> I have updated the KIP-418 according to
>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>>        >>>>>>>>>> I can see your point: this is to make
>>>>>>>>>>>>>> the name
>>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>>        >>>>>>> String#split
>>>>>>>>>>>>>>        >>>>>>>>>> that also returns an array, right? But
>>>>>>>>>>>>>> is it
>>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>>        loss of
>>>>>>>>>>>>>>        >>>>>>>> backwards
>>>>>>>>>>>>>>        >>>>>>>>>> compatibility? We can have overloaded
>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>> as well
>>>>>>>>>>>>>>        without
>>>>>>>>>>>>>>        >>>>>>>> affecting
>>>>>>>>>>>>>>        >>>>>>>>>> the existing code. Maybe the old
>>>>>>>>>>>>>> array-based
>>>>>>>>>>>>>> `branch`
>>>>>>>>>>>>> method
>>>>>>>>>>>>>>        >>>>> should
>>>>>>>>>>>>>>        >>>>>>> be
>>>>>>>>>>>>>>        >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>>        >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>>        >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> Totally agree with 'addBranch->branch'
>>>>>>>>>>>>>> rename.
>>>>>>>>>>>>>> 'default'
>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>        >>>>>>> however, a
>>>>>>>>>>>>>>        >>>>>>>>>> reserved word, so unfortunately we
>>>>>>>>>>>>>> cannot have a
>>>>>>>>>>>>>> method
>>>>>>>>>>>>>>        with such
>>>>>>>>>>>>>>        >>>>>>> name
>>>>>>>>>>>>>>        >>>>>>>>> :-)
>>>>>>>>>>>>>>        >>>>>>>>>>> defaultBranch() does take an
>>>>>>>>>>>>>> `Predicate` as
>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>        >>>>> think
>>>>>>>>>>>>>>        >>>>>>> that
>>>>>>>>>>>>>>        >>>>>>>>>> is not required?
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> Absolutely! I think that was just
>>>>>>>>>>>>>> copy-paste
>>>>>>>>>>>>>> error or
>>>>>>>>>>>>>>        something.
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> please revise the new version of the KIP
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>>        >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> Ivan
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>> Thanks for driving the discussion of
>>>>>>>>>>>>>> this KIP.
>>>>>>>>>>>>>> It seems
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>        >>>>>>> everybody
>>>>>>>>>>>>>>        >>>>>>>>>>> agrees that the current branch() method
>>>>>>>>>>>>>> using
>>>>>>>>>>>>>> arrays is
>>>>>>>>>>>>> not
>>>>>>>>>>>>>>        >>>>>> optimal.
>>>>>>>>>>>>>>        >>>>>>>>>>> I had a quick look into the PR and I
>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>        proposal.
>>>>>>>>>>>>>>        >>>>>>> There
>>>>>>>>>>>>>>        >>>>>>>>>>> are some minor things we need to
>>>>>>>>>>>>>> consider. I
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>        recommend the
>>>>>>>>>>>>>>        >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>>        BranchingKStream#default()
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> It's just a suggestion to get slightly
>>>>>>>>>>>>>> shorter
>>>>>>>>>>>>>> method
>>>>>>>>>>>>> names.
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> In the current PR, defaultBranch() does
>>>>>>>>>>>>>> take an
>>>>>>>>>>>>>>        `Predicate` as
>>>>>>>>>>>>>>        >>>>>>>> argument,
>>>>>>>>>>>>>>        >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>>> recently
>>>>>>>>>>>>>>        accepted and
>>>>>>>>>>>>>>        >>>>>> is
>>>>>>>>>>>>>>        >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> Ie, we should add overloads that
>>>>>>>>>>>>>> accepted a
>>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>>        parameter.
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> For the issue that the created
>>>>>>>>>>>>>> `KStream` object
>>>>>>>>>>>>>> are in
>>>>>>>>>>>>>>        different
>>>>>>>>>>>>>>        >>>>>>>> scopes:
>>>>>>>>>>>>>>        >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>>        index)` method
>>>>>>>>>>>>>>        >>>>>>> that
>>>>>>>>>>>>>>        >>>>>>>>>>> returns the corresponding "branched"
>>>>>>>>>>>>>> result
>>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>>        object?
>>>>>>>>>>>>>>        >>>>>> Maybe,
>>>>>>>>>>>>>>        >>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>> second argument of `addBranch()` should
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>>        >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>>        >>>>>>>> but
>>>>>>>>>>>>>>        >>>>>>>>>>> a `Function<KStream,KStream>` and
>>>>>>>>>>>>>> `get()` could
>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>        whatever
>>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> Finally, I would also suggest to update
>>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>        current
>>>>>>>>>>>>>>        >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>>        >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>        makes sense
>>>>>>>>>>>>>>        >>>>>> for
>>>>>>>>>>>>>>        >>>>>>>> you
>>>>>>>>>>>>>>        >>>>>>>>> to
>>>>>>>>>>>>>>        >>>>>>>>>>>> revise the KIP and continue the
>>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>        we'll
>>>>>>>>>>>>>>        >>>>> need
>>>>>>>>>>>>>>        >>>>>>>> some
>>>>>>>>>>>>>>        >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>>        >>>>> whether
>>>>>>>>>>>>>>        >>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> KIP
>>>>>>>>>>>>>>        >>>>>>>>>>>> could be adopted.  It would be great
>>>>>>>>>>>>>> to hear
>>>>>>>>>>>>>> if they
>>>>>>>>>>>>>>        think this
>>>>>>>>>>>>>>        >>>>>> is
>>>>>>>>>>>>>>        >>>>>>> a
>>>>>>>>>>>>>>        >>>>>>>>> good
>>>>>>>>>>>>>>        >>>>>>>>>>>> idea overall.  I'm not sure if that
>>>>>>>>>>>>>> happens
>>>>>>>>>>>>>> just by
>>>>>>>>>>>>>>        starting a
>>>>>>>>>>>>>>        >>>>>>> vote,
>>>>>>>>>>>>>>        >>>>>>>>> or if
>>>>>>>>>>>>>>        >>>>>>>>>>>> there is generally some indication of
>>>>>>>>>>>>>> interest
>>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>> That being said, I'll continue the
>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>>        assuming
>>>>>>>>>>>>>>        >>>>> we
>>>>>>>>>>>>>>        >>>>>> do
>>>>>>>>>>>>>>        >>>>>>>>> move
>>>>>>>>>>>>>>        >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>>> returns
>>>>>>>>>>>>>>        >>>>>> KBranchedStream",
>>>>>>>>>>>>>>        >>>>>>> do
>>>>>>>>>>>>>>        >>>>>>>>> we
>>>>>>>>>>>>>>        >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>>> would
>>>>>>>>>>>>>>        >>>>> favor
>>>>>>>>>>>>>>        >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>        >>>>>>> accomplish
>>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>>        similar
>>>>>>>>>>>>>>        >>>>>>>>> anyway.  We
>>>>>>>>>>>>>>        >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>        >>>>>>> impossible/difficult
>>>>>>>>>>>>>>        >>>>>>>>> that
>>>>>>>>>>>>>>        >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>>> structure would
>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>        >>>>> it's
>>>>>>>>>>>>>>        >>>>>>>> just a
>>>>>>>>>>>>>>        >>>>>>>>>>>> little sloppy overall in terms of
>>>>>>>>>>>>>> naming and
>>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>>        >>>>>>> particular,
>>>>>>>>>>>>>>        >>>>>>>>>>>> passing in the "predicates" and
>>>>>>>>>>>>>> "children"
>>>>>>>>>>>>>> lists which
>>>>>>>>>>>>> get
>>>>>>>>>>>>>>        >>>>>> modified
>>>>>>>>>>>>>>        >>>>>>>> in
>>>>>>>>>>>>>>        >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>>        KStreamLazyBranch is
>>>>>>>>>>>>>>        >>>>> a
>>>>>>>>>>>>>>        >>>>>>> bit
>>>>>>>>>>>>>>        >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>        >>>>>>>>>>>> Paul
>>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>        >>>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>> I read your code carefully and now I
>>>>>>>>>>>>>> am fully
>>>>>>>>>>>>>>        convinced: your
>>>>>>>>>>>>>>        >>>>>>>> proposal
>>>>>>>>>>>>>>        >>>>>>>>>>>>> looks better and should work. We just
>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>> document
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>> crucial
>>>>>>>>>>>>>>        >>>>>>>>> fact
>>>>>>>>>>>>>>        >>>>>>>>>>>>> that KStream consumers are invoked as
>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>> added.
>>>>>>>>>>>>>>        And then
>>>>>>>>>>>>>>        >>>>>> it's
>>>>>>>>>>>>>>        >>>>>>>> all
>>>>>>>>>>>>>>        >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>> What shall we do now? I should
>>>>>>>>>>>>>> re-write the
>>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>>        resume the
>>>>>>>>>>>>>>        >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>> Why are you telling that your PR
>>>>>>>>>>>>>> 'should not
>>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>>        >>>>> starting
>>>>>>>>>>>>>>        >>>>>>>> point
>>>>>>>>>>>>>>        >>>>>>>>> if
>>>>>>>>>>>>>>        >>>>>>>>>>>>> we go in this direction'? To me it
>>>>>>>>>>>>>> looks like
>>>>>>>>>>>>>> a good
>>>>>>>>>>>>>>        starting
>>>>>>>>>>>>>>        >>>>>>> point.
>>>>>>>>>>>>>>        >>>>>>>>> But
>>>>>>>>>>>>>>        >>>>>>>>>>>>> as a novice in this project I might
>>>>>>>>>>>>>> miss some
>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>        >>>>> details.
>>>>>>>>>>>>>>        >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>>        >>>>> stream.branch()
>>>>>>>>>>>>>>        >>>>>>>>> solution
>>>>>>>>>>>>>>        >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>>        >>>>> invoked
>>>>>>>>>>>>>>        >>>>>> as
>>>>>>>>>>>>>>        >>>>>>>>> they’re
>>>>>>>>>>>>>>        >>>>>>>>>>>>> added, not during
>>>>>>>>>>>>>> streamsBuilder.build(). So
>>>>>>>>>>>>>> the user
>>>>>>>>>>>>>>        still
>>>>>>>>>>>>>>        >>>>>> ought
>>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>>        >>>>>>>>> be
>>>>>>>>>>>>>>        >>>>>>>>>>>>> able to call couponIssuer.coupons()
>>>>>>>>>>>>>> afterward
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>        depend on
>>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> branched
>>>>>>>>>>>>>>        >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>> The issue I mean to point out is
>>>>>>>>>>>>>> that it is
>>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>>        access
>>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> branched
>>>>>>>>>>>>>>        >>>>>>>>>>>>> streams in the same scope as the
>>>>>>>>>>>>>> original
>>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>>        is, not
>>>>>>>>>>>>>>        >>>>>>>> inside
>>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>> couponIssuer), which is a problem
>>>>>>>>>>>>>> with both
>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>        >>>>> solutions.
>>>>>>>>>>>>>>        >>>>>> It
>>>>>>>>>>>>>>        >>>>>>>>> can be
>>>>>>>>>>>>>>        >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>> [Also, great to hear additional
>>>>>>>>>>>>>> interest in
>>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>>        excited
>>>>>>>>>>>>>>        >>>>> to
>>>>>>>>>>>>>>        >>>>>>>> hear
>>>>>>>>>>>>>>        >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>>        >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>>        >>>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> The idea to postpone the wiring of
>>>>>>>>>>>>>> branches
>>>>>>>>>>>>>> to the
>>>>>>>>>>>>>>        >>>>>>>>>>>>> streamsBuilder.build() also looked
>>>>>>>>>>>>>> great for
>>>>>>>>>>>>>> me at
>>>>>>>>>>>>> first
>>>>>>>>>>>>>>        >>>>> glance,
>>>>>>>>>>>>>>        >>>>>>> but
>>>>>>>>>>>>>>        >>>>>>>>> ---
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>>        same
>>>>>>>>>>>>>>        >>>>>> scope
>>>>>>>>>>>>>>        >>>>>>> as
>>>>>>>>>>>>>>        >>>>>>>>> each
>>>>>>>>>>>>>>        >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>>> them back
>>>>>>>>>>>>> together
>>>>>>>>>>>>>>        >>>>> again
>>>>>>>>>>>>>>        >>>>>> I
>>>>>>>>>>>>>>        >>>>>>>>> don't see
>>>>>>>>>>>>>>        >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> You just took the words right out
>>>>>>>>>>>>>> of my
>>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>>        just
>>>>>>>>>>>>>>        >>>>>> going
>>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>>        >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Consider the example from Bill's
>>>>>>>>>>>>>> book, p.
>>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>>        we need
>>>>>>>>>>>>>>        >>>>> to
>>>>>>>>>>>>>>        >>>>>>>>> identify
>>>>>>>>>>>>>>        >>>>>>>>>>>>> customers who have bought coffee and
>>>>>>>>>>>>>> made a
>>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>>        in the
>>>>>>>>>>>>>>        >>>>>>>>> electronics
>>>>>>>>>>>>>>        >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>        circumstances
>>>>>>>>>>>>>>        >>>>>> using
>>>>>>>>>>>>>>        >>>>>>>> my
>>>>>>>>>>>>>>        >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>> coffePurchases;
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       /*In the real world the code
>>>>>>>>>>>>>> here
>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>        complex, so
>>>>>>>>>>>>>>        >>>>>>>>> creation of
>>>>>>>>>>>>>>        >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>>        order to
>>>>>>>>>>>>>>        >>>>>>>> separate
>>>>>>>>>>>>>>        >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>>        >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> /*Alas, this won't work if we're
>>>>>>>>>>>>>> going to
>>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>>        everything
>>>>>>>>>>>>>>        >>>>>>>> later,
>>>>>>>>>>>>>>        >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Does this make sense?  In order to
>>>>>>>>>>>>>> properly
>>>>>>>>>>>>>>        initialize the
>>>>>>>>>>>>>>        >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>>        >>>>>>>>>>>>> we need the terminal operation to be
>>>>>>>>>>>>>> called
>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>        >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>>        >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>>        essentially
>>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> next
>>>>>>>>>>>>>>        >>>>>>>>>>>>> KIP I was going to write here. I have
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>>        based on
>>>>>>>>>>>>>>        >>>>> my
>>>>>>>>>>>>>>        >>>>>>>>> experience,
>>>>>>>>>>>>>>        >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>>        fluent
>>>>>>>>>>>>>>        >>>>> API
>>>>>>>>>>>>>>        >>>>>>>> based
>>>>>>>>>>>>>>        >>>>>>>>>>>>> off of
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>>        (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>>        >>>>>> and
>>>>>>>>>>>>>>        >>>>>>> I
>>>>>>>>>>>>>>        >>>>>>>>> think
>>>>>>>>>>>>>>        >>>>>>>>>>>>> I
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>>        >>>>>>> compatibility
>>>>>>>>>>>>>>        >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    there aren't any direct ones. 
>>>>>>>>>>>>>> I was
>>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>>        that Java
>>>>>>>>>>>>>>        >>>>> is
>>>>>>>>>>>>>>        >>>>>>>> smart
>>>>>>>>>>>>>>        >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>>        returning one
>>>>>>>>>>>>>>        >>>>>>> thing
>>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>>        >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    with no arguments returning
>>>>>>>>>>>>>> another
>>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Requiring a terminal method:
>>>>>>>>>>>>>> We don't
>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>        need
>>>>>>>>>>>>>>        >>>>> it.
>>>>>>>>>>>>>>        >>>>>>> We
>>>>>>>>>>>>>>        >>>>>>>>> can
>>>>>>>>>>>>>>        >>>>>>>>>>>>> just
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>>        shares
>>>>>>>>>>>>>>        >>>>>> its
>>>>>>>>>>>>>>        >>>>>>>>> state
>>>>>>>>>>>>>>        >>>>>>>>>>>>> with the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    ProcessorSupplier that will
>>>>>>>>>>>>>> actually do
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        branching.
>>>>>>>>>>>>>>        >>>>>>> It's
>>>>>>>>>>>>>>        >>>>>>>>> not
>>>>>>>>>>>>>>        >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>>> think it
>>>>>>>>>>>>>>        demonstrates
>>>>>>>>>>>>>>        >>>>>> its
>>>>>>>>>>>>>>        >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>>> request should
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>        >>>>> final
>>>>>>>>>>>>>>        >>>>>> or
>>>>>>>>>>>>>>        >>>>>>>>> even a
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> starting point if we go in this
>>>>>>>>>>>>>> direction,
>>>>>>>>>>>>>> I just
>>>>>>>>>>>>>>        wanted to
>>>>>>>>>>>>>>        >>>>>> see
>>>>>>>>>>>>>>        >>>>>>>> how
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> challenging it would be to get the
>>>>>>>>>>>>>> API
>>>>>>>>>>>>>> working.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I will say though, that I'm not
>>>>>>>>>>>>>> sure the
>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>        solution
>>>>>>>>>>>>>>        >>>>>>> could
>>>>>>>>>>>>>>        >>>>>>>> be
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecated in favor of this, which
>>>>>>>>>>>>>> I had
>>>>>>>>>>>>>> originally
>>>>>>>>>>>>>>        >>>>> suggested
>>>>>>>>>>>>>>        >>>>>>>> was a
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> possibility.  The reason is that
>>>>>>>>>>>>>> the newly
>>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>        streams
>>>>>>>>>>>>>>        >>>>>> are
>>>>>>>>>>>>>>        >>>>>>>> not
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>>        is, if we
>>>>>>>>>>>>>>        >>>>>>> wanted
>>>>>>>>>>>>>>        >>>>>>>>> to
>>>>>>>>>>>>>>        >>>>>>>>>>>>> merge
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> them back together again I don't
>>>>>>>>>>>>>> see a way
>>>>>>>>>>>>>> to do
>>>>>>>>>>>>>>        that.  The
>>>>>>>>>>>>>>        >>>>>> KIP
>>>>>>>>>>>>>>        >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>>> means is that
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>        >>>>>> either
>>>>>>>>>>>>>>        >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecating the existing
>>>>>>>>>>>>>> branch(...) is
>>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>>        table.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>> <ma...@mail.ru>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>        >>>>>> point.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> First, it seems that it's
>>>>>>>>>>>>>> commonly agreed
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>        branch API
>>>>>>>>>>>>>>        >>>>>>> needs
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> There are two potential ways to
>>>>>>>>>>>>>> do it:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>   
>>>>>>>>>>>>>> .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>>        returns
>>>>>>>>>>>>>>        >>>>>> its
>>>>>>>>>>>>>>        >>>>>>>>> argument
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards
>>>>>>>>>>>>>> compatible. 2)
>>>>>>>>>>>>>> The code
>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>        >>>>> make
>>>>>>>>>>>>>>        >>>>>>>> sense
>>>>>>>>>>>>>>        >>>>>>>>>>>>> until
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>        instance
>>>>>>>>>>>>>>        >>>>>>>>> contrasts the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>>        >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>>        >>>>>>>>>>>>> and
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: Generally follows the way
>>>>>>>>>>>>>> KStreams
>>>>>>>>>>>>>> interface
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>        >>>>>> defined.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>        >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault()). And for a user it
>>>>>>>>>>>>>> is very
>>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>>        miss the
>>>>>>>>>>>>>>        >>>>>> fact
>>>>>>>>>>>>>>        >>>>>>>>> that one
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> of the terminal methods should be
>>>>>>>>>>>>>> called.
>>>>>>>>>>>>>> If these
>>>>>>>>>>>>>>        methods
>>>>>>>>>>>>>>        >>>>>> are
>>>>>>>>>>>>>>        >>>>>>>> not
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Colleagues, what are your
>>>>>>>>>>>>>> thoughts? Can
>>>>>>>>>>>>>> we do
>>>>>>>>>>>>> better?
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev
>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev
>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> I see your point when you are
>>>>>>>>>>>>>> talking
>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>>        implemented the
>>>>>>>>>>>>>>        >>>>>>> easy
>>>>>>>>>>>>>>        >>>>>>>>> way.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Let me comment on two of your
>>>>>>>>>>>>>> ideas.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>>> method that
>>>>>>>>>>>>> assumes
>>>>>>>>>>>>>>        >>>>> nothing
>>>>>>>>>>>>>>        >>>>>>>> will
>>>>>>>>>>>>>>        >>>>>>>>>>>>> reach
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> throwing an exception if such a
>>>>>>>>>>>>>> case
>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should
>>>>>>>>>>>>>> not be
>>>>>>>>>>>>>> the only
>>>>>>>>>>>>> option
>>>>>>>>>>>>>>        >>>>>> besides
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> `default`, because there are
>>>>>>>>>>>>>> scenarios
>>>>>>>>>>>>>> when we
>>>>>>>>>>>>>>        want to
>>>>>>>>>>>>>>        >>>>>> just
>>>>>>>>>>>>>>        >>>>>>>>> silently
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> drop the messages that didn't
>>>>>>>>>>>>>> match any
>>>>>>>>>>>>>>        predicate. 2)
>>>>>>>>>>>>>>        >>>>>>> Throwing
>>>>>>>>>>>>>>        >>>>>>>>> an
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> exception in the middle of data
>>>>>>>>>>>>>> flow
>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>        looks
>>>>>>>>>>>>>>        >>>>>> like a
>>>>>>>>>>>>>>        >>>>>>>> bad
>>>>>>>>>>>>>>        >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> In stream processing paradigm,
>>>>>>>>>>>>>> I would
>>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>>        emit a
>>>>>>>>>>>>>>        >>>>>>>> special
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> message to a dedicated stream.
>>>>>>>>>>>>>> This is
>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>> where
>>>>>>>>>>>>>>        >>>>>>> `default`
>>>>>>>>>>>>>>        >>>>>>>>> can
>>>>>>>>>>>>>>        >>>>>>>>>>>>> be
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>>        >>>>>>>>> track
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>        a clear
>>>>>>>>>>>>>>        >>>>>>> error
>>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> You mean a runtime exception,
>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>> program is
>>>>>>>>>>>>>>        >>>>> compiled
>>>>>>>>>>>>>>        >>>>>>> and
>>>>>>>>>>>>>>        >>>>>>>>> run?
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that
>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>> won't
>>>>>>>>>>>>>>        compile if
>>>>>>>>>>>>>>        >>>>> used
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>>        method chain
>>>>>>>>>>>>>>        >>>>>>>> starting
>>>>>>>>>>>>>>        >>>>>>>>>>>>> from
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> KStream object? There is a huge
>>>>>>>>>>>>>> cost
>>>>>>>>>>>>>> difference
>>>>>>>>>>>>>>        between
>>>>>>>>>>>>>>        >>>>>>>> runtime
>>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a
>>>>>>>>>>>>>> failure
>>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>>        >>>>> instantly
>>>>>>>>>>>>>>        >>>>>> on
>>>>>>>>>>>>>>        >>>>>>>>> unit
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> tests, it costs more for the
>>>>>>>>>>>>>> project
>>>>>>>>>>>>>> than a
>>>>>>>>>>>>>>        compilation
>>>>>>>>>>>>>>        >>>>>>>> failure.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen
>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>>        required.
>>>>>>>>>>>>>>        >>>>>>> But
>>>>>>>>>>>>>>        >>>>>>>> is
>>>>>>>>>>>>>>        >>>>>>>>>>>>> that
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user
>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>> want a
>>>>>>>>>>>>>>        >>>>>> defaultBranch
>>>>>>>>>>>>>>        >>>>>>>>> they
>>>>>>>>>>>>>>        >>>>>>>>>>>>> can
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>>        just as
>>>>>>>>>>>>>>        >>>>>>>>> easily.  In
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> think it creates an
>>>>>>>>>>>>>> opportunity for a
>>>>>>>>>>>>>> nicer API
>>>>>>>>>>>>> - a
>>>>>>>>>>>>>>        >>>>> user
>>>>>>>>>>>>>>        >>>>>>>> could
>>>>>>>>>>>>>>        >>>>>>>>>>>>> specify
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> terminal method that assumes
>>>>>>>>>>>>>> nothing
>>>>>>>>>>>>>> will reach
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>> default
>>>>>>>>>>>>>>        >>>>>>>>> branch,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> throwing an exception if such
>>>>>>>>>>>>>> a case
>>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>> That
>>>>>>>>>>>>>>        >>>>> seems
>>>>>>>>>>>>>>        >>>>>>> like
>>>>>>>>>>>>>>        >>>>>>>>> an
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> improvement over the current
>>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>        which allows
>>>>>>>>>>>>>>        >>>>>> for
>>>>>>>>>>>>>>        >>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>> more
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>>> getting
>>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>>        to be
>>>>>>>>>>>>>>        >>>>>> well
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>>        >>>>>>>>> track
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> branches that haven't been
>>>>>>>>>>>>>> terminated
>>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>>        a clear
>>>>>>>>>>>>>>        >>>>>>> error
>>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially
>>>>>>>>>>>>>> now that
>>>>>>>>>>>>>> there is
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>        >>>>> "build
>>>>>>>>>>>>>>        >>>>>>>> step"
>>>>>>>>>>>>>>        >>>>>>>>>>>>> where
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> topology is actually wired up,
>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>        >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>>        >>>>>>> is
>>>>>>>>>>>>>>        >>>>>>>>>>>>> called.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>>> argument, I
>>>>>>>>>>>>> agree
>>>>>>>>>>>>>>        >>>>> that
>>>>>>>>>>>>>>        >>>>>>> it's
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> allow users to do other
>>>>>>>>>>>>>> operations on
>>>>>>>>>>>>>> the input
>>>>>>>>>>>>>>        stream.
>>>>>>>>>>>>>>        >>>>>>> With
>>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> solution, it ought to work the
>>>>>>>>>>>>>> same
>>>>>>>>>>>>>> way all
>>>>>>>>>>>>> other
>>>>>>>>>>>>>>        >>>>>>> operations
>>>>>>>>>>>>>>        >>>>>>>>> do -
>>>>>>>>>>>>>>        >>>>>>>>>>>>> if
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>        >>>>> times,
>>>>>>>>>>>>>>        >>>>>>> you
>>>>>>>>>>>>>>        >>>>>>>>> just
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> stream as a variable so you
>>>>>>>>>>>>>> can call
>>>>>>>>>>>>>> as many
>>>>>>>>>>>>>>        operations
>>>>>>>>>>>>>>        >>>>>> on
>>>>>>>>>>>>>>        >>>>>>> it
>>>>>>>>>>>>>>        >>>>>>>>> as
>>>>>>>>>>>>>>        >>>>>>>>>>>>> you
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02
>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>> <ma...@mail.ru>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I afraid this won't work
>>>>>>>>>>>>>> because we
>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>        always need
>>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a
>>>>>>>>>>>>>> terminal
>>>>>>>>>>>>> operation we
>>>>>>>>>>>>>>        >>>>> don't
>>>>>>>>>>>>>>        >>>>>>>> know
>>>>>>>>>>>>>>        >>>>>>>>>>>>> when to
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf
>>>>>>>>>>>>>> returns its
>>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>>        so we
>>>>>>>>>>>>>>        >>>>> can
>>>>>>>>>>>>>>        >>>>>> do
>>>>>>>>>>>>>>        >>>>>>>>>>>>> something
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> more with the original branch
>>>>>>>>>>>>>> after
>>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>>> need of
>>>>>>>>>>>>> special
>>>>>>>>>>>>>>        >>>>> object
>>>>>>>>>>>>>>        >>>>>>>>>>>>> construction
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most
>>>>>>>>>>>>>> KStream
>>>>>>>>>>>>>> methods.
>>>>>>>>>>>>> But
>>>>>>>>>>>>>>        >>>>> here
>>>>>>>>>>>>>>        >>>>>> we
>>>>>>>>>>>>>>        >>>>>>>>> have a
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> special case: we build the
>>>>>>>>>>>>>> switch to
>>>>>>>>>>>>>> split the
>>>>>>>>>>>>>>        flow,
>>>>>>>>>>>>>>        >>>>> so
>>>>>>>>>>>>>>        >>>>>> I
>>>>>>>>>>>>>>        >>>>>>>>> think
>>>>>>>>>>>>>>        >>>>>>>>>>>>> this
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen
>>>>>>>>>>>>>> пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to
>>>>>>>>>>>>>> improve
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>        API, but I
>>>>>>>>>>>>>>        >>>>>> find
>>>>>>>>>>>>>>        >>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>>> since it
>>>>>>>>>>>>>>        contrasts the
>>>>>>>>>>>>>>        >>>>>>> fluency
>>>>>>>>>>>>>>        >>>>>>>>> of
>>>>>>>>>>>>>>        >>>>>>>>>>>>> other
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> KStream method calls. 
>>>>>>>>>>>>>> Ideally I'd
>>>>>>>>>>>>>> like to
>>>>>>>>>>>>>>        just call
>>>>>>>>>>>>>>        >>>>> a
>>>>>>>>>>>>>>        >>>>>>>>> method on
>>>>>>>>>>>>>>        >>>>>>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> so it still reads top to
>>>>>>>>>>>>>> bottom if
>>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>>        cases
>>>>>>>>>>>>>>        >>>>> are
>>>>>>>>>>>>>>        >>>>>>>>> defined
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think the
>>>>>>>>>>>>>> addBranch(predicate,
>>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>>        is very
>>>>>>>>>>>>>>        >>>>>> nice
>>>>>>>>>>>>>>        >>>>>>>>> and the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we
>>>>>>>>>>>>>> flipped
>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>        how we
>>>>>>>>>>>>>>        >>>>>>> specify
>>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>> source
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>          
>>>>>>>>>>>>>> .addBranch(predicate1,
>>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>          
>>>>>>>>>>>>>> .addBranch(predicate2,
>>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>>        >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>>        >>>>>>>>> or
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> which is added to by
>>>>>>>>>>>>>> addBranch() and
>>>>>>>>>>>>>>        terminated by
>>>>>>>>>>>>>>        >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> returns void).  This is
>>>>>>>>>>>>>> obviously
>>>>>>>>>>>>>>        incompatible with
>>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> current
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would
>>>>>>>>>>>>>> have to
>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>        different
>>>>>>>>>>>>>>        >>>>>> name,
>>>>>>>>>>>>>>        >>>>>>>> but
>>>>>>>>>>>>>>        >>>>>>>>> that
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem
>>>>>>>>>>>>>> - we
>>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>>        >>>>>> something
>>>>>>>>>>>>>>        >>>>>>>> like
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and
>>>>>>>>>>>>>> deprecate the
>>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the
>>>>>>>>>>>>>> motivations of
>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>        KIP?  It
>>>>>>>>>>>>>>        >>>>>> seems
>>>>>>>>>>>>>>        >>>>>>>>> like it
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>>> branching
>>>>>>>>>>>>>>        while also
>>>>>>>>>>>>>>        >>>>>>>> allowing
>>>>>>>>>>>>>>        >>>>>>>>> you
>>>>>>>>>>>>>>        >>>>>>>>>>>>> to
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> dynamically build of
>>>>>>>>>>>>>> branches off of
>>>>>>>>>>>>>>        KBranchedStreams
>>>>>>>>>>>>>>        >>>>>> if
>>>>>>>>>>>>>>        >>>>>>>>> desired.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28
>>>>>>>>>>>>>> PM Ivan
>>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>>        ks){
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>>        String> ks){
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>> KafkaStreamsBrancher<String,
>>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>>        this::handleFirstCase)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>>        this::handleSecondCase)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill
>>>>>>>>>>>>>> Bejeck пишет:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>>        >>>>> takes a
>>>>>>>>>>>>>>        >>>>>>>>> Consumer
>>>>>>>>>>>>>>        >>>>>>>>>>>>> as a
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> argument which returns
>>>>>>>>>>>>>> nothing,
>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>>        example in
>>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>>        >>>>>>>> KIP
>>>>>>>>>>>>>>        >>>>>>>>>>>>> shows
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch
>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>>        >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed
>>>>>>>>>>>>>> something, but
>>>>>>>>>>>>>> how would
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>        >>>>> handle
>>>>>>>>>>>>>>        >>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>> case
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>>> wants to
>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>        >>>>>>>> processing
>>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>>        >>>>>>>>>>>>> not
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal
>>>>>>>>>>>>>> node on
>>>>>>>>>>>>>> the
>>>>>>>>>>>>> branched
>>>>>>>>>>>>>>        >>>>>> stream
>>>>>>>>>>>>>>        >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's
>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>>        we had
>>>>>>>>>>>>>>        >>>>>>>> something
>>>>>>>>>>>>>>        >>>>>>>>> like
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at
>>>>>>>>>>>>>> 6:15 PM
>>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>>> <
>>>>>>>>>>>>>>        >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>>> discussion for
>>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>>        >>>>> 418.
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a
>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>> about
>>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>>        >>>>> Please
>>>>>>>>>>>>>>        >>>>>>>> take
>>>>>>>>>>>>>>        >>>>>>>>> a
>>>>>>>>>>>>>>        >>>>>>>>>>>>> look
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>>        feedback :)
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>>        >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>>        >
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hello, John, hello Matthias!

Thank you very much for your detailed feedback!

-----------------------------------------

John,

 > It looks like you missed my reply on Apr 23rd.

For some unknown reason it didn't reach my inbox, fortunately we have 
all the emails on the web.

 > 1. Can you propose to deprecate (but not remove) the existing 
‘branch’ method?

Done, in "Compatibility, Deprecation, and Migration Plan" section.

 > 2. [Explain why 'branch' operator is superior to branching directly 
off of the parent KStream for the needs of dynamic branching]

Done, see an ugly counterexample in 'Dynamic Branching' section.

 > 3. [`withConsumer` is confusing with Kafka Consumer... maybe `withSink`?]

As Mathhias noted, `withSink` can also be confusing. I renamed this 
method to `withJavaConsumer` per Matthias' suggestion.

 > 4. ...It seems like there are two disjoint use cases: EITHER using 
chain and the result map OR using just the sink

This is discussed below.

----------------------------------------------

Mathhias,

 > 1. [We should rename `KBranchedStream` -> `BranchedKStream`]

Done.

 > 2. [Ambiguous phrase about 'parameterless' version of the `branch` 
method]

Fixed.


 > 3. Overview of newly added methods/interfaces

Done in `Proposed Changes` section.


 > 4. [Concerning John's note] > I don't think that using both 
`withChain()` and `withConsumer()` is the
issue, as the KIP clearly states that the result of `withChain()` will 
be given to the `Consumer`.

Yes, I agree!

 > The issue is really with the `Consumer` and the returned `Map` of 
`defautBranch()` and `noDefaultBranch()`. Maybe a reasonable 
implementation would be to not add the "branch" to the result map if 
`withConsumer` is used?

But is it also an issue? With Kafka Streams, we can split the topology 
graph at any point. Technically, it's OK to do both: feed the KStream to 
a [Java]Consumer AND save it in resulting Map. If one doesn't need the 
stream in the Map, one simply does not extract it from there :-)

In the current version of KIP it is assumed that the returned map 
contains ALL the branches, either tagged with IDs explicitly set by the 
programmer, or with some default auto-generated ids. Dealing with this 
map is the user's responsibility.

What seems to me to be an issue is introducing exclusions to this 
general rule, like 'swallowing' some streams by provided 
[Java]Consumers. This can make things complicated. What if a user 
provides both the name of the branch and a [Java]Consumer? What do they 
mean in this case? Should we 'swallow' the stream or save it to the map? 
There's no point in 'saving the space' in this map, so maybe just leave 
it as it is?

----

I rewrote the KIP and also fixed a couple of typos.

Looking forward for your feedback again!

Regards,

Ivan.



08.05.2020 22:55, Matthias J. Sax пишет:
> Thanks for updating the KIP!
> 
> I also have some minor comment:
> 
> 
> 
> (1) We should rename `KBranchedStream` -> `BranchedKStream`
> 
> (Most classed follow this naming pattern now, eg, CoGroupedKStream,
> TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
> and `KGroupedKTable` that we cannot rename without a breaking change...
> so we just keep them.)
> 
> 
> 
> (2) Quote:
> 
>> Both branch and defaultBranch operations also have overloaded parameterless alternatives.
> 
> I think `branch()` always needs to take a `Predicate` and assume you
> meant that `Branched` is optional. Can you maybe rephrase it accordingly
> as `branch()` would not be "parameterless".
> 
> 
> 
> (3) Can you maybe add an overview in the "Public Interface" section) of
> newly added and deprecated methods/classes (cf.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)
> 
> 
> 
> (4) What is unclear from the KIP is the interaction of `withConsumer()`
> and the finally returned `Map<String, KStream>`. This related to John's
> 4th comment:
> 
>> It seems like there are really two disjoint use cases: EITHER using chain and the result map OR using just the sink.
> 
> I don't think that using both `withChain()` and `withConsumer()` is the
> issue though, as the KIP clearly states that the result of `withChain()`
> will be given to the `Consumer`. The issue is really with the `Consumer`
> and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.
> 
> Maybe a reasonable implementation would be to not add the "branch" to
> the result map if `withConsumer` is used? As long as we clearly document
> it in the JavaDocs, this might be fine?
> 
> 
> 
> (5) Reply to John's comments:
> 
>> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you were talking about the kafka Consumer interface (which doesn’t make sense, of course). I get that you were referring to the java Consumer interface, but we should still probably to to avoid the ambiguity. Just throwing out a suggestion, how about ‘withSink’?
> 
> IMHO, `withSink` has the issue that it might be confused with a "sink
> node", ie., writing the KStream to a topic.
> 
> Maybe `withJavaConsumer` would make it less ambiguous?
> 
> 
> 
> 
> -Matthias
> 
> 
> 
> 
> On 5/8/20 7:13 AM, John Roesler wrote:
>> Hi Ivan,
>>
>> It looks like you missed my reply on Apr 23rd. I think it’s close, but I had a few last comments.
>>
>> Thanks,
>> John
>>
>> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>>> Hello everyone,
>>>
>>> will someone please take a look at the reworked KIP?
>>>
>>> I believe that now it follows design principles and takes into account
>>> all the arguments discussed here.
>>>
>>>
>>> Regards,
>>>
>>> Ivan
>>>
>>>
>>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>>> Hi,
>>>>
>>>> I have read the John's "DSL design principles" and have completely
>>>> rewritten the KIP, see
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>
>>>>
>>>>
>>>> This version includes all the previous discussion results and follows
>>>> the design principles, with one exception.
>>>>
>>>> The exception is
>>>>
>>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>>
>>>> which formally violates 'no more than one parameter' rule, but I think
>>>> here it is justified.
>>>>
>>>> We must provide a predicate for a branch and don't need to provide one
>>>> for the default branch. Thus for both operations we may use a single
>>>> Branched parameter class, with an extra method parameter for `branch`.
>>>>
>>>> Since predicate is a natural, necessary part of a branch, no
>>>> 'proliferation of overloads, deprecations, etc.' is expected here as it
>>>> is said in the rationale for the 'single parameter rule'.
>>>>
>>>> WDYT, is this KIP mature enough to begin voting?
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>>> Ivan,
>>>>>
>>>>> no worries about getting side tracked. Glad to have you back!
>>>>>
>>>>> The DSL improved further in the meantime and we already have a `Named`
>>>>> config object to name operators. It seems reasonable to me to build on
>>>>> this.
>>>>>
>>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>>> want to follow:
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
>>>>>
>>>>> -- might be worth to checkout.
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>>> Hi everyone!
>>>>>>
>>>>>> Let me revive the discussion of this KIP.
>>>>>>
>>>>>> I'm very sorry for stopping my participation in the discussion in June
>>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>>> spare time. But I think I must finish this, because we invested
>>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>>> propose other things before this one is finalized.
>>>>>>
>>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>>> KafkaStreamBrancher class of my invention (the original idea for this
>>>>>> KIP) worked for me -- that's another reason why I gave up pushing the
>>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>>> branches, I worked around it this way:
>>>>>>
>>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>>> new KafkaStreamBrancher<....>()
>>>>>>       .branch(....)
>>>>>>       .defaultBranch(result::set)
>>>>>>       .onTopOf(someStream);
>>>>>> result.get()...
>>>>>>
>>>>>>
>>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>>
>>>>>> I think that Matthias came up with a bright solution in his post from
>>>>>> May, 24th 2019. Let me quote it:
>>>>>>
>>>>>> KStream#split() -> KBranchedStream
>>>>>> // branch is not easily accessible in current scope
>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>     -> KBranchedStream
>>>>>> // assign a name to the branch and
>>>>>> // return the sub-stream to the current scope later
>>>>>> //
>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>     -> KBranchedStream
>>>>>> // default branch is not easily accessible
>>>>>> // return map of all named sub-stream into current scope
>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>     -> Map<String,KStream>
>>>>>> // assign custom name to default-branch
>>>>>> // return map of all named sub-stream into current scope
>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>     -> Map<String,KStream>
>>>>>> // assign a default name for default
>>>>>> // return map of all named sub-stream into current scope
>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>     -> Map<String,KStream>
>>>>>> // return map of all names sub-stream into current scope
>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>     -> Map<String,KStream>
>>>>>>
>>>>>> I believe this would satisfy everyone. Optional names seems to be a good
>>>>>> idea: when you don't need to have the branches in the same scope, you
>>>>>> just don't use names and you don't risk making your code brittle. Or,
>>>>>> you might want to add names just for debugging purposes. Or, finally,
>>>>>> you might use the returned Map to have the named branches in the
>>>>>> original scope.
>>>>>>
>>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>>> familiar with Streams API design principles than me.
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>>
>>>>>>
>>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>>> I am moving this KIP into "inactive status". Feel free to resume the
>>>>>>> KIP
>>>>>>> at any point.
>>>>>>>
>>>>>>> If anybody else is interested in picking up this KIP, feel free to
>>>>>>> do so.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>>> Ivan,
>>>>>>>>
>>>>>>>> did you see my last reply? What do you think about my proposal to mix
>>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>>> Thanks for the input John!
>>>>>>>>>
>>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>>
>>>>>>>>> If you want to get the `KStream` as part of the `Map` back using a
>>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern
>>>>>>>>> using a
>>>>>>>>> `Consumer`, no.
>>>>>>>>>
>>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>>
>>>>>>>>> I wanted to sketch out a high level proposal to merge both patterns
>>>>>>>>> only. Your suggestions to align the new API with the existing API
>>>>>>>>> make
>>>>>>>>> totally sense.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>>
>>>>>>>>> If both are mandatory, what do we gain by it? The returned `Map` only
>>>>>>>>> contains the corresponding branches, so why should we prefix all of
>>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>>> `split()`, the same question raises?
>>>>>>>>>
>>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if
>>>>>>>>> `Named` is
>>>>>>>>> optional in `branch()` and we generate `-X` suffix using a counter
>>>>>>>>> for
>>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>>> changing names if branches are added/removed. Also, how would the
>>>>>>>>> names
>>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>>> returned in the `Map`).
>>>>>>>>>
>>>>>>>>> If `Named` is optional for both, it could happen that a user
>>>>>>>>> misses to
>>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Hence, I am actually in favor to not allow a default name but keep
>>>>>>>>> `split()` without parameter and make `Named` in `branch()` required
>>>>>>>>> if a
>>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>>> specifying a
>>>>>>>>> name is required if a `Function` is used.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> About
>>>>>>>>>
>>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>>
>>>>>>>>> I don't think that the branching predicate is a configuration and
>>>>>>>>> hence
>>>>>>>>> would not include it in a configuration object.
>>>>>>>>>
>>>>>>>>>>        withChain(...);
>>>>>>>>>
>>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>>>>>>>>> seem to be a configuration. We can also not prevent a user to call
>>>>>>>>> `withName()` in combination of `withChain()` what does not make sense
>>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time
>>>>>>>>> check
>>>>>>>>> seems less appealing. Also, it could happen that neither
>>>>>>>>> `withChain()`
>>>>>>>>> not `withName()` is called and the branch is missing in the returned
>>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>>
>>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config
>>>>>>>>> object
>>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>>> others,
>>>>>>>>> but this seems not to be the case here. If we add new configuration
>>>>>>>>> later, we can also just move forward by deprecating the methods that
>>>>>>>>> accept `Named` and add new methods that accepted `BranchConfig` (that
>>>>>>>>> would of course implement `Named`).
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thoughts?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> @Ivan, what do you think about the general idea to blend the two main
>>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>>>>>>>>> everyone. Returning the map from the terminal operations also solves
>>>>>>>>>> the problem of merging/joining the branched streams, if we want
>>>>>>>>>> to add
>>>>>>>>>> support for the compliment later on.
>>>>>>>>>>
>>>>>>>>>> Under your suggestion, it seems that the name is required.
>>>>>>>>>> Otherwise,
>>>>>>>>>> we wouldn't have keys for the map to return. I this this is actually
>>>>>>>>>> not too bad, since experience has taught us that, although names for
>>>>>>>>>> operations are not required to define stream processing logic, it
>>>>>>>>>> does
>>>>>>>>>> significantly improve the operational experience when you can map
>>>>>>>>>> the
>>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>>> wouldn't (have to) reference the name to chain extra processing onto
>>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>>
>>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>>> operator itself, and then all the branches get index-suffixed names
>>>>>>>>>> built from the branch operator name. I guess under this proposal, we
>>>>>>>>>> could naturally append the branch name to the branching operator
>>>>>>>>>> name,
>>>>>>>>>> like this:
>>>>>>>>>>
>>>>>>>>>>       stream.split(Named.withName("mysplit")) //creates node
>>>>>>>>>> "mysplit"
>>>>>>>>>>                  .branch(..., ..., "abranch") // creates node
>>>>>>>>>> "mysplit-abranch"
>>>>>>>>>>                  .defaultBranch(...) // creates node
>>>>>>>>>> "mysplit-default"
>>>>>>>>>>
>>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>>
>>>>>>>>>> We don't have a defined grammar, so there's plenty of room to debate
>>>>>>>>>> the "best" syntax in the context of each operation, but in general,
>>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>>
>>>>>>>>>>        operator(function, config_object?) OR operator(config_object)
>>>>>>>>>>
>>>>>>>>>> where config_object is often just Named in the "function" variant.
>>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>>
>>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>>
>>>>>>>>>>      operator(function, function, string)
>>>>>>>>>>
>>>>>>>>>> where the string is the name.
>>>>>>>>>> My first question is whether the name should instead be specified
>>>>>>>>>> with
>>>>>>>>>> the NamedOperation interface.
>>>>>>>>>>
>>>>>>>>>> My second question is whether we should just roll all these
>>>>>>>>>> arguments
>>>>>>>>>> up into a config object like:
>>>>>>>>>>
>>>>>>>>>>       KBranchedStream#branch(BranchConfig)
>>>>>>>>>>
>>>>>>>>>>       interface BranchConfig extends NamedOperation {
>>>>>>>>>>        withPredicate(...);
>>>>>>>>>>        withChain(...);
>>>>>>>>>>        withName(...);
>>>>>>>>>>      }
>>>>>>>>>>
>>>>>>>>>> Although I guess we'd like to call BranchConfig something more like
>>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>>
>>>>>>>>>> This makes the source code a little noisier, but it also makes us
>>>>>>>>>> more
>>>>>>>>>> future-proof, as we can deal with a wide range of alternatives
>>>>>>>>>> purely
>>>>>>>>>> in the config interface, and never have to deal with adding
>>>>>>>>>> overloads
>>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>>
>>>>>>>>>> WDYT?
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> -John
>>>>>>>>>>
>>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>>
>>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of view.
>>>>>>>>>>> Good
>>>>>>>>>>> suggestion.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>>
>>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both
>>>>>>>>>>>> approaches:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>>
>>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>>      -> KBranchedStream
>>>>>>>>>>>>
>>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>>> //
>>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>,
>>>>>>>>>>>> String)
>>>>>>>>>>>>      -> KBranchedStream
>>>>>>>>>>>>
>>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>
>>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>
>>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>
>>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>>      -> Map<String,KStream>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name and
>>>>>>>>>>>> return
>>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>>> implementation can
>>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>>> returned
>>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>>
>>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>>
>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>>> necessity.
>>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>>
>>>>>>>>>>>>> Separately, I'm interested to see where the present discussion
>>>>>>>>>>>>> leads.
>>>>>>>>>>>>> I've written enough Javascript code in my life to be
>>>>>>>>>>>>> suspicious of
>>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>>> references (or
>>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>>> that this
>>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>>> logic when
>>>>>>>>>>>>> their nested closure situation got out of hand. Unfortunately,
>>>>>>>>>>>>> it's
>>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>>>>>>>>> readability (but in different ways for different reasons). In
>>>>>>>>>>>>> other
>>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>>> solution if
>>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>>
>>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>>> references aren't
>>>>>>>>>>>>> the end of the story either, and you can see that by observing
>>>>>>>>>>>>> that
>>>>>>>>>>>>> there have been two follow-on eras, as they continue trying to
>>>>>>>>>>>>> cope with
>>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>>> First, you
>>>>>>>>>>>>> have Futures/Promises, which essentially let you convert nested
>>>>>>>>>>>>> code to
>>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>>> this).
>>>>>>>>>>>>> Most lately, you have async/await, which is an effort to apply
>>>>>>>>>>>>> language
>>>>>>>>>>>>> (not just API) syntax to the problem, and offer the "flattest"
>>>>>>>>>>>>> possible
>>>>>>>>>>>>> programming style to solve the problem (because you get back to
>>>>>>>>>>>>> just one
>>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>>
>>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>>> nowhere
>>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to take
>>>>>>>>>>>>> the JS
>>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>>> valuable
>>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>>> bringing this
>>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time, just
>>>>>>>>>>>>> like JS,
>>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge (and
>>>>>>>>>>>>> I'd also
>>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly punt on
>>>>>>>>>>>>> it, by
>>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>>> there a DSL
>>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>>> -John
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        Ivan, I’ll definitely forfeit my point on the clumsiness of
>>>>>>>>>>>>> the
>>>>>>>>>>>>>        branch(predicate, consumer) solution, I don’t see any real
>>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>>        for the dynamic case.
>>>>>>>>>>>>>
>>>>>>>>>>>>>        IMO the one trade off to consider at this point is the
>>>>>>>>>>>>> scope
>>>>>>>>>>>>>        question. I don’t know if I totally agree that “we rarely
>>>>>>>>>>>>> need them
>>>>>>>>>>>>>        in the same scope” since merging the branches back together
>>>>>>>>>>>>> later
>>>>>>>>>>>>>        seems like a perfectly plausible use case that can be a lot
>>>>>>>>>>>>> nicer
>>>>>>>>>>>>>        when the branched streams are in the same scope. That being
>>>>>>>>>>>>> said,
>>>>>>>>>>>>>        for the reasons Ivan listed, I think it is overall the
>>>>>>>>>>>>> better
>>>>>>>>>>>>>        solution - working around the scope thing is easy enough if
>>>>>>>>>>>>> you need
>>>>>>>>>>>>>        to.
>>>>>>>>>>>>>
>>>>>>>>>>>>>        > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>>        <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > Hello everyone, thank you all for joining the discussion!
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > Well, I don't think the idea of named branches, be it a
>>>>>>>>>>>>>        LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>>> definition
>>>>>>>>>>>>>        matters) or `branch` method  taking name and Consumer
>>>>>>>>>>>>> has more
>>>>>>>>>>>>>        advantages than drawbacks.
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > In my opinion, the only real positive outcome from
>>>>>>>>>>>>> Michael's
>>>>>>>>>>>>>        proposal is that all the returned branches are in the same
>>>>>>>>>>>>> scope.
>>>>>>>>>>>>>        But 1) we rarely need them in the same scope 2) there is a
>>>>>>>>>>>>>        workaround for the scope problem, described in the KIP.
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > 'Inlining the complex logic' is not a problem, because we
>>>>>>>>>>>>> can use
>>>>>>>>>>>>>        method references instead of lambdas. In real world
>>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>>        tend to split the complex logic to methods anyway, so the
>>>>>>>>>>>>> code is
>>>>>>>>>>>>>        going to be clean.
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > The drawbacks are strong. The cohesion between predicates
>>>>>>>>>>>>> and
>>>>>>>>>>>>>        handlers is lost. We have to define predicates in one
>>>>>>>>>>>>> place, and
>>>>>>>>>>>>>        handlers in another. This opens the door for bugs:
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > - what if we forget to define a handler for a name? or a
>>>>>>>>>>>>> name for
>>>>>>>>>>>>>        a handler?
>>>>>>>>>>>>>        > - what if we misspell a name?
>>>>>>>>>>>>>        > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > What Michael propose would have been totally OK if we had
>>>>>>>>>>>>> been
>>>>>>>>>>>>>        writing the API in Lua, Ruby or Python. In those
>>>>>>>>>>>>> languages the
>>>>>>>>>>>>>        "dynamic naming" approach would have looked most concise
>>>>>>>>>>>>> and
>>>>>>>>>>>>>        beautiful. But in Java we expect all the problems
>>>>>>>>>>>>> related to
>>>>>>>>>>>>>        identifiers to be eliminated in compile time.
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > And if we do, what advantage are we supposed to get
>>>>>>>>>>>>> besides having
>>>>>>>>>>>>>        all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>>> missing your
>>>>>>>>>>>>>        point?
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > ---
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > Earlier in this discussion John Roesler also proposed
>>>>>>>>>>>>> to do
>>>>>>>>>>>>>        without "start branching" operator, and later Paul
>>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>>        the case when we have to add a dynamic number of
>>>>>>>>>>>>> branches, the
>>>>>>>>>>>>>        current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>>        me address both comments here.
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > 1) "Start branching" operator (I think that *split* is a
>>>>>>>>>>>>> good name
>>>>>>>>>>>>>        for it indeed) is critical when we need to do a dynamic
>>>>>>>>>>>>> branching,
>>>>>>>>>>>>>        see example below.
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > 2) No, dynamic branching in current KIP is not clumsy at
>>>>>>>>>>>>> all.
>>>>>>>>>>>>>        Imagine a real-world scenario when you need one branch per
>>>>>>>>>>>>> enum
>>>>>>>>>>>>>        value (say, RecordType). You can have something like this:
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > /*John:if we had to start with stream.branch(...) here,
>>>>>>>>>>>>> it would
>>>>>>>>>>>>>        have been much messier.*/
>>>>>>>>>>>>>        > KBranchedStream branched = stream.split();
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > /*Not clumsy at all :-)*/
>>>>>>>>>>>>>        > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>>        >             branched = branched.branch((k, v) ->
>>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>>        recordType,
>>>>>>>>>>>>>        >                     recordType::processRecords);
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > Regards,
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > Ivan
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>        > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>>        >> I also agree with Michael's observation about the core
>>>>>>>>>>>>> problem of
>>>>>>>>>>>>>        >> current `branch()` implementation.
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>>> object. My
>>>>>>>>>>>>>        thinking
>>>>>>>>>>>>>        >> was more aligned with Paul's proposal to just add a name
>>>>>>>>>>>>> to each
>>>>>>>>>>>>>        >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >> It makes the code easier to read, and also make the
>>>>>>>>>>>>> order of
>>>>>>>>>>>>>        >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>>>>>        >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>        >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>        >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>>        >> An open question is the case for which no
>>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>>> be
>>>>>>>>>>>>>        >> specified. Atm, `split()` and `branch()` would return
>>>>>>>>>>>>>        `BranchedKStream`
>>>>>>>>>>>>>        >> and the call to `defaultBranch()` that returns the
>>>>>>>>>>>>> `Map` is
>>>>>>>>>>>> mandatory
>>>>>>>>>>>>>        >> (what is not the case atm). Or is this actually not a
>>>>>>>>>>>>> real
>>>>>>>>>>>> problem,
>>>>>>>>>>>>>        >> because users can just ignore the branch returned by
>>>>>>>>>>>>>        `defaultBranch()`
>>>>>>>>>>>>>        >> in the result `Map` ?
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >> About "inlining": So far, it seems to be a matter of
>>>>>>>>>>>>> personal
>>>>>>>>>>>>>        >> preference. I can see arguments for both, but no "killer
>>>>>>>>>>>>>        argument" yet
>>>>>>>>>>>>>        >> that clearly make the case for one or the other.
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >> -Matthias
>>>>>>>>>>>>>        >>
>>>>>>>>>>>>>        >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>>        >>> Perhaps inlining is the wrong terminology. It doesn’t
>>>>>>>>>>>>> require
>>>>>>>>>>>>>        that a lambda with the full downstream topology be defined
>>>>>>>>>>>>> inline -
>>>>>>>>>>>>>        it can be a method reference as with Ivan’s original
>>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>>        The advantage of putting the predicate and its downstream
>>>>>>>>>>>>> logic
>>>>>>>>>>>>>        (Consumer) together in branch() is that they are required
>>>>>>>>>>>>> to be near
>>>>>>>>>>>>>        to each other.
>>>>>>>>>>>>>        >>>
>>>>>>>>>>>>>        >>> Ultimately the downstream code has to live somewhere,
>>>>>>>>>>>>> and deep
>>>>>>>>>>>>>        branch trees will be hard to read regardless.
>>>>>>>>>>>>>        >>>
>>>>>>>>>>>>>        >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>>        <michael.drogalis@confluent.io
>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>        >>>>
>>>>>>>>>>>>>        >>>> I'm less enthusiastic about inlining the branch logic
>>>>>>>>>>>>> with its
>>>>>>>>>>>>>        downstream
>>>>>>>>>>>>>        >>>> functionality. Programs that have deep branch trees
>>>>>>>>>>>>> will
>>>>>>>>>>>>>        quickly become
>>>>>>>>>>>>>        >>>> harder to read as a single unit.
>>>>>>>>>>>>>        >>>>
>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>>        <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Also +1 on the issues/goals as Michael outlined them,
>>>>>>>>>>>>> I think
>>>>>>>>>>>>>        that sets a
>>>>>>>>>>>>>        >>>>> great framework for the discussion.
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Regarding the SortedMap solution, my understanding is
>>>>>>>>>>>>> that the
>>>>>>>>>>>>>        current
>>>>>>>>>>>>>        >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>>        decisions) is
>>>>>>>>>>>>>        >>>>> roughly this:
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> stream.split()
>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>>>>>        >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>>>>>        >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Obviously some ordering is necessary, since branching
>>>>>>>>>>>>> as a
>>>>>>>>>>>>>        construct
>>>>>>>>>>>>>        >>>>> doesn't work without it, but this solution seems
>>>>>>>>>>>>> like it
>>>>>>>>>>>>>        provides as much
>>>>>>>>>>>>>        >>>>> associativity as the SortedMap solution, because each
>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>        call
>>>>>>>>>>>>>        >>>>> directly associates the "conditional" with the "code
>>>>>>>>>>>>> block."
>>>>>>>>>>>>>        The value it
>>>>>>>>>>>>>        >>>>> provides over the KIP solution is the accessing of
>>>>>>>>>>>>> streams in
>>>>>>>>>>>>>        the same
>>>>>>>>>>>>>        >>>>> scope.
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> The KIP solution is less "dynamic" than the SortedMap
>>>>>>>>>>>>> solution
>>>>>>>>>>>>>        in the sense
>>>>>>>>>>>>>        >>>>> that it is slightly clumsier to add a dynamic
>>>>>>>>>>>>> number of
>>>>>>>>>>>>>        branches, but it is
>>>>>>>>>>>>>        >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>>> should favor
>>>>>>>>>>>>>        the "static"
>>>>>>>>>>>>>        >>>>> case anyway, and should make it simple and
>>>>>>>>>>>>> readable to
>>>>>>>>>>>>>        fluently declare and
>>>>>>>>>>>>>        >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>>> impossible to
>>>>>>>>>>>>>        ignore a
>>>>>>>>>>>>>        >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>>> identical
>>>>>>>>>>>>>        SortedMap
>>>>>>>>>>>>>        >>>>> solution on top of it.
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> I could also see a middle ground where instead of
>>>>>>>>>>>>> a raw
>>>>>>>>>>>>>        SortedMap being
>>>>>>>>>>>>>        >>>>> taken in, branch() takes a name and not a Consumer.
>>>>>>>>>>>>> Something
>>>>>>>>>>>>>        like this:
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>>>>>        >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>>        >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>>        >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Pros for that solution:
>>>>>>>>>>>>>        >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>>        >>>>> - no double brace initialization, hopefully slightly
>>>>>>>>>>>>> more
>>>>>>>>>>>>>        readable than
>>>>>>>>>>>>>        >>>>> SortedMap
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Cons
>>>>>>>>>>>>>        >>>>> - downstream branch logic cannot be specified inline
>>>>>>>>>>>>> which
>>>>>>>>>>>>>        makes it harder
>>>>>>>>>>>>>        >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>>        unlike the KIP)
>>>>>>>>>>>>>        >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>>> streams (like
>>>>>>>>>>>>>        existing
>>>>>>>>>>>>>        >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> (KBranchedStreams could even work *both* ways but
>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>        that's overdoing
>>>>>>>>>>>>>        >>>>> it).
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Overall I'm curious how important it is to be able to
>>>>>>>>>>>>> easily
>>>>>>>>>>>>>        access the
>>>>>>>>>>>>>        >>>>> branched KStream in the same scope as the original.
>>>>>>>>>>>>> It's
>>>>>>>>>>>>>        possible that it
>>>>>>>>>>>>>        >>>>> doesn't need to be handled directly by the API, but
>>>>>>>>>>>>> instead
>>>>>>>>>>>>>        left up to the
>>>>>>>>>>>>>        >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> Paul
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>>>>>>>>        <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>        >>>>> wrote:
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>        >>>>>> I'd like to +1 what Michael said about the issues
>>>>>>>>>>>>> with the
>>>>>>>>>>>>>        existing
>>>>>>>>>>>>>        >>>>> branch
>>>>>>>>>>>>>        >>>>>> method, I agree with what he's outlined and I think
>>>>>>>>>>>>> we should
>>>>>>>>>>>>>        proceed by
>>>>>>>>>>>>>        >>>>>> trying to alleviate these problems. Specifically it
>>>>>>>>>>>>> seems
>>>>>>>>>>>>>        important to be
>>>>>>>>>>>>>        >>>>>> able to cleanly access the individual branches (eg
>>>>>>>>>>>>> by mapping
>>>>>>>>>>>>>        >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>>> intention of
>>>>>>>>>>>>>        this KIP.
>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>        >>>>>> That said, I don't think we should so easily give in
>>>>>>>>>>>>> to the
>>>>>>>>>>>>>        double brace
>>>>>>>>>>>>>        >>>>>> anti-pattern or force ours users into it if at all
>>>>>>>>>>>>> possible to
>>>>>>>>>>>>>        >>>>> avoid...just
>>>>>>>>>>>>>        >>>>>> my two cents.
>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>        >>>>>> Cheers,
>>>>>>>>>>>>>        >>>>>> Sophie
>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>        >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>>>>>>>>        >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>>        <ma...@confluent.io>> wrote:
>>>>>>>>>>>>>        >>>>>>
>>>>>>>>>>>>>        >>>>>>> I’d like to propose a different way of thinking
>>>>>>>>>>>>> about this.
>>>>>>>>>>>>>        To me,
>>>>>>>>>>>>>        >>>>> there
>>>>>>>>>>>>>        >>>>>>> are three problems with the existing branch
>>>>>>>>>>>>> signature:
>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>        >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>>> raises unsafe
>>>>>>>>>>>> type
>>>>>>>>>>>>>        >>>>>> warnings.
>>>>>>>>>>>>>        >>>>>>> 2. The way in which you use the stream branches is
>>>>>>>>>>>>>        positionally coupled
>>>>>>>>>>>>>        >>>>>> to
>>>>>>>>>>>>>        >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>>        >>>>>>> 3. It is brittle to extend existing branch calls
>>>>>>>>>>>>> with
>>>>>>>>>>>>>        additional code
>>>>>>>>>>>>>        >>>>>>> paths.
>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>        >>>>>>> Using associative constructs instead of relying on
>>>>>>>>>>>>> ordered
>>>>>>>>>>>>>        constructs
>>>>>>>>>>>>>        >>>>>> would
>>>>>>>>>>>>>        >>>>>>> be a stronger approach. Consider a signature that
>>>>>>>>>>>>> instead
>>>>>>>>>>>>>        looks like
>>>>>>>>>>>>>        >>>>>> this:
>>>>>>>>>>>>>        >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>>        Predicate<?
>>>>>>>>>>>>>        >>>>>>> super K,? super V>>);
>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>        >>>>>>> Branches are given names in a map, and as a result,
>>>>>>>>>>>>> the API
>>>>>>>>>>>>>        returns a
>>>>>>>>>>>>>        >>>>>>> mapping of names to streams. The ordering of the
>>>>>>>>>>>> conditionals is
>>>>>>>>>>>>>        >>>>>> maintained
>>>>>>>>>>>>>        >>>>>>> because it’s a sorted map. Insert order determines
>>>>>>>>>>>>> the order
>>>>>>>>>>>> of
>>>>>>>>>>>>>        >>>>>> evaluation.
>>>>>>>>>>>>>        >>>>>>> This solves problem 1 because there are no more
>>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>>        solves
>>>>>>>>>>>>>        >>>>>> problem
>>>>>>>>>>>>>        >>>>>>> 2 because you no longer lean on ordering to
>>>>>>>>>>>>> access the
>>>>>>>>>>>>>        branch you’re
>>>>>>>>>>>>>        >>>>>>> interested in. It solves problem 3 because you can
>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>        another
>>>>>>>>>>>>>        >>>>>>> conditional by simply attaching another name to the
>>>>>>>>>>>>>        structure, rather
>>>>>>>>>>>>>        >>>>>> than
>>>>>>>>>>>>>        >>>>>>> messing with the existing indices.
>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>        >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>>> inline is
>>>>>>>>>>>>>        historically
>>>>>>>>>>>>>        >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>>>>>>>>        voluminously, but
>>>>>>>>>>>>>        >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>>        >>>>>>>
>>>>>>>>>>>>>        >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>>        <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>        >>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>> Hi Ivan,
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>> Thanks for the update.
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>> FWIW, I agree with Matthias that the current
>>>>>>>>>>>>> "start
>>>>>>>>>>>> branching"
>>>>>>>>>>>>>        >>>>> operator
>>>>>>>>>>>>>        >>>>>>> is
>>>>>>>>>>>>>        >>>>>>>> confusing when named the same way as the actual
>>>>>>>>>>>>> branches.
>>>>>>>>>>>>>        "Split"
>>>>>>>>>>>>>        >>>>> seems
>>>>>>>>>>>>>        >>>>>>>> like a good name. Alternatively, we can do without
>>>>>>>>>>>>> a "start
>>>>>>>>>>>>>        >>>>> branching"
>>>>>>>>>>>>>        >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>> stream
>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>        >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>        >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>> Tentatively, I think that this branching operation
>>>>>>>>>>>>> should be
>>>>>>>>>>>>>        >>>>> terminal.
>>>>>>>>>>>>>        >>>>>>> That
>>>>>>>>>>>>>        >>>>>>>> way, we don't create ambiguity about how to use
>>>>>>>>>>>>> it. That
>>>>>>>>>>>>>        is, `branch`
>>>>>>>>>>>>>        >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>>        `void`, to
>>>>>>>>>>>>>        >>>>>>>> enforce that it comes last, and that there is only
>>>>>>>>>>>>> one
>>>>>>>>>>>>>        definition of
>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>        >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>>> warning if
>>>>>>>>>>>>>        there's no
>>>>>>>>>>>>>        >>>>>>> default,
>>>>>>>>>>>>>        >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>>        record
>>>>>>>>>>>>>        >>>>>> falls
>>>>>>>>>>>>>        >>>>>>>> though with no default.
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>> Thoughts?
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>> Thanks,
>>>>>>>>>>>>>        >>>>>>>> -John
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>>>>>>>>        >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>>>>>>>>        >>>>>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> this is to make the name similar to String#split
>>>>>>>>>>>>>        >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>>        >>>>>>>>> The intend was to avoid name duplication. The
>>>>>>>>>>>>> return type
>>>>>>>>>>>>>        should
>>>>>>>>>>>>>        >>>>>> _not_
>>>>>>>>>>>>>        >>>>>>>>> be an array.
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> The current proposal is
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> stream.branch()
>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>>>>>>>>        `branch()` does
>>>>>>>>>>>>>        >>>>> not
>>>>>>>>>>>>>        >>>>>>>>> take any parameters and has different semantics
>>>>>>>>>>>>> than the
>>>>>>>>>>>> later
>>>>>>>>>>>>>        >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>>> snippet above,
>>>>>>>>>>>> it's
>>>>>>>>>>>>>        >>>>> hidden
>>>>>>>>>>>>>        >>>>>>>>> that the first call is `KStream#branch()` while
>>>>>>>>>>>>> the others
>>>>>>>>>>>> are
>>>>>>>>>>>>>        >>>>>>>>> `KBranchedStream#branch()` what makes reading the
>>>>>>>>>>>>> code
>>>>>>>>>>>> harder.
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> Because I suggested to rename `addBranch()` ->
>>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>>        I though
>>>>>>>>>>>>>        >>>>>> it
>>>>>>>>>>>>>        >>>>>>>>> might be better to also rename `KStream#branch()`
>>>>>>>>>>>>> to avoid
>>>>>>>>>>>> the
>>>>>>>>>>>>>        >>>>> naming
>>>>>>>>>>>>>        >>>>>>>>> overlap that seems to be confusing. The following
>>>>>>>>>>>>> reads
>>>>>>>>>>>> much
>>>>>>>>>>>>>        >>>>> cleaner
>>>>>>>>>>>>>        >>>>>> to
>>>>>>>>>>>>>        >>>>>>>> me:
>>>>>>>>>>>>>        >>>>>>>>> stream.split()
>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>        >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>>        >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> Maybe there is a better alternative to `split()`
>>>>>>>>>>>>> though to
>>>>>>>>>>>>>        avoid
>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>        >>>>>>>>> naming overlap.
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>>> unfortunately
>>>>>>>>>>>> we
>>>>>>>>>>>>>        >>>>> cannot
>>>>>>>>>>>>>        >>>>>>> have
>>>>>>>>>>>>>        >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> Bummer. Didn't consider this. Maybe we can still
>>>>>>>>>>>>> come up
>>>>>>>>>>>>>        with a
>>>>>>>>>>>>>        >>>>> short
>>>>>>>>>>>>>        >>>>>>>> name?
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> Can you add the interface `KBranchedStream` to
>>>>>>>>>>>>> the KIP
>>>>>>>>>>>>>        with all
>>>>>>>>>>>>>        >>>>> it's
>>>>>>>>>>>>>        >>>>>>>>> methods? It will be part of public API and
>>>>>>>>>>>>> should be
>>>>>>>>>>>>>        contained in
>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>        >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>>> return type of
>>>>>>>>>>>>>        >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>>        `KBranchedStream#get(int
>>>>>>>>>>>>>        >>>>>>> index)
>>>>>>>>>>>>>        >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>>        branched-KStreams. Would
>>>>>>>>>>>>>        >>>>>> be
>>>>>>>>>>>>>        >>>>>>>>> nice to get your feedback about it. It seems you
>>>>>>>>>>>>> suggest
>>>>>>>>>>>>>        that users
>>>>>>>>>>>>>        >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>>        access them.
>>>>>>>>>>>>>        >>>>> We
>>>>>>>>>>>>>        >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>>> approaches. It
>>>>>>>>>>>> feels
>>>>>>>>>>>>>        >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>>> built-in support
>>>>>>>>>>>>>        to get
>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>        >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>> -Matthias
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>>        >>>>>>>>>> Hi all!
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> I have updated the KIP-418 according to the new
>>>>>>>>>>>>> vision.
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>>        >>>>>>>>>> I can see your point: this is to make the name
>>>>>>>>>>>>> similar to
>>>>>>>>>>>>>        >>>>>>> String#split
>>>>>>>>>>>>>        >>>>>>>>>> that also returns an array, right? But is it
>>>>>>>>>>>>> worth the
>>>>>>>>>>>>>        loss of
>>>>>>>>>>>>>        >>>>>>>> backwards
>>>>>>>>>>>>>        >>>>>>>>>> compatibility? We can have overloaded branch()
>>>>>>>>>>>>> as well
>>>>>>>>>>>>>        without
>>>>>>>>>>>>>        >>>>>>>> affecting
>>>>>>>>>>>>>        >>>>>>>>>> the existing code. Maybe the old array-based
>>>>>>>>>>>>> `branch`
>>>>>>>>>>>> method
>>>>>>>>>>>>>        >>>>> should
>>>>>>>>>>>>>        >>>>>>> be
>>>>>>>>>>>>>        >>>>>>>>>> deprecated, but this is a subject for
>>>>>>>>>>>>> discussion.
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>>        >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>>        >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> Totally agree with 'addBranch->branch' rename.
>>>>>>>>>>>>> 'default'
>>>>>>>>>>>> is,
>>>>>>>>>>>>>        >>>>>>> however, a
>>>>>>>>>>>>>        >>>>>>>>>> reserved word, so unfortunately we cannot have a
>>>>>>>>>>>>> method
>>>>>>>>>>>>>        with such
>>>>>>>>>>>>>        >>>>>>> name
>>>>>>>>>>>>>        >>>>>>>>> :-)
>>>>>>>>>>>>>        >>>>>>>>>>> defaultBranch() does take an `Predicate` as
>>>>>>>>>>>>> argument,
>>>>>>>>>>>> but I
>>>>>>>>>>>>>        >>>>> think
>>>>>>>>>>>>>        >>>>>>> that
>>>>>>>>>>>>>        >>>>>>>>>> is not required?
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> Absolutely! I think that was just copy-paste
>>>>>>>>>>>>> error or
>>>>>>>>>>>>>        something.
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> please revise the new version of the KIP and
>>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>>        >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> Ivan
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>>        >>>>>>>>>>> Thanks for driving the discussion of this KIP.
>>>>>>>>>>>>> It seems
>>>>>>>>>>>> that
>>>>>>>>>>>>>        >>>>>>> everybody
>>>>>>>>>>>>>        >>>>>>>>>>> agrees that the current branch() method using
>>>>>>>>>>>>> arrays is
>>>>>>>>>>>> not
>>>>>>>>>>>>>        >>>>>> optimal.
>>>>>>>>>>>>>        >>>>>>>>>>> I had a quick look into the PR and I like the
>>>>>>>>>>>>> overall
>>>>>>>>>>>>>        proposal.
>>>>>>>>>>>>>        >>>>>>> There
>>>>>>>>>>>>>        >>>>>>>>>>> are some minor things we need to consider. I
>>>>>>>>>>>>> would
>>>>>>>>>>>>>        recommend the
>>>>>>>>>>>>>        >>>>>>>>>>> following renaming:
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>>        >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>>        BranchingKStream#default()
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> It's just a suggestion to get slightly shorter
>>>>>>>>>>>>> method
>>>>>>>>>>>> names.
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>>>>>>>>        `Predicate` as
>>>>>>>>>>>>>        >>>>>>>> argument,
>>>>>>>>>>>>>        >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>>> recently
>>>>>>>>>>>>>        accepted and
>>>>>>>>>>>>>        >>>>>> is
>>>>>>>>>>>>>        >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> Ie, we should add overloads that accepted a
>>>>>>>>>>>>> `Named`
>>>>>>>>>>>>>        parameter.
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> For the issue that the created `KStream` object
>>>>>>>>>>>>> are in
>>>>>>>>>>>>>        different
>>>>>>>>>>>>>        >>>>>>>> scopes:
>>>>>>>>>>>>>        >>>>>>>>>>> could we extend `KBranchedStream` with a
>>>>>>>>>>>>> `get(int
>>>>>>>>>>>>>        index)` method
>>>>>>>>>>>>>        >>>>>>> that
>>>>>>>>>>>>>        >>>>>>>>>>> returns the corresponding "branched" result
>>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>>        object?
>>>>>>>>>>>>>        >>>>>> Maybe,
>>>>>>>>>>>>>        >>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>> second argument of `addBranch()` should not
>>>>>>>>>>>>> be a
>>>>>>>>>>>>>        >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>>        >>>>>>>> but
>>>>>>>>>>>>>        >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could
>>>>>>>>>>>>> return
>>>>>>>>>>>>>        whatever
>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> Finally, I would also suggest to update the KIP
>>>>>>>>>>>>> with the
>>>>>>>>>>>>>        current
>>>>>>>>>>>>>        >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>> -Matthias
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>>        >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>>> think it
>>>>>>>>>>>>>        makes sense
>>>>>>>>>>>>>        >>>>>> for
>>>>>>>>>>>>>        >>>>>>>> you
>>>>>>>>>>>>>        >>>>>>>>> to
>>>>>>>>>>>>>        >>>>>>>>>>>> revise the KIP and continue the discussion.
>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>        we'll
>>>>>>>>>>>>>        >>>>> need
>>>>>>>>>>>>>        >>>>>>>> some
>>>>>>>>>>>>>        >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>>        >>>>> whether
>>>>>>>>>>>>>        >>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>> KIP
>>>>>>>>>>>>>        >>>>>>>>>>>> could be adopted.  It would be great to hear
>>>>>>>>>>>>> if they
>>>>>>>>>>>>>        think this
>>>>>>>>>>>>>        >>>>>> is
>>>>>>>>>>>>>        >>>>>>> a
>>>>>>>>>>>>>        >>>>>>>>> good
>>>>>>>>>>>>>        >>>>>>>>>>>> idea overall.  I'm not sure if that happens
>>>>>>>>>>>>> just by
>>>>>>>>>>>>>        starting a
>>>>>>>>>>>>>        >>>>>>> vote,
>>>>>>>>>>>>>        >>>>>>>>> or if
>>>>>>>>>>>>>        >>>>>>>>>>>> there is generally some indication of interest
>>>>>>>>>>>> beforehand.
>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>> That being said, I'll continue the discussion
>>>>>>>>>>>>> a bit:
>>>>>>>>>>>>>        assuming
>>>>>>>>>>>>>        >>>>> we
>>>>>>>>>>>>>        >>>>>> do
>>>>>>>>>>>>>        >>>>>>>>> move
>>>>>>>>>>>>>        >>>>>>>>>>>> forward the solution of "stream.branch()
>>>>>>>>>>>>> returns
>>>>>>>>>>>>>        >>>>>> KBranchedStream",
>>>>>>>>>>>>>        >>>>>>> do
>>>>>>>>>>>>>        >>>>>>>>> we
>>>>>>>>>>>>>        >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>>> would
>>>>>>>>>>>>>        >>>>> favor
>>>>>>>>>>>>>        >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>>> that
>>>>>>>>>>>>>        >>>>>>> accomplish
>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>>        similar
>>>>>>>>>>>>>        >>>>>>>>> anyway.  We
>>>>>>>>>>>>>        >>>>>>>>>>>> just need to be sure we're not making
>>>>>>>>>>>>> something
>>>>>>>>>>>>>        >>>>>>> impossible/difficult
>>>>>>>>>>>>>        >>>>>>>>> that
>>>>>>>>>>>>>        >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>>> structure would
>>>>>>>>>>>> work,
>>>>>>>>>>>>>        >>>>> it's
>>>>>>>>>>>>>        >>>>>>>> just a
>>>>>>>>>>>>>        >>>>>>>>>>>> little sloppy overall in terms of naming and
>>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>>        >>>>>>> particular,
>>>>>>>>>>>>>        >>>>>>>>>>>> passing in the "predicates" and "children"
>>>>>>>>>>>>> lists which
>>>>>>>>>>>> get
>>>>>>>>>>>>>        >>>>>> modified
>>>>>>>>>>>>>        >>>>>>>> in
>>>>>>>>>>>>>        >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>>        KStreamLazyBranch is
>>>>>>>>>>>>>        >>>>> a
>>>>>>>>>>>>>        >>>>>>> bit
>>>>>>>>>>>>>        >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>        >>>>>>>>>>>> Paul
>>>>>>>>>>>>>        >>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan
>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>        >>>>>>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>>>>>>>>        convinced: your
>>>>>>>>>>>>>        >>>>>>>> proposal
>>>>>>>>>>>>>        >>>>>>>>>>>>> looks better and should work. We just have to
>>>>>>>>>>>>> document
>>>>>>>>>>>> the
>>>>>>>>>>>>>        >>>>>> crucial
>>>>>>>>>>>>>        >>>>>>>>> fact
>>>>>>>>>>>>>        >>>>>>>>>>>>> that KStream consumers are invoked as they're
>>>>>>>>>>>>> added.
>>>>>>>>>>>>>        And then
>>>>>>>>>>>>>        >>>>>> it's
>>>>>>>>>>>>>        >>>>>>>> all
>>>>>>>>>>>>>        >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>> What shall we do now? I should re-write the
>>>>>>>>>>>>> KIP and
>>>>>>>>>>>>>        resume the
>>>>>>>>>>>>>        >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>> Why are you telling that your PR 'should not
>>>>>>>>>>>>> be even a
>>>>>>>>>>>>>        >>>>> starting
>>>>>>>>>>>>>        >>>>>>>> point
>>>>>>>>>>>>>        >>>>>>>>> if
>>>>>>>>>>>>>        >>>>>>>>>>>>> we go in this direction'? To me it looks like
>>>>>>>>>>>>> a good
>>>>>>>>>>>>>        starting
>>>>>>>>>>>>>        >>>>>>> point.
>>>>>>>>>>>>>        >>>>>>>>> But
>>>>>>>>>>>>>        >>>>>>>>>>>>> as a novice in this project I might miss some
>>>>>>>>>>>>> important
>>>>>>>>>>>>>        >>>>> details.
>>>>>>>>>>>>>        >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>        >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>> Maybe I’m missing the point, but I
>>>>>>>>>>>>> believe the
>>>>>>>>>>>>>        >>>>> stream.branch()
>>>>>>>>>>>>>        >>>>>>>>> solution
>>>>>>>>>>>>>        >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>>        >>>>> invoked
>>>>>>>>>>>>>        >>>>>> as
>>>>>>>>>>>>>        >>>>>>>>> they’re
>>>>>>>>>>>>>        >>>>>>>>>>>>> added, not during streamsBuilder.build(). So
>>>>>>>>>>>>> the user
>>>>>>>>>>>>>        still
>>>>>>>>>>>>>        >>>>>> ought
>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>        >>>>>>>>> be
>>>>>>>>>>>>>        >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward
>>>>>>>>>>>>> and
>>>>>>>>>>>>>        depend on
>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>        >>>>>>>>> branched
>>>>>>>>>>>>>        >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>        >>>>>>>>>>>>>> The issue I mean to point out is that it is
>>>>>>>>>>>>> hard to
>>>>>>>>>>>>>        access
>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>        >>>>>>>>> branched
>>>>>>>>>>>>>        >>>>>>>>>>>>> streams in the same scope as the original
>>>>>>>>>>>>> stream (that
>>>>>>>>>>>>>        is, not
>>>>>>>>>>>>>        >>>>>>>> inside
>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>> couponIssuer), which is a problem with both
>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>        >>>>> solutions.
>>>>>>>>>>>>>        >>>>>> It
>>>>>>>>>>>>>        >>>>>>>>> can be
>>>>>>>>>>>>>        >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>        >>>>>>>>>>>>>> [Also, great to hear additional interest in
>>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>>        excited
>>>>>>>>>>>>>        >>>>> to
>>>>>>>>>>>>>        >>>>>>>> hear
>>>>>>>>>>>>>        >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>        >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>        >>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan
>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>        >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>        >>>>>>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> The idea to postpone the wiring of branches
>>>>>>>>>>>>> to the
>>>>>>>>>>>>>        >>>>>>>>>>>>> streamsBuilder.build() also looked great for
>>>>>>>>>>>>> me at
>>>>>>>>>>>> first
>>>>>>>>>>>>>        >>>>> glance,
>>>>>>>>>>>>>        >>>>>>> but
>>>>>>>>>>>>>        >>>>>>>>> ---
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>>> available in the
>>>>>>>>>>>>>        same
>>>>>>>>>>>>>        >>>>>> scope
>>>>>>>>>>>>>        >>>>>>> as
>>>>>>>>>>>>>        >>>>>>>>> each
>>>>>>>>>>>>>        >>>>>>>>>>>>> other.  That is, if we wanted to merge
>>>>>>>>>>>>> them back
>>>>>>>>>>>> together
>>>>>>>>>>>>>        >>>>> again
>>>>>>>>>>>>>        >>>>>> I
>>>>>>>>>>>>>        >>>>>>>>> don't see
>>>>>>>>>>>>>        >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> You just took the words right out of my
>>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>>        just
>>>>>>>>>>>>>        >>>>>> going
>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>        >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Consider the example from Bill's book, p.
>>>>>>>>>>>>> 101: say
>>>>>>>>>>>>>        we need
>>>>>>>>>>>>>        >>>>> to
>>>>>>>>>>>>>        >>>>>>>>> identify
>>>>>>>>>>>>>        >>>>>>>>>>>>> customers who have bought coffee and made a
>>>>>>>>>>>>> purchase
>>>>>>>>>>>>>        in the
>>>>>>>>>>>>>        >>>>>>>>> electronics
>>>>>>>>>>>>>        >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> This is the code I usually write under
>>>>>>>>>>>>> these
>>>>>>>>>>>>>        circumstances
>>>>>>>>>>>>>        >>>>>> using
>>>>>>>>>>>>>        >>>>>>>> my
>>>>>>>>>>>>>        >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   private KStream<....>
>>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>       /*In the real world the code here
>>>>>>>>>>>>> can be
>>>>>>>>>>>>>        complex, so
>>>>>>>>>>>>>        >>>>>>>>> creation of
>>>>>>>>>>>>>        >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>>> justified, in
>>>>>>>>>>>>>        order to
>>>>>>>>>>>>>        >>>>>>>> separate
>>>>>>>>>>>>>        >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> }
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>>        >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to
>>>>>>>>>>>>> wire up
>>>>>>>>>>>>>        everything
>>>>>>>>>>>>>        >>>>>>>> later,
>>>>>>>>>>>>>        >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>>>>>>>>        initialize the
>>>>>>>>>>>>>        >>>>>>>>> CouponIssuer
>>>>>>>>>>>>>        >>>>>>>>>>>>> we need the terminal operation to be called
>>>>>>>>>>>>> before
>>>>>>>>>>>>>        >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>>        >>>>>>>>>>>>> is called.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>>        essentially
>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>        >>>>>>>>> next
>>>>>>>>>>>>>        >>>>>>>>>>>>> KIP I was going to write here. I have some
>>>>>>>>>>>>> thoughts
>>>>>>>>>>>>>        based on
>>>>>>>>>>>>>        >>>>> my
>>>>>>>>>>>>>        >>>>>>>>> experience,
>>>>>>>>>>>>>        >>>>>>>>>>>>> so I will join the discussion on KIP-401
>>>>>>>>>>>>> soon.]
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>>> concept of a
>>>>>>>>>>>>>        fluent
>>>>>>>>>>>>>        >>>>> API
>>>>>>>>>>>>>        >>>>>>>> based
>>>>>>>>>>>>>        >>>>>>>>>>>>> off of
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>>        (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>>        >>>>>> and
>>>>>>>>>>>>>        >>>>>>> I
>>>>>>>>>>>>>        >>>>>>>>> think
>>>>>>>>>>>>>        >>>>>>>>>>>>> I
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>>> earlier about
>>>>>>>>>>>>>        >>>>>>> compatibility
>>>>>>>>>>>>>        >>>>>>>>>>>>> issues,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was
>>>>>>>>>>>>> unaware
>>>>>>>>>>>>>        that Java
>>>>>>>>>>>>>        >>>>> is
>>>>>>>>>>>>>        >>>>>>>> smart
>>>>>>>>>>>>>        >>>>>>>>>>>>> enough to
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    distinguish between a
>>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>>        returning one
>>>>>>>>>>>>>        >>>>>>> thing
>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>        >>>>>>>>>>>>> branch()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    with no arguments returning another
>>>>>>>>>>>>> thing.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't
>>>>>>>>>>>>> actually
>>>>>>>>>>>>>        need
>>>>>>>>>>>>>        >>>>> it.
>>>>>>>>>>>>>        >>>>>>> We
>>>>>>>>>>>>>        >>>>>>>>> can
>>>>>>>>>>>>>        >>>>>>>>>>>>> just
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>>        shares
>>>>>>>>>>>>>        >>>>>> its
>>>>>>>>>>>>>        >>>>>>>>> state
>>>>>>>>>>>>>        >>>>>>>>>>>>> with the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do
>>>>>>>>>>>>> the
>>>>>>>>>>>>>        branching.
>>>>>>>>>>>>>        >>>>>>> It's
>>>>>>>>>>>>>        >>>>>>>>> not
>>>>>>>>>>>>>        >>>>>>>>>>>>> terribly
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>    pretty in its current form, but I
>>>>>>>>>>>>> think it
>>>>>>>>>>>>>        demonstrates
>>>>>>>>>>>>>        >>>>>> its
>>>>>>>>>>>>>        >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>>> request should
>>>>>>>>>>>> be
>>>>>>>>>>>>>        >>>>> final
>>>>>>>>>>>>>        >>>>>> or
>>>>>>>>>>>>>        >>>>>>>>> even a
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> starting point if we go in this direction,
>>>>>>>>>>>>> I just
>>>>>>>>>>>>>        wanted to
>>>>>>>>>>>>>        >>>>>> see
>>>>>>>>>>>>>        >>>>>>>> how
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> challenging it would be to get the API
>>>>>>>>>>>>> working.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> I will say though, that I'm not sure the
>>>>>>>>>>>>> existing
>>>>>>>>>>>>>        solution
>>>>>>>>>>>>>        >>>>>>> could
>>>>>>>>>>>>>        >>>>>>>> be
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecated in favor of this, which I had
>>>>>>>>>>>>> originally
>>>>>>>>>>>>>        >>>>> suggested
>>>>>>>>>>>>>        >>>>>>>> was a
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> possibility.  The reason is that the newly
>>>>>>>>>>>>> branched
>>>>>>>>>>>>>        streams
>>>>>>>>>>>>>        >>>>>> are
>>>>>>>>>>>>>        >>>>>>>> not
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>>> other.  That
>>>>>>>>>>>>>        is, if we
>>>>>>>>>>>>>        >>>>>>> wanted
>>>>>>>>>>>>>        >>>>>>>>> to
>>>>>>>>>>>>>        >>>>>>>>>>>>> merge
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> them back together again I don't see a way
>>>>>>>>>>>>> to do
>>>>>>>>>>>>>        that.  The
>>>>>>>>>>>>>        >>>>>> KIP
>>>>>>>>>>>>>        >>>>>>>>>>>>> proposal
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>>> means is that
>>>>>>>>>>>> for
>>>>>>>>>>>>>        >>>>>> either
>>>>>>>>>>>>>        >>>>>>>>>>>>> solution,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> deprecating the existing branch(...) is
>>>>>>>>>>>>> not on the
>>>>>>>>>>>>>        table.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan
>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>>>>>>>>        >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>>> discussed up to
>>>>>>>>>>>> this
>>>>>>>>>>>>>        >>>>>> point.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed
>>>>>>>>>>>>> that
>>>>>>>>>>>>>        branch API
>>>>>>>>>>>>>        >>>>>>> needs
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> improvement. Motivation is given in
>>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>>        returns
>>>>>>>>>>>>>        >>>>>> its
>>>>>>>>>>>>>        >>>>>>>>> argument
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2)
>>>>>>>>>>>>> The code
>>>>>>>>>>>> won't
>>>>>>>>>>>>>        >>>>> make
>>>>>>>>>>>>>        >>>>>>>> sense
>>>>>>>>>>>>>        >>>>>>>>>>>>> until
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> all the necessary ingredients are
>>>>>>>>>>>>> provided.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>        instance
>>>>>>>>>>>>>        >>>>>>>>> contrasts the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>>        >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>>        >>>>>>>>>>>>> and
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams
>>>>>>>>>>>>> interface
>>>>>>>>>>>> is
>>>>>>>>>>>>>        >>>>>> defined.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> CONS: We need to define two terminal
>>>>>>>>>>>>> methods
>>>>>>>>>>>>>        >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very
>>>>>>>>>>>>> easy to
>>>>>>>>>>>>>        miss the
>>>>>>>>>>>>>        >>>>>> fact
>>>>>>>>>>>>>        >>>>>>>>> that one
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> of the terminal methods should be called.
>>>>>>>>>>>>> If these
>>>>>>>>>>>>>        methods
>>>>>>>>>>>>>        >>>>>> are
>>>>>>>>>>>>>        >>>>>>>> not
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>>> runtime.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can
>>>>>>>>>>>>> we do
>>>>>>>>>>>> better?
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> I see your point when you are talking
>>>>>>>>>>>>> about
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot
>>>>>>>>>>>>> not be
>>>>>>>>>>>>>        implemented the
>>>>>>>>>>>>>        >>>>>>> easy
>>>>>>>>>>>>>        >>>>>>>>> way.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> user could specify a terminal
>>>>>>>>>>>>> method that
>>>>>>>>>>>> assumes
>>>>>>>>>>>>>        >>>>> nothing
>>>>>>>>>>>>>        >>>>>>>> will
>>>>>>>>>>>>>        >>>>>>>>>>>>> reach
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>>>>>> occurs.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be
>>>>>>>>>>>>> the only
>>>>>>>>>>>> option
>>>>>>>>>>>>>        >>>>>> besides
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios
>>>>>>>>>>>>> when we
>>>>>>>>>>>>>        want to
>>>>>>>>>>>>>        >>>>>> just
>>>>>>>>>>>>>        >>>>>>>>> silently
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>>>>>>>>        predicate. 2)
>>>>>>>>>>>>>        >>>>>>> Throwing
>>>>>>>>>>>>>        >>>>>>>>> an
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> exception in the middle of data flow
>>>>>>>>>>>>> processing
>>>>>>>>>>>>>        looks
>>>>>>>>>>>>>        >>>>>> like a
>>>>>>>>>>>>>        >>>>>>>> bad
>>>>>>>>>>>>>        >>>>>>>>>>>>> idea.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would
>>>>>>>>>>>>> prefer to
>>>>>>>>>>>>>        emit a
>>>>>>>>>>>>>        >>>>>>>> special
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is
>>>>>>>>>>>>> exactly
>>>>>>>>>>>> where
>>>>>>>>>>>>>        >>>>>>> `default`
>>>>>>>>>>>>>        >>>>>>>>> can
>>>>>>>>>>>>>        >>>>>>>>>>>>> be
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>        >>>>>>>>> track
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>        a clear
>>>>>>>>>>>>>        >>>>>>> error
>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the
>>>>>>>>>>>>> program is
>>>>>>>>>>>>>        >>>>> compiled
>>>>>>>>>>>>>        >>>>>>> and
>>>>>>>>>>>>>        >>>>>>>>> run?
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply
>>>>>>>>>>>>> won't
>>>>>>>>>>>>>        compile if
>>>>>>>>>>>>>        >>>>> used
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an
>>>>>>>>>>>>> API as a
>>>>>>>>>>>>>        method chain
>>>>>>>>>>>>>        >>>>>>>> starting
>>>>>>>>>>>>>        >>>>>>>>>>>>> from
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost
>>>>>>>>>>>>> difference
>>>>>>>>>>>>>        between
>>>>>>>>>>>>>        >>>>>>>> runtime
>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure
>>>>>>>>>>>>> uncovers
>>>>>>>>>>>>>        >>>>> instantly
>>>>>>>>>>>>>        >>>>>> on
>>>>>>>>>>>>>        >>>>>>>>> unit
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> tests, it costs more for the project
>>>>>>>>>>>>> than a
>>>>>>>>>>>>>        compilation
>>>>>>>>>>>>>        >>>>>>>> failure.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>>> operation being
>>>>>>>>>>>>>        required.
>>>>>>>>>>>>>        >>>>>>> But
>>>>>>>>>>>>>        >>>>>>>> is
>>>>>>>>>>>>>        >>>>>>>>>>>>> that
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't
>>>>>>>>>>>>> want a
>>>>>>>>>>>>>        >>>>>> defaultBranch
>>>>>>>>>>>>>        >>>>>>>>> they
>>>>>>>>>>>>>        >>>>>>>>>>>>> can
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>>        just as
>>>>>>>>>>>>>        >>>>>>>>> easily.  In
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a
>>>>>>>>>>>>> nicer API
>>>>>>>>>>>> - a
>>>>>>>>>>>>>        >>>>> user
>>>>>>>>>>>>>        >>>>>>>> could
>>>>>>>>>>>>>        >>>>>>>>>>>>> specify
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing
>>>>>>>>>>>>> will reach
>>>>>>>>>>>> the
>>>>>>>>>>>>>        >>>>>> default
>>>>>>>>>>>>>        >>>>>>>>> branch,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>>>>>> occurs.
>>>>>>>>>>>> That
>>>>>>>>>>>>>        >>>>> seems
>>>>>>>>>>>>>        >>>>>>> like
>>>>>>>>>>>>>        >>>>>>>>> an
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> improvement over the current branch()
>>>>>>>>>>>>> API,
>>>>>>>>>>>>>        which allows
>>>>>>>>>>>>>        >>>>>> for
>>>>>>>>>>>>>        >>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>> more
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly
>>>>>>>>>>>>> getting
>>>>>>>>>>>> dropped.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>>> certainly has
>>>>>>>>>>>>>        to be
>>>>>>>>>>>>>        >>>>>> well
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>>        >>>>> InternalTopologyBuilder
>>>>>>>>>>>>>        >>>>>>> to
>>>>>>>>>>>>>        >>>>>>>>> track
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>>>>>> and raise
>>>>>>>>>>>>>        a clear
>>>>>>>>>>>>>        >>>>>>> error
>>>>>>>>>>>>>        >>>>>>>>>>>>> before it
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that
>>>>>>>>>>>>> there is
>>>>>>>>>>>> a
>>>>>>>>>>>>>        >>>>> "build
>>>>>>>>>>>>>        >>>>>>>> step"
>>>>>>>>>>>>>        >>>>>>>>>>>>> where
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>>>>>>>>        >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>>        >>>>>>> is
>>>>>>>>>>>>>        >>>>>>>>>>>>> called.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>>> argument, I
>>>>>>>>>>>> agree
>>>>>>>>>>>>>        >>>>> that
>>>>>>>>>>>>>        >>>>>>> it's
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> allow users to do other operations on
>>>>>>>>>>>>> the input
>>>>>>>>>>>>>        stream.
>>>>>>>>>>>>>        >>>>>>> With
>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same
>>>>>>>>>>>>> way all
>>>>>>>>>>>> other
>>>>>>>>>>>>>        >>>>>>> operations
>>>>>>>>>>>>>        >>>>>>>>> do -
>>>>>>>>>>>>>        >>>>>>>>>>>>> if
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> want to process off the original
>>>>>>>>>>>>> KStream
>>>>>>>>>>>> multiple
>>>>>>>>>>>>>        >>>>> times,
>>>>>>>>>>>>>        >>>>>>> you
>>>>>>>>>>>>>        >>>>>>>>> just
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call
>>>>>>>>>>>>> as many
>>>>>>>>>>>>>        operations
>>>>>>>>>>>>>        >>>>>> on
>>>>>>>>>>>>>        >>>>>>> it
>>>>>>>>>>>>>        >>>>>>>>> as
>>>>>>>>>>>>>        >>>>>>>>>>>>> you
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan
>>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>>        >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we
>>>>>>>>>>>>> do not
>>>>>>>>>>>>>        always need
>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>>>>>>>>> operation we
>>>>>>>>>>>>>        >>>>> don't
>>>>>>>>>>>>>        >>>>>>>> know
>>>>>>>>>>>>>        >>>>>>>>>>>>> when to
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch
>>>>>>>>>>>>> switch'.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its
>>>>>>>>>>>>> argument,
>>>>>>>>>>>>>        so we
>>>>>>>>>>>>>        >>>>> can
>>>>>>>>>>>>>        >>>>>> do
>>>>>>>>>>>>>        >>>>>>>>>>>>> something
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> more with the original branch after
>>>>>>>>>>>>> branching.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> I understand your point that the
>>>>>>>>>>>>> need of
>>>>>>>>>>>> special
>>>>>>>>>>>>>        >>>>> object
>>>>>>>>>>>>>        >>>>>>>>>>>>> construction
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream
>>>>>>>>>>>>> methods.
>>>>>>>>>>>> But
>>>>>>>>>>>>>        >>>>> here
>>>>>>>>>>>>>        >>>>>> we
>>>>>>>>>>>>>        >>>>>>>>> have a
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to
>>>>>>>>>>>>> split the
>>>>>>>>>>>>>        flow,
>>>>>>>>>>>>>        >>>>> so
>>>>>>>>>>>>>        >>>>>> I
>>>>>>>>>>>>>        >>>>>>>>> think
>>>>>>>>>>>>>        >>>>>>>>>>>>> this
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve
>>>>>>>>>>>>> this
>>>>>>>>>>>>>        API, but I
>>>>>>>>>>>>>        >>>>>> find
>>>>>>>>>>>>>        >>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing
>>>>>>>>>>>>> since it
>>>>>>>>>>>>>        contrasts the
>>>>>>>>>>>>>        >>>>>>> fluency
>>>>>>>>>>>>>        >>>>>>>>> of
>>>>>>>>>>>>>        >>>>>>>>>>>>> other
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd
>>>>>>>>>>>>> like to
>>>>>>>>>>>>>        just call
>>>>>>>>>>>>>        >>>>> a
>>>>>>>>>>>>>        >>>>>>>>> method on
>>>>>>>>>>>>>        >>>>>>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if
>>>>>>>>>>>>> the branch
>>>>>>>>>>>>>        cases
>>>>>>>>>>>>>        >>>>> are
>>>>>>>>>>>>>        >>>>>>>>> defined
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate,
>>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>>        is very
>>>>>>>>>>>>>        >>>>>> nice
>>>>>>>>>>>>>        >>>>>>>>> and the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped
>>>>>>>>>>>>> around
>>>>>>>>>>>>>        how we
>>>>>>>>>>>>>        >>>>>>> specify
>>>>>>>>>>>>>        >>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>> source
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>>        >>>>>>>> KStreamBrancher
>>>>>>>>>>>>>        >>>>>>>>> or
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>>>>>>>>        terminated by
>>>>>>>>>>>>>        >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>>>>>>>>        incompatible with
>>>>>>>>>>>>>        >>>>> the
>>>>>>>>>>>>>        >>>>>>>>> current
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to
>>>>>>>>>>>>> have a
>>>>>>>>>>>>>        different
>>>>>>>>>>>>>        >>>>>> name,
>>>>>>>>>>>>>        >>>>>>>> but
>>>>>>>>>>>>>        >>>>>>>>> that
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we
>>>>>>>>>>>>> could call it
>>>>>>>>>>>>>        >>>>>> something
>>>>>>>>>>>>>        >>>>>>>> like
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the
>>>>>>>>>>>>> old API.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of
>>>>>>>>>>>>> your
>>>>>>>>>>>>>        KIP?  It
>>>>>>>>>>>>>        >>>>>> seems
>>>>>>>>>>>>>        >>>>>>>>> like it
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>>> branching
>>>>>>>>>>>>>        while also
>>>>>>>>>>>>>        >>>>>>>> allowing
>>>>>>>>>>>>>        >>>>>>>>> you
>>>>>>>>>>>>>        >>>>>>>>>>>>> to
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>>>>>>>>        KBranchedStreams
>>>>>>>>>>>>>        >>>>>> if
>>>>>>>>>>>>>        >>>>>>>>> desired.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan
>>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>>        ks){
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>>        String> ks){
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String,
>>>>>>>>>>>>> String>()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>>        this::handleFirstCase)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>>        this::handleSecondCase)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>>        >>>>> takes a
>>>>>>>>>>>>>        >>>>>>>>> Consumer
>>>>>>>>>>>>>        >>>>>>>>>>>>> as a
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing,
>>>>>>>>>>>>> and the
>>>>>>>>>>>>>        example in
>>>>>>>>>>>>>        >>>>>> the
>>>>>>>>>>>>>        >>>>>>>> KIP
>>>>>>>>>>>>>        >>>>>>>>>>>>> shows
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a
>>>>>>>>>>>>> terminal node
>>>>>>>>>>>>>        >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but
>>>>>>>>>>>>> how would
>>>>>>>>>>>> we
>>>>>>>>>>>>>        >>>>> handle
>>>>>>>>>>>>>        >>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>> case
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>>> wants to
>>>>>>>>>>>> continue
>>>>>>>>>>>>>        >>>>>>>> processing
>>>>>>>>>>>>>        >>>>>>>>> and
>>>>>>>>>>>>>        >>>>>>>>>>>>> not
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on
>>>>>>>>>>>>> the
>>>>>>>>>>>> branched
>>>>>>>>>>>>>        >>>>>> stream
>>>>>>>>>>>>>        >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic
>>>>>>>>>>>>> as is if
>>>>>>>>>>>>>        we had
>>>>>>>>>>>>>        >>>>>>>> something
>>>>>>>>>>>>>        >>>>>>>>> like
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[]
>>>>>>>>>>>>> branches =
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM
>>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>>> <
>>>>>>>>>>>>>        >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>>> discussion for
>>>>>>>>>>>> KIP-
>>>>>>>>>>>>>        >>>>> 418.
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion
>>>>>>>>>>>>> about
>>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>>        >>>>> Please
>>>>>>>>>>>>>        >>>>>>>> take
>>>>>>>>>>>>>        >>>>>>>>> a
>>>>>>>>>>>>>        >>>>>>>>>>>>> look
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>>        feedback :)
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>>        >>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>>        >>>>>>>>>>>>>
>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>>        >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>        >>>>>>>>>
>>>>>>>>>>>>>        >
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>>
>>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <mj...@apache.org>.
Thanks for updating the KIP!

I also have some minor comment:



(1) We should rename `KBranchedStream` -> `BranchedKStream`

(Most classed follow this naming pattern now, eg, CoGroupedKStream,
TimeWindowedKStream etc -- there is just the "legacy" `KGroupedStream`
and `KGroupedKTable` that we cannot rename without a breaking change...
so we just keep them.)



(2) Quote:

> Both branch and defaultBranch operations also have overloaded parameterless alternatives.

I think `branch()` always needs to take a `Predicate` and assume you
meant that `Branched` is optional. Can you maybe rephrase it accordingly
as `branch()` would not be "parameterless".



(3) Can you maybe add an overview in the "Public Interface" section) of
newly added and deprecated methods/classes (cf.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup#KIP-150-Kafka-StreamsCogroup-PublicInterfaces)



(4) What is unclear from the KIP is the interaction of `withConsumer()`
and the finally returned `Map<String, KStream>`. This related to John's
4th comment:

> It seems like there are really two disjoint use cases: EITHER using chain and the result map OR using just the sink.

I don't think that using both `withChain()` and `withConsumer()` is the
issue though, as the KIP clearly states that the result of `withChain()`
will be given to the `Consumer`. The issue is really with the `Consumer`
and the returned `Map` of `defautBranch()` and `noDefaultBranch()`.

Maybe a reasonable implementation would be to not add the "branch" to
the result map if `withConsumer` is used? As long as we clearly document
it in the JavaDocs, this might be fine?



(5) Reply to John's comments:

> 3. Reading the KIP, I found ‘withConsumer’ confusing; I thought you were talking about the kafka Consumer interface (which doesn’t make sense, of course). I get that you were referring to the java Consumer interface, but we should still probably to to avoid the ambiguity. Just throwing out a suggestion, how about ‘withSink’?

IMHO, `withSink` has the issue that it might be confused with a "sink
node", ie., writing the KStream to a topic.

Maybe `withJavaConsumer` would make it less ambiguous?




-Matthias




On 5/8/20 7:13 AM, John Roesler wrote:
> Hi Ivan,
> 
> It looks like you missed my reply on Apr 23rd. I think it’s close, but I had a few last comments. 
> 
> Thanks,
> John
> 
> On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
>> Hello everyone,
>>
>> will someone please take a look at the reworked KIP?
>>
>> I believe that now it follows design principles and takes into account 
>> all the arguments discussed here.
>>
>>
>> Regards,
>>
>> Ivan
>>
>>
>> 23.04.2020 2:45, Ivan Ponomarev пишет:
>>> Hi,
>>>
>>> I have read the John's "DSL design principles" and have completely 
>>> rewritten the KIP, see 
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>
>>>
>>>
>>> This version includes all the previous discussion results and follows 
>>> the design principles, with one exception.
>>>
>>> The exception is
>>>
>>> branch(Predicate<K,V> predicate, Branched<K,V> branched)
>>>
>>> which formally violates 'no more than one parameter' rule, but I think 
>>> here it is justified.
>>>
>>> We must provide a predicate for a branch and don't need to provide one 
>>> for the default branch. Thus for both operations we may use a single 
>>> Branched parameter class, with an extra method parameter for `branch`.
>>>
>>> Since predicate is a natural, necessary part of a branch, no 
>>> 'proliferation of overloads, deprecations, etc.' is expected here as it 
>>> is said in the rationale for the 'single parameter rule'.
>>>
>>> WDYT, is this KIP mature enough to begin voting?
>>>
>>> Regards,
>>>
>>> Ivan
>>>
>>> 21.04.2020 2:09, Matthias J. Sax пишет:
>>>> Ivan,
>>>>
>>>> no worries about getting side tracked. Glad to have you back!
>>>>
>>>> The DSL improved further in the meantime and we already have a `Named`
>>>> config object to name operators. It seems reasonable to me to build on 
>>>> this.
>>>>
>>>> Furthermore, John did a writeup about "DSL design principles" that we
>>>> want to follow:
>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar 
>>>>
>>>> -- might be worth to checkout.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>>>> Hi everyone!
>>>>>
>>>>> Let me revive the discussion of this KIP.
>>>>>
>>>>> I'm very sorry for stopping my participation in the discussion in June
>>>>> 2019. My project work was very intensive then and it didn't leave me
>>>>> spare time. But I think I must finish this, because we invested
>>>>> substantial effort into this discussion and I'm not feel entitled to
>>>>> propose other things before this one is finalized.
>>>>>
>>>>> During these months I proceeded with writing and reviewing Kafka
>>>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>>>> KafkaStreamBrancher class of my invention (the original idea for this
>>>>> KIP) worked for me -- that's another reason why I gave up pushing the
>>>>> KIP forward. When I was coming across the problem with the scope of
>>>>> branches, I worked around it this way:
>>>>>
>>>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>>>> new KafkaStreamBrancher<....>()
>>>>>      .branch(....)
>>>>>      .defaultBranch(result::set)
>>>>>      .onTopOf(someStream);
>>>>> result.get()...
>>>>>
>>>>>
>>>>> And yes, of course I don't feel very happy with this approach.
>>>>>
>>>>> I think that Matthias came up with a bright solution in his post from
>>>>> May, 24th 2019. Let me quote it:
>>>>>
>>>>> KStream#split() -> KBranchedStream
>>>>> // branch is not easily accessible in current scope
>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>    -> KBranchedStream
>>>>> // assign a name to the branch and
>>>>> // return the sub-stream to the current scope later
>>>>> //
>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>    -> KBranchedStream
>>>>> // default branch is not easily accessible
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>    -> Map<String,KStream>
>>>>> // assign custom name to default-branch
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>    -> Map<String,KStream>
>>>>> // assign a default name for default
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>    -> Map<String,KStream>
>>>>> // return map of all names sub-stream into current scope
>>>>> KBranchedStream#noDefaultBranch()
>>>>>    -> Map<String,KStream>
>>>>>
>>>>> I believe this would satisfy everyone. Optional names seems to be a good
>>>>> idea: when you don't need to have the branches in the same scope, you
>>>>> just don't use names and you don't risk making your code brittle. Or,
>>>>> you might want to add names just for debugging purposes. Or, finally,
>>>>> you might use the returned Map to have the named branches in the
>>>>> original scope.
>>>>>
>>>>> There also was an input from John Roesler on June 4th, 2019, who
>>>>> suggested using Named class. I can't comment on this. The idea seems
>>>>> reasonable, but in this matter I'd rather trust people who are more
>>>>> familiar with Streams API design principles than me.
>>>>>
>>>>> Regards,
>>>>>
>>>>> Ivan
>>>>>
>>>>>
>>>>>
>>>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>>>> I am moving this KIP into "inactive status". Feel free to resume the 
>>>>>> KIP
>>>>>> at any point.
>>>>>>
>>>>>> If anybody else is interested in picking up this KIP, feel free to 
>>>>>> do so.
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>>>> Ivan,
>>>>>>>
>>>>>>> did you see my last reply? What do you think about my proposal to mix
>>>>>>> both approaches and try to get best-of-both worlds?
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>>>> Thanks for the input John!
>>>>>>>>
>>>>>>>>> under your suggestion, it seems that the name is required
>>>>>>>>
>>>>>>>> If you want to get the `KStream` as part of the `Map` back using a
>>>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern 
>>>>>>>> using a
>>>>>>>> `Consumer`, no.
>>>>>>>>
>>>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>>>
>>>>>>>> I wanted to sketch out a high level proposal to merge both patterns
>>>>>>>> only. Your suggestions to align the new API with the existing API 
>>>>>>>> make
>>>>>>>> totally sense.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>>>
>>>>>>>> If both are mandatory, what do we gain by it? The returned `Map` only
>>>>>>>> contains the corresponding branches, so why should we prefix all of
>>>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>>>> `split()`, the same question raises?
>>>>>>>>
>>>>>>>> Requiring `Named` in `split()` seems only to make sense, if 
>>>>>>>> `Named` is
>>>>>>>> optional in `branch()` and we generate `-X` suffix using a counter 
>>>>>>>> for
>>>>>>>> different branch name. However, this might lead to the problem of
>>>>>>>> changing names if branches are added/removed. Also, how would the 
>>>>>>>> names
>>>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>>>> returned in the `Map`).
>>>>>>>>
>>>>>>>> If `Named` is optional for both, it could happen that a user 
>>>>>>>> misses to
>>>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>>>
>>>>>>>>
>>>>>>>> Hence, I am actually in favor to not allow a default name but keep
>>>>>>>> `split()` without parameter and make `Named` in `branch()` required
>>>>>>>> if a
>>>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>>>> specifying a
>>>>>>>> name is required if a `Function` is used.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> About
>>>>>>>>
>>>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>>>
>>>>>>>> I don't think that the branching predicate is a configuration and 
>>>>>>>> hence
>>>>>>>> would not include it in a configuration object.
>>>>>>>>
>>>>>>>>>       withChain(...);
>>>>>>>>
>>>>>>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>>>>>>>> seem to be a configuration. We can also not prevent a user to call
>>>>>>>> `withName()` in combination of `withChain()` what does not make sense
>>>>>>>> IMHO. We could of course throw an RTE but not have a compile time 
>>>>>>>> check
>>>>>>>> seems less appealing. Also, it could happen that neither 
>>>>>>>> `withChain()`
>>>>>>>> not `withName()` is called and the branch is missing in the returned
>>>>>>>> `Map` what lead to runtime issues, too.
>>>>>>>>
>>>>>>>> Hence, I don't think that we should add `BranchConfig`. A config 
>>>>>>>> object
>>>>>>>> is helpful if each configuration can be set independently of all
>>>>>>>> others,
>>>>>>>> but this seems not to be the case here. If we add new configuration
>>>>>>>> later, we can also just move forward by deprecating the methods that
>>>>>>>> accept `Named` and add new methods that accepted `BranchConfig` (that
>>>>>>>> would of course implement `Named`).
>>>>>>>>
>>>>>>>>
>>>>>>>> Thoughts?
>>>>>>>>
>>>>>>>>
>>>>>>>> @Ivan, what do you think about the general idea to blend the two main
>>>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>>>>>>>> everyone. Returning the map from the terminal operations also solves
>>>>>>>>> the problem of merging/joining the branched streams, if we want 
>>>>>>>>> to add
>>>>>>>>> support for the compliment later on.
>>>>>>>>>
>>>>>>>>> Under your suggestion, it seems that the name is required. 
>>>>>>>>> Otherwise,
>>>>>>>>> we wouldn't have keys for the map to return. I this this is actually
>>>>>>>>> not too bad, since experience has taught us that, although names for
>>>>>>>>> operations are not required to define stream processing logic, it 
>>>>>>>>> does
>>>>>>>>> significantly improve the operational experience when you can map 
>>>>>>>>> the
>>>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>>>> wouldn't (have to) reference the name to chain extra processing onto
>>>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>>>
>>>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>>>> operator itself, and then all the branches get index-suffixed names
>>>>>>>>> built from the branch operator name. I guess under this proposal, we
>>>>>>>>> could naturally append the branch name to the branching operator 
>>>>>>>>> name,
>>>>>>>>> like this:
>>>>>>>>>
>>>>>>>>>      stream.split(Named.withName("mysplit")) //creates node 
>>>>>>>>> "mysplit"
>>>>>>>>>                 .branch(..., ..., "abranch") // creates node
>>>>>>>>> "mysplit-abranch"
>>>>>>>>>                 .defaultBranch(...) // creates node 
>>>>>>>>> "mysplit-default"
>>>>>>>>>
>>>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>>>
>>>>>>>>> We don't have a defined grammar, so there's plenty of room to debate
>>>>>>>>> the "best" syntax in the context of each operation, but in general,
>>>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>>>
>>>>>>>>>       operator(function, config_object?) OR operator(config_object)
>>>>>>>>>
>>>>>>>>> where config_object is often just Named in the "function" variant.
>>>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>>>
>>>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>>>
>>>>>>>>>     operator(function, function, string)
>>>>>>>>>
>>>>>>>>> where the string is the name.
>>>>>>>>> My first question is whether the name should instead be specified 
>>>>>>>>> with
>>>>>>>>> the NamedOperation interface.
>>>>>>>>>
>>>>>>>>> My second question is whether we should just roll all these 
>>>>>>>>> arguments
>>>>>>>>> up into a config object like:
>>>>>>>>>
>>>>>>>>>      KBranchedStream#branch(BranchConfig)
>>>>>>>>>
>>>>>>>>>      interface BranchConfig extends NamedOperation {
>>>>>>>>>       withPredicate(...);
>>>>>>>>>       withChain(...);
>>>>>>>>>       withName(...);
>>>>>>>>>     }
>>>>>>>>>
>>>>>>>>> Although I guess we'd like to call BranchConfig something more like
>>>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>>>
>>>>>>>>> This makes the source code a little noisier, but it also makes us 
>>>>>>>>> more
>>>>>>>>> future-proof, as we can deal with a wide range of alternatives 
>>>>>>>>> purely
>>>>>>>>> in the config interface, and never have to deal with adding 
>>>>>>>>> overloads
>>>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>>>
>>>>>>>>> WDYT?
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>>>
>>>>>>>>>> Matthias: I think that's pretty reasonable from my point of view.
>>>>>>>>>> Good
>>>>>>>>>> suggestion.
>>>>>>>>>>
>>>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>>>> <ma...@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Interesting discussion.
>>>>>>>>>>>
>>>>>>>>>>> I am wondering, if we cannot unify the advantage of both 
>>>>>>>>>>> approaches:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>>>
>>>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>>>     -> KBranchedStream
>>>>>>>>>>>
>>>>>>>>>>> // assign a name to the branch and
>>>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>>>> //
>>>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, 
>>>>>>>>>>> String)
>>>>>>>>>>>     -> KBranchedStream
>>>>>>>>>>>
>>>>>>>>>>> // default branch is not easily accessible
>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>>>
>>>>>>>>>>> // assign custom name to default-branch
>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>>>
>>>>>>>>>>> // assign a default name for default
>>>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>>>
>>>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name and
>>>>>>>>>>> return
>>>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>>>> implementation can
>>>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>>>> returned
>>>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>>>
>>>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>>>
>>>>>>>>>>> Thoughts?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>
>>>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>>>> dynamic case.
>>>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>>>> necessity.
>>>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>>>
>>>>>>>>>>>> Separately, I'm interested to see where the present discussion
>>>>>>>>>>>> leads.
>>>>>>>>>>>> I've written enough Javascript code in my life to be 
>>>>>>>>>>>> suspicious of
>>>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>>>> references (or
>>>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>>>> that this
>>>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>>>> logic when
>>>>>>>>>>>> their nested closure situation got out of hand. Unfortunately, 
>>>>>>>>>>>> it's
>>>>>>>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>>>>>>>> readability (but in different ways for different reasons). In 
>>>>>>>>>>>> other
>>>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>>>> solution if
>>>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>>>
>>>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>>>> references aren't
>>>>>>>>>>>> the end of the story either, and you can see that by observing 
>>>>>>>>>>>> that
>>>>>>>>>>>> there have been two follow-on eras, as they continue trying to
>>>>>>>>>>>> cope with
>>>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>>>> First, you
>>>>>>>>>>>> have Futures/Promises, which essentially let you convert nested
>>>>>>>>>>>> code to
>>>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>>>> this).
>>>>>>>>>>>> Most lately, you have async/await, which is an effort to apply
>>>>>>>>>>>> language
>>>>>>>>>>>> (not just API) syntax to the problem, and offer the "flattest"
>>>>>>>>>>>> possible
>>>>>>>>>>>> programming style to solve the problem (because you get back to
>>>>>>>>>>>> just one
>>>>>>>>>>>> code block per functional unit).
>>>>>>>>>>>>
>>>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>>>> nowhere
>>>>>>>>>>>> near as callback heavy as JS, so I don't think we have to take
>>>>>>>>>>>> the JS
>>>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>>>> valuable
>>>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>>>> bringing this
>>>>>>>>>>>> up to inspire further/deeper discussion. At the same time, just
>>>>>>>>>>>> like JS,
>>>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>>>
>>>>>>>>>>>> Separately again, I'm interested in the post-branch merge (and
>>>>>>>>>>>> I'd also
>>>>>>>>>>>> add join) problem that Paul brought up. We can clearly punt on
>>>>>>>>>>>> it, by
>>>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>>>> there a DSL
>>>>>>>>>>>> way to do it?
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>>>> -John
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>       Ivan, I’ll definitely forfeit my point on the clumsiness of
>>>>>>>>>>>> the
>>>>>>>>>>>>       branch(predicate, consumer) solution, I don’t see any real
>>>>>>>>>>>> drawbacks
>>>>>>>>>>>>       for the dynamic case.
>>>>>>>>>>>>
>>>>>>>>>>>>       IMO the one trade off to consider at this point is the 
>>>>>>>>>>>> scope
>>>>>>>>>>>>       question. I don’t know if I totally agree that “we rarely
>>>>>>>>>>>> need them
>>>>>>>>>>>>       in the same scope” since merging the branches back together
>>>>>>>>>>>> later
>>>>>>>>>>>>       seems like a perfectly plausible use case that can be a lot
>>>>>>>>>>>> nicer
>>>>>>>>>>>>       when the branched streams are in the same scope. That being
>>>>>>>>>>>> said,
>>>>>>>>>>>>       for the reasons Ivan listed, I think it is overall the 
>>>>>>>>>>>> better
>>>>>>>>>>>>       solution - working around the scope thing is easy enough if
>>>>>>>>>>>> you need
>>>>>>>>>>>>       to.
>>>>>>>>>>>>
>>>>>>>>>>>>       > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>>>       <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > Hello everyone, thank you all for joining the discussion!
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > Well, I don't think the idea of named branches, be it a
>>>>>>>>>>>>       LinkedHashMap (no other Map will do, because order of
>>>>>>>>>>>> definition
>>>>>>>>>>>>       matters) or `branch` method  taking name and Consumer 
>>>>>>>>>>>> has more
>>>>>>>>>>>>       advantages than drawbacks.
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > In my opinion, the only real positive outcome from 
>>>>>>>>>>>> Michael's
>>>>>>>>>>>>       proposal is that all the returned branches are in the same
>>>>>>>>>>>> scope.
>>>>>>>>>>>>       But 1) we rarely need them in the same scope 2) there is a
>>>>>>>>>>>>       workaround for the scope problem, described in the KIP.
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > 'Inlining the complex logic' is not a problem, because we
>>>>>>>>>>>> can use
>>>>>>>>>>>>       method references instead of lambdas. In real world
>>>>>>>>>>>> scenarios you
>>>>>>>>>>>>       tend to split the complex logic to methods anyway, so the
>>>>>>>>>>>> code is
>>>>>>>>>>>>       going to be clean.
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > The drawbacks are strong. The cohesion between predicates
>>>>>>>>>>>> and
>>>>>>>>>>>>       handlers is lost. We have to define predicates in one
>>>>>>>>>>>> place, and
>>>>>>>>>>>>       handlers in another. This opens the door for bugs:
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > - what if we forget to define a handler for a name? or a
>>>>>>>>>>>> name for
>>>>>>>>>>>>       a handler?
>>>>>>>>>>>>       > - what if we misspell a name?
>>>>>>>>>>>>       > - what if we copy-paste and duplicate a name?
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > What Michael propose would have been totally OK if we had
>>>>>>>>>>>> been
>>>>>>>>>>>>       writing the API in Lua, Ruby or Python. In those 
>>>>>>>>>>>> languages the
>>>>>>>>>>>>       "dynamic naming" approach would have looked most concise 
>>>>>>>>>>>> and
>>>>>>>>>>>>       beautiful. But in Java we expect all the problems 
>>>>>>>>>>>> related to
>>>>>>>>>>>>       identifiers to be eliminated in compile time.
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > And if we do, what advantage are we supposed to get
>>>>>>>>>>>> besides having
>>>>>>>>>>>>       all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>>>> missing your
>>>>>>>>>>>>       point?
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > ---
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > Earlier in this discussion John Roesler also proposed 
>>>>>>>>>>>> to do
>>>>>>>>>>>>       without "start branching" operator, and later Paul
>>>>>>>>>>>> mentioned that in
>>>>>>>>>>>>       the case when we have to add a dynamic number of 
>>>>>>>>>>>> branches, the
>>>>>>>>>>>>       current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>>>> solution. Let
>>>>>>>>>>>>       me address both comments here.
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > 1) "Start branching" operator (I think that *split* is a
>>>>>>>>>>>> good name
>>>>>>>>>>>>       for it indeed) is critical when we need to do a dynamic
>>>>>>>>>>>> branching,
>>>>>>>>>>>>       see example below.
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > 2) No, dynamic branching in current KIP is not clumsy at
>>>>>>>>>>>> all.
>>>>>>>>>>>>       Imagine a real-world scenario when you need one branch per
>>>>>>>>>>>> enum
>>>>>>>>>>>>       value (say, RecordType). You can have something like this:
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > /*John:if we had to start with stream.branch(...) here,
>>>>>>>>>>>> it would
>>>>>>>>>>>>       have been much messier.*/
>>>>>>>>>>>>       > KBranchedStream branched = stream.split();
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > /*Not clumsy at all :-)*/
>>>>>>>>>>>>       > for (RecordType recordType : RecordType.values())
>>>>>>>>>>>>       >             branched = branched.branch((k, v) ->
>>>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>>>       recordType,
>>>>>>>>>>>>       >                     recordType::processRecords);
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > Regards,
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > Ivan
>>>>>>>>>>>>       >
>>>>>>>>>>>>       >
>>>>>>>>>>>>       > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>>>       >> I also agree with Michael's observation about the core
>>>>>>>>>>>> problem of
>>>>>>>>>>>>       >> current `branch()` implementation.
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>>>> object. My
>>>>>>>>>>>>       thinking
>>>>>>>>>>>>       >> was more aligned with Paul's proposal to just add a name
>>>>>>>>>>>> to each
>>>>>>>>>>>>       >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >> It makes the code easier to read, and also make the
>>>>>>>>>>>> order of
>>>>>>>>>>>>       >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>>>>       >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>       >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>       >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>>>       >> An open question is the case for which no
>>>>>>>>>>>> defaultBranch() should
>>>>>>>>>>> be
>>>>>>>>>>>>       >> specified. Atm, `split()` and `branch()` would return
>>>>>>>>>>>>       `BranchedKStream`
>>>>>>>>>>>>       >> and the call to `defaultBranch()` that returns the 
>>>>>>>>>>>> `Map` is
>>>>>>>>>>> mandatory
>>>>>>>>>>>>       >> (what is not the case atm). Or is this actually not a 
>>>>>>>>>>>> real
>>>>>>>>>>> problem,
>>>>>>>>>>>>       >> because users can just ignore the branch returned by
>>>>>>>>>>>>       `defaultBranch()`
>>>>>>>>>>>>       >> in the result `Map` ?
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >> About "inlining": So far, it seems to be a matter of
>>>>>>>>>>>> personal
>>>>>>>>>>>>       >> preference. I can see arguments for both, but no "killer
>>>>>>>>>>>>       argument" yet
>>>>>>>>>>>>       >> that clearly make the case for one or the other.
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >> -Matthias
>>>>>>>>>>>>       >>
>>>>>>>>>>>>       >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>>>       >>> Perhaps inlining is the wrong terminology. It doesn’t
>>>>>>>>>>>> require
>>>>>>>>>>>>       that a lambda with the full downstream topology be defined
>>>>>>>>>>>> inline -
>>>>>>>>>>>>       it can be a method reference as with Ivan’s original
>>>>>>>>>>>> suggestion.
>>>>>>>>>>>>       The advantage of putting the predicate and its downstream
>>>>>>>>>>>> logic
>>>>>>>>>>>>       (Consumer) together in branch() is that they are required
>>>>>>>>>>>> to be near
>>>>>>>>>>>>       to each other.
>>>>>>>>>>>>       >>>
>>>>>>>>>>>>       >>> Ultimately the downstream code has to live somewhere,
>>>>>>>>>>>> and deep
>>>>>>>>>>>>       branch trees will be hard to read regardless.
>>>>>>>>>>>>       >>>
>>>>>>>>>>>>       >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>>>       <michael.drogalis@confluent.io
>>>>>>>>>>>>       <ma...@confluent.io>> wrote:
>>>>>>>>>>>>       >>>>
>>>>>>>>>>>>       >>>> I'm less enthusiastic about inlining the branch logic
>>>>>>>>>>>> with its
>>>>>>>>>>>>       downstream
>>>>>>>>>>>>       >>>> functionality. Programs that have deep branch trees 
>>>>>>>>>>>> will
>>>>>>>>>>>>       quickly become
>>>>>>>>>>>>       >>>> harder to read as a single unit.
>>>>>>>>>>>>       >>>>
>>>>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>>>       <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Also +1 on the issues/goals as Michael outlined them,
>>>>>>>>>>>> I think
>>>>>>>>>>>>       that sets a
>>>>>>>>>>>>       >>>>> great framework for the discussion.
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Regarding the SortedMap solution, my understanding is
>>>>>>>>>>>> that the
>>>>>>>>>>>>       current
>>>>>>>>>>>>       >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>>>> (pending naming
>>>>>>>>>>>>       decisions) is
>>>>>>>>>>>>       >>>>> roughly this:
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> stream.split()
>>>>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>>>>       >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Obviously some ordering is necessary, since branching
>>>>>>>>>>>> as a
>>>>>>>>>>>>       construct
>>>>>>>>>>>>       >>>>> doesn't work without it, but this solution seems 
>>>>>>>>>>>> like it
>>>>>>>>>>>>       provides as much
>>>>>>>>>>>>       >>>>> associativity as the SortedMap solution, because each
>>>>>>>>>>>> branch()
>>>>>>>>>>>>       call
>>>>>>>>>>>>       >>>>> directly associates the "conditional" with the "code
>>>>>>>>>>>> block."
>>>>>>>>>>>>       The value it
>>>>>>>>>>>>       >>>>> provides over the KIP solution is the accessing of
>>>>>>>>>>>> streams in
>>>>>>>>>>>>       the same
>>>>>>>>>>>>       >>>>> scope.
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> The KIP solution is less "dynamic" than the SortedMap
>>>>>>>>>>>> solution
>>>>>>>>>>>>       in the sense
>>>>>>>>>>>>       >>>>> that it is slightly clumsier to add a dynamic 
>>>>>>>>>>>> number of
>>>>>>>>>>>>       branches, but it is
>>>>>>>>>>>>       >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>>>> should favor
>>>>>>>>>>>>       the "static"
>>>>>>>>>>>>       >>>>> case anyway, and should make it simple and 
>>>>>>>>>>>> readable to
>>>>>>>>>>>>       fluently declare and
>>>>>>>>>>>>       >>>>> access your branches in-line.  It also makes it
>>>>>>>>>>>> impossible to
>>>>>>>>>>>>       ignore a
>>>>>>>>>>>>       >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>>>> identical
>>>>>>>>>>>>       SortedMap
>>>>>>>>>>>>       >>>>> solution on top of it.
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> I could also see a middle ground where instead of 
>>>>>>>>>>>> a raw
>>>>>>>>>>>>       SortedMap being
>>>>>>>>>>>>       >>>>> taken in, branch() takes a name and not a Consumer.
>>>>>>>>>>>> Something
>>>>>>>>>>>>       like this:
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>>>>       >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>>>       >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>>>       >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Pros for that solution:
>>>>>>>>>>>>       >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>>>       >>>>> - no double brace initialization, hopefully slightly
>>>>>>>>>>>> more
>>>>>>>>>>>>       readable than
>>>>>>>>>>>>       >>>>> SortedMap
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Cons
>>>>>>>>>>>>       >>>>> - downstream branch logic cannot be specified inline
>>>>>>>>>>>> which
>>>>>>>>>>>>       makes it harder
>>>>>>>>>>>>       >>>>> to read top to bottom (like existing API and
>>>>>>>>>>>> SortedMap, but
>>>>>>>>>>>>       unlike the KIP)
>>>>>>>>>>>>       >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>>>> streams (like
>>>>>>>>>>>>       existing
>>>>>>>>>>>>       >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> (KBranchedStreams could even work *both* ways but
>>>>>>>>>>>> perhaps
>>>>>>>>>>>>       that's overdoing
>>>>>>>>>>>>       >>>>> it).
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Overall I'm curious how important it is to be able to
>>>>>>>>>>>> easily
>>>>>>>>>>>>       access the
>>>>>>>>>>>>       >>>>> branched KStream in the same scope as the original.
>>>>>>>>>>>> It's
>>>>>>>>>>>>       possible that it
>>>>>>>>>>>>       >>>>> doesn't need to be handled directly by the API, but
>>>>>>>>>>>> instead
>>>>>>>>>>>>       left up to the
>>>>>>>>>>>>       >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> Paul
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>>>>>>>       <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>       >>>>> wrote:
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>       >>>>>> I'd like to +1 what Michael said about the issues
>>>>>>>>>>>> with the
>>>>>>>>>>>>       existing
>>>>>>>>>>>>       >>>>> branch
>>>>>>>>>>>>       >>>>>> method, I agree with what he's outlined and I think
>>>>>>>>>>>> we should
>>>>>>>>>>>>       proceed by
>>>>>>>>>>>>       >>>>>> trying to alleviate these problems. Specifically it
>>>>>>>>>>>> seems
>>>>>>>>>>>>       important to be
>>>>>>>>>>>>       >>>>>> able to cleanly access the individual branches (eg
>>>>>>>>>>>> by mapping
>>>>>>>>>>>>       >>>>>> name->stream), which I thought was the original
>>>>>>>>>>>> intention of
>>>>>>>>>>>>       this KIP.
>>>>>>>>>>>>       >>>>>>
>>>>>>>>>>>>       >>>>>> That said, I don't think we should so easily give in
>>>>>>>>>>>> to the
>>>>>>>>>>>>       double brace
>>>>>>>>>>>>       >>>>>> anti-pattern or force ours users into it if at all
>>>>>>>>>>>> possible to
>>>>>>>>>>>>       >>>>> avoid...just
>>>>>>>>>>>>       >>>>>> my two cents.
>>>>>>>>>>>>       >>>>>>
>>>>>>>>>>>>       >>>>>> Cheers,
>>>>>>>>>>>>       >>>>>> Sophie
>>>>>>>>>>>>       >>>>>>
>>>>>>>>>>>>       >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>>>>>>>       >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>>>       <ma...@confluent.io>> wrote:
>>>>>>>>>>>>       >>>>>>
>>>>>>>>>>>>       >>>>>>> I’d like to propose a different way of thinking
>>>>>>>>>>>> about this.
>>>>>>>>>>>>       To me,
>>>>>>>>>>>>       >>>>> there
>>>>>>>>>>>>       >>>>>>> are three problems with the existing branch 
>>>>>>>>>>>> signature:
>>>>>>>>>>>>       >>>>>>>
>>>>>>>>>>>>       >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>>>> raises unsafe
>>>>>>>>>>> type
>>>>>>>>>>>>       >>>>>> warnings.
>>>>>>>>>>>>       >>>>>>> 2. The way in which you use the stream branches is
>>>>>>>>>>>>       positionally coupled
>>>>>>>>>>>>       >>>>>> to
>>>>>>>>>>>>       >>>>>>> the ordering of the conditionals.
>>>>>>>>>>>>       >>>>>>> 3. It is brittle to extend existing branch calls 
>>>>>>>>>>>> with
>>>>>>>>>>>>       additional code
>>>>>>>>>>>>       >>>>>>> paths.
>>>>>>>>>>>>       >>>>>>>
>>>>>>>>>>>>       >>>>>>> Using associative constructs instead of relying on
>>>>>>>>>>>> ordered
>>>>>>>>>>>>       constructs
>>>>>>>>>>>>       >>>>>> would
>>>>>>>>>>>>       >>>>>>> be a stronger approach. Consider a signature that
>>>>>>>>>>>> instead
>>>>>>>>>>>>       looks like
>>>>>>>>>>>>       >>>>>> this:
>>>>>>>>>>>>       >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>>>       Predicate<?
>>>>>>>>>>>>       >>>>>>> super K,? super V>>);
>>>>>>>>>>>>       >>>>>>>
>>>>>>>>>>>>       >>>>>>> Branches are given names in a map, and as a result,
>>>>>>>>>>>> the API
>>>>>>>>>>>>       returns a
>>>>>>>>>>>>       >>>>>>> mapping of names to streams. The ordering of the
>>>>>>>>>>> conditionals is
>>>>>>>>>>>>       >>>>>> maintained
>>>>>>>>>>>>       >>>>>>> because it’s a sorted map. Insert order determines
>>>>>>>>>>>> the order
>>>>>>>>>>> of
>>>>>>>>>>>>       >>>>>> evaluation.
>>>>>>>>>>>>       >>>>>>> This solves problem 1 because there are no more
>>>>>>>>>>>> varargs. It
>>>>>>>>>>>>       solves
>>>>>>>>>>>>       >>>>>> problem
>>>>>>>>>>>>       >>>>>>> 2 because you no longer lean on ordering to 
>>>>>>>>>>>> access the
>>>>>>>>>>>>       branch you’re
>>>>>>>>>>>>       >>>>>>> interested in. It solves problem 3 because you can
>>>>>>>>>>>> introduce
>>>>>>>>>>>>       another
>>>>>>>>>>>>       >>>>>>> conditional by simply attaching another name to the
>>>>>>>>>>>>       structure, rather
>>>>>>>>>>>>       >>>>>> than
>>>>>>>>>>>>       >>>>>>> messing with the existing indices.
>>>>>>>>>>>>       >>>>>>>
>>>>>>>>>>>>       >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>>>> inline is
>>>>>>>>>>>>       historically
>>>>>>>>>>>>       >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>>>>>>>       voluminously, but
>>>>>>>>>>>>       >>>>>>> double brace initialization would clean up the
>>>>>>>>>>>> aesthetics.
>>>>>>>>>>>>       >>>>>>>
>>>>>>>>>>>>       >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>>>       <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>       >>>>> wrote:
>>>>>>>>>>>>       >>>>>>>> Hi Ivan,
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>> Thanks for the update.
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>> FWIW, I agree with Matthias that the current 
>>>>>>>>>>>> "start
>>>>>>>>>>> branching"
>>>>>>>>>>>>       >>>>> operator
>>>>>>>>>>>>       >>>>>>> is
>>>>>>>>>>>>       >>>>>>>> confusing when named the same way as the actual
>>>>>>>>>>>> branches.
>>>>>>>>>>>>       "Split"
>>>>>>>>>>>>       >>>>> seems
>>>>>>>>>>>>       >>>>>>>> like a good name. Alternatively, we can do without
>>>>>>>>>>>> a "start
>>>>>>>>>>>>       >>>>> branching"
>>>>>>>>>>>>       >>>>>>>> operator at all, and just do:
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>> stream
>>>>>>>>>>>>       >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>       >>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>       >>>>>>>>      .defaultBranch();
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>> Tentatively, I think that this branching operation
>>>>>>>>>>>> should be
>>>>>>>>>>>>       >>>>> terminal.
>>>>>>>>>>>>       >>>>>>> That
>>>>>>>>>>>>       >>>>>>>> way, we don't create ambiguity about how to use
>>>>>>>>>>>> it. That
>>>>>>>>>>>>       is, `branch`
>>>>>>>>>>>>       >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>>>       `void`, to
>>>>>>>>>>>>       >>>>>>>> enforce that it comes last, and that there is only
>>>>>>>>>>>> one
>>>>>>>>>>>>       definition of
>>>>>>>>>>>>       >>>>>> the
>>>>>>>>>>>>       >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>>>> warning if
>>>>>>>>>>>>       there's no
>>>>>>>>>>>>       >>>>>>> default,
>>>>>>>>>>>>       >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>>>> exception) if a
>>>>>>>>>>>>       record
>>>>>>>>>>>>       >>>>>> falls
>>>>>>>>>>>>       >>>>>>>> though with no default.
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>> Thoughts?
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>> Thanks,
>>>>>>>>>>>>       >>>>>>>> -John
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>>>>>>>       >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>>>>>>>       >>>>>>>> wrote:
>>>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> this is to make the name similar to String#split
>>>>>>>>>>>>       >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>>>       >>>>>>>>> The intend was to avoid name duplication. The
>>>>>>>>>>>> return type
>>>>>>>>>>>>       should
>>>>>>>>>>>>       >>>>>> _not_
>>>>>>>>>>>>       >>>>>>>>> be an array.
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> The current proposal is
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> stream.branch()
>>>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>       >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>>>>>>>       `branch()` does
>>>>>>>>>>>>       >>>>> not
>>>>>>>>>>>>       >>>>>>>>> take any parameters and has different semantics
>>>>>>>>>>>> than the
>>>>>>>>>>> later
>>>>>>>>>>>>       >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>>>> snippet above,
>>>>>>>>>>> it's
>>>>>>>>>>>>       >>>>> hidden
>>>>>>>>>>>>       >>>>>>>>> that the first call is `KStream#branch()` while
>>>>>>>>>>>> the others
>>>>>>>>>>> are
>>>>>>>>>>>>       >>>>>>>>> `KBranchedStream#branch()` what makes reading the
>>>>>>>>>>>> code
>>>>>>>>>>> harder.
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> Because I suggested to rename `addBranch()` ->
>>>>>>>>>>>> `branch()`,
>>>>>>>>>>>>       I though
>>>>>>>>>>>>       >>>>>> it
>>>>>>>>>>>>       >>>>>>>>> might be better to also rename `KStream#branch()`
>>>>>>>>>>>> to avoid
>>>>>>>>>>> the
>>>>>>>>>>>>       >>>>> naming
>>>>>>>>>>>>       >>>>>>>>> overlap that seems to be confusing. The following
>>>>>>>>>>>> reads
>>>>>>>>>>> much
>>>>>>>>>>>>       >>>>> cleaner
>>>>>>>>>>>>       >>>>>> to
>>>>>>>>>>>>       >>>>>>>> me:
>>>>>>>>>>>>       >>>>>>>>> stream.split()
>>>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>>>       >>>>>>>>>      .defaultBranch();
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> Maybe there is a better alternative to `split()`
>>>>>>>>>>>> though to
>>>>>>>>>>>>       avoid
>>>>>>>>>>>>       >>>>> the
>>>>>>>>>>>>       >>>>>>>>> naming overlap.
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>>>> unfortunately
>>>>>>>>>>> we
>>>>>>>>>>>>       >>>>> cannot
>>>>>>>>>>>>       >>>>>>> have
>>>>>>>>>>>>       >>>>>>>>> a method with such name :-)
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> Bummer. Didn't consider this. Maybe we can still
>>>>>>>>>>>> come up
>>>>>>>>>>>>       with a
>>>>>>>>>>>>       >>>>> short
>>>>>>>>>>>>       >>>>>>>> name?
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> Can you add the interface `KBranchedStream` to
>>>>>>>>>>>> the KIP
>>>>>>>>>>>>       with all
>>>>>>>>>>>>       >>>>> it's
>>>>>>>>>>>>       >>>>>>>>> methods? It will be part of public API and 
>>>>>>>>>>>> should be
>>>>>>>>>>>>       contained in
>>>>>>>>>>>>       >>>>> the
>>>>>>>>>>>>       >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>>>> return type of
>>>>>>>>>>>>       >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>>>       `KBranchedStream#get(int
>>>>>>>>>>>>       >>>>>>> index)
>>>>>>>>>>>>       >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>>>       branched-KStreams. Would
>>>>>>>>>>>>       >>>>>> be
>>>>>>>>>>>>       >>>>>>>>> nice to get your feedback about it. It seems you
>>>>>>>>>>>> suggest
>>>>>>>>>>>>       that users
>>>>>>>>>>>>       >>>>>>>>> would need to write custom utility code
>>>>>>>>>>>> otherwise, to
>>>>>>>>>>>>       access them.
>>>>>>>>>>>>       >>>>> We
>>>>>>>>>>>>       >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>>>> approaches. It
>>>>>>>>>>> feels
>>>>>>>>>>>>       >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>>>> built-in support
>>>>>>>>>>>>       to get
>>>>>>>>>>>>       >>>>> the
>>>>>>>>>>>>       >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>> -Matthias
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>>>       >>>>>>>>>> Hi all!
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> I have updated the KIP-418 according to the new
>>>>>>>>>>>> vision.
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>>>       >>>>>>>>>> I can see your point: this is to make the name
>>>>>>>>>>>> similar to
>>>>>>>>>>>>       >>>>>>> String#split
>>>>>>>>>>>>       >>>>>>>>>> that also returns an array, right? But is it
>>>>>>>>>>>> worth the
>>>>>>>>>>>>       loss of
>>>>>>>>>>>>       >>>>>>>> backwards
>>>>>>>>>>>>       >>>>>>>>>> compatibility? We can have overloaded branch()
>>>>>>>>>>>> as well
>>>>>>>>>>>>       without
>>>>>>>>>>>>       >>>>>>>> affecting
>>>>>>>>>>>>       >>>>>>>>>> the existing code. Maybe the old array-based
>>>>>>>>>>>> `branch`
>>>>>>>>>>> method
>>>>>>>>>>>>       >>>>> should
>>>>>>>>>>>>       >>>>>>> be
>>>>>>>>>>>>       >>>>>>>>>> deprecated, but this is a subject for 
>>>>>>>>>>>> discussion.
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>>>       >>>>> BranchingKStream#branch(),
>>>>>>>>>>>>       >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> Totally agree with 'addBranch->branch' rename.
>>>>>>>>>>>> 'default'
>>>>>>>>>>> is,
>>>>>>>>>>>>       >>>>>>> however, a
>>>>>>>>>>>>       >>>>>>>>>> reserved word, so unfortunately we cannot have a
>>>>>>>>>>>> method
>>>>>>>>>>>>       with such
>>>>>>>>>>>>       >>>>>>> name
>>>>>>>>>>>>       >>>>>>>>> :-)
>>>>>>>>>>>>       >>>>>>>>>>> defaultBranch() does take an `Predicate` as
>>>>>>>>>>>> argument,
>>>>>>>>>>> but I
>>>>>>>>>>>>       >>>>> think
>>>>>>>>>>>>       >>>>>>> that
>>>>>>>>>>>>       >>>>>>>>>> is not required?
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> Absolutely! I think that was just copy-paste
>>>>>>>>>>>> error or
>>>>>>>>>>>>       something.
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> Dear colleagues,
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> please revise the new version of the KIP and
>>>>>>>>>>>> Paul's PR
>>>>>>>>>>>>       >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> Ivan
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>>>       >>>>>>>>>>> Thanks for driving the discussion of this KIP.
>>>>>>>>>>>> It seems
>>>>>>>>>>> that
>>>>>>>>>>>>       >>>>>>> everybody
>>>>>>>>>>>>       >>>>>>>>>>> agrees that the current branch() method using
>>>>>>>>>>>> arrays is
>>>>>>>>>>> not
>>>>>>>>>>>>       >>>>>> optimal.
>>>>>>>>>>>>       >>>>>>>>>>> I had a quick look into the PR and I like the
>>>>>>>>>>>> overall
>>>>>>>>>>>>       proposal.
>>>>>>>>>>>>       >>>>>>> There
>>>>>>>>>>>>       >>>>>>>>>>> are some minor things we need to consider. I 
>>>>>>>>>>>> would
>>>>>>>>>>>>       recommend the
>>>>>>>>>>>>       >>>>>>>>>>> following renaming:
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>>>       >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>>>       >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>>>       BranchingKStream#default()
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> It's just a suggestion to get slightly shorter
>>>>>>>>>>>> method
>>>>>>>>>>> names.
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>>>>>>>       `Predicate` as
>>>>>>>>>>>>       >>>>>>>> argument,
>>>>>>>>>>>>       >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>>>> recently
>>>>>>>>>>>>       accepted and
>>>>>>>>>>>>       >>>>>> is
>>>>>>>>>>>>       >>>>>>>>>>> currently implemented:
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>
>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL 
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> Ie, we should add overloads that accepted a
>>>>>>>>>>>> `Named`
>>>>>>>>>>>>       parameter.
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> For the issue that the created `KStream` object
>>>>>>>>>>>> are in
>>>>>>>>>>>>       different
>>>>>>>>>>>>       >>>>>>>> scopes:
>>>>>>>>>>>>       >>>>>>>>>>> could we extend `KBranchedStream` with a 
>>>>>>>>>>>> `get(int
>>>>>>>>>>>>       index)` method
>>>>>>>>>>>>       >>>>>>> that
>>>>>>>>>>>>       >>>>>>>>>>> returns the corresponding "branched" result
>>>>>>>>>>>> `KStream`
>>>>>>>>>>>>       object?
>>>>>>>>>>>>       >>>>>> Maybe,
>>>>>>>>>>>>       >>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>> second argument of `addBranch()` should not 
>>>>>>>>>>>> be a
>>>>>>>>>>>>       >>>>>> `Consumer<KStream>`
>>>>>>>>>>>>       >>>>>>>> but
>>>>>>>>>>>>       >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could
>>>>>>>>>>>> return
>>>>>>>>>>>>       whatever
>>>>>>>>>>>>       >>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>> `Function` returns?
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> Finally, I would also suggest to update the KIP
>>>>>>>>>>>> with the
>>>>>>>>>>>>       current
>>>>>>>>>>>>       >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>> -Matthias
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>>       >>>>>>>>>>>> Ivan,
>>>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>>>> think it
>>>>>>>>>>>>       makes sense
>>>>>>>>>>>>       >>>>>> for
>>>>>>>>>>>>       >>>>>>>> you
>>>>>>>>>>>>       >>>>>>>>> to
>>>>>>>>>>>>       >>>>>>>>>>>> revise the KIP and continue the discussion.
>>>>>>>>>>>> Obviously
>>>>>>>>>>>>       we'll
>>>>>>>>>>>>       >>>>> need
>>>>>>>>>>>>       >>>>>>>> some
>>>>>>>>>>>>       >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>>>> binding votes on
>>>>>>>>>>>>       >>>>> whether
>>>>>>>>>>>>       >>>>>>> the
>>>>>>>>>>>>       >>>>>>>>> KIP
>>>>>>>>>>>>       >>>>>>>>>>>> could be adopted.  It would be great to hear
>>>>>>>>>>>> if they
>>>>>>>>>>>>       think this
>>>>>>>>>>>>       >>>>>> is
>>>>>>>>>>>>       >>>>>>> a
>>>>>>>>>>>>       >>>>>>>>> good
>>>>>>>>>>>>       >>>>>>>>>>>> idea overall.  I'm not sure if that happens
>>>>>>>>>>>> just by
>>>>>>>>>>>>       starting a
>>>>>>>>>>>>       >>>>>>> vote,
>>>>>>>>>>>>       >>>>>>>>> or if
>>>>>>>>>>>>       >>>>>>>>>>>> there is generally some indication of interest
>>>>>>>>>>> beforehand.
>>>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>> That being said, I'll continue the discussion
>>>>>>>>>>>> a bit:
>>>>>>>>>>>>       assuming
>>>>>>>>>>>>       >>>>> we
>>>>>>>>>>>>       >>>>>> do
>>>>>>>>>>>>       >>>>>>>>> move
>>>>>>>>>>>>       >>>>>>>>>>>> forward the solution of "stream.branch() 
>>>>>>>>>>>> returns
>>>>>>>>>>>>       >>>>>> KBranchedStream",
>>>>>>>>>>>>       >>>>>>> do
>>>>>>>>>>>>       >>>>>>>>> we
>>>>>>>>>>>>       >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>>>> KStream[]"?  I
>>>>>>>>>>> would
>>>>>>>>>>>>       >>>>> favor
>>>>>>>>>>>>       >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>>>> exclusive APIs
>>>>>>>>>>> that
>>>>>>>>>>>>       >>>>>>> accomplish
>>>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>>>> they're fairly
>>>>>>>>>>>>       similar
>>>>>>>>>>>>       >>>>>>>>> anyway.  We
>>>>>>>>>>>>       >>>>>>>>>>>> just need to be sure we're not making 
>>>>>>>>>>>> something
>>>>>>>>>>>>       >>>>>>> impossible/difficult
>>>>>>>>>>>>       >>>>>>>>> that
>>>>>>>>>>>>       >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>>>> structure would
>>>>>>>>>>> work,
>>>>>>>>>>>>       >>>>> it's
>>>>>>>>>>>>       >>>>>>>> just a
>>>>>>>>>>>>       >>>>>>>>>>>> little sloppy overall in terms of naming and
>>>>>>>>>>>> clarity. In
>>>>>>>>>>>>       >>>>>>> particular,
>>>>>>>>>>>>       >>>>>>>>>>>> passing in the "predicates" and "children"
>>>>>>>>>>>> lists which
>>>>>>>>>>> get
>>>>>>>>>>>>       >>>>>> modified
>>>>>>>>>>>>       >>>>>>>> in
>>>>>>>>>>>>       >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>>>       KStreamLazyBranch is
>>>>>>>>>>>>       >>>>> a
>>>>>>>>>>>>       >>>>>>> bit
>>>>>>>>>>>>       >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>> Thanks,
>>>>>>>>>>>>       >>>>>>>>>>>> Paul
>>>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan 
>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>       >>>>>>>>> wrote:
>>>>>>>>>>>>       >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>>>>>>>       convinced: your
>>>>>>>>>>>>       >>>>>>>> proposal
>>>>>>>>>>>>       >>>>>>>>>>>>> looks better and should work. We just have to
>>>>>>>>>>>> document
>>>>>>>>>>> the
>>>>>>>>>>>>       >>>>>> crucial
>>>>>>>>>>>>       >>>>>>>>> fact
>>>>>>>>>>>>       >>>>>>>>>>>>> that KStream consumers are invoked as they're
>>>>>>>>>>>> added.
>>>>>>>>>>>>       And then
>>>>>>>>>>>>       >>>>>> it's
>>>>>>>>>>>>       >>>>>>>> all
>>>>>>>>>>>>       >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>> What shall we do now? I should re-write the
>>>>>>>>>>>> KIP and
>>>>>>>>>>>>       resume the
>>>>>>>>>>>>       >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>> Why are you telling that your PR 'should not
>>>>>>>>>>>> be even a
>>>>>>>>>>>>       >>>>> starting
>>>>>>>>>>>>       >>>>>>>> point
>>>>>>>>>>>>       >>>>>>>>> if
>>>>>>>>>>>>       >>>>>>>>>>>>> we go in this direction'? To me it looks like
>>>>>>>>>>>> a good
>>>>>>>>>>>>       starting
>>>>>>>>>>>>       >>>>>>> point.
>>>>>>>>>>>>       >>>>>>>>> But
>>>>>>>>>>>>       >>>>>>>>>>>>> as a novice in this project I might miss some
>>>>>>>>>>>> important
>>>>>>>>>>>>       >>>>> details.
>>>>>>>>>>>>       >>>>>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>> Ivan
>>>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>       >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>       >>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>> Maybe I’m missing the point, but I 
>>>>>>>>>>>> believe the
>>>>>>>>>>>>       >>>>> stream.branch()
>>>>>>>>>>>>       >>>>>>>>> solution
>>>>>>>>>>>>       >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>>>> consumers will be
>>>>>>>>>>>>       >>>>> invoked
>>>>>>>>>>>>       >>>>>> as
>>>>>>>>>>>>       >>>>>>>>> they’re
>>>>>>>>>>>>       >>>>>>>>>>>>> added, not during streamsBuilder.build(). So
>>>>>>>>>>>> the user
>>>>>>>>>>>>       still
>>>>>>>>>>>>       >>>>>> ought
>>>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>>>       >>>>>>>>> be
>>>>>>>>>>>>       >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward
>>>>>>>>>>>> and
>>>>>>>>>>>>       depend on
>>>>>>>>>>>>       >>>>> the
>>>>>>>>>>>>       >>>>>>>>> branched
>>>>>>>>>>>>       >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>       >>>>>>>>>>>>>> The issue I mean to point out is that it is
>>>>>>>>>>>> hard to
>>>>>>>>>>>>       access
>>>>>>>>>>>>       >>>>> the
>>>>>>>>>>>>       >>>>>>>>> branched
>>>>>>>>>>>>       >>>>>>>>>>>>> streams in the same scope as the original
>>>>>>>>>>>> stream (that
>>>>>>>>>>>>       is, not
>>>>>>>>>>>>       >>>>>>>> inside
>>>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>> couponIssuer), which is a problem with both
>>>>>>>>>>>> proposed
>>>>>>>>>>>>       >>>>> solutions.
>>>>>>>>>>>>       >>>>>> It
>>>>>>>>>>>>       >>>>>>>>> can be
>>>>>>>>>>>>       >>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>       >>>>>>>>>>>>>> [Also, great to hear additional interest in
>>>>>>>>>>>> 401, I’m
>>>>>>>>>>>>       excited
>>>>>>>>>>>>       >>>>> to
>>>>>>>>>>>>       >>>>>>>> hear
>>>>>>>>>>>>       >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>       >>>>>>>>>>>>>> Paul
>>>>>>>>>>>>       >>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan 
>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>       >>>>>>>>> wrote:
>>>>>>>>>>>>       >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> The idea to postpone the wiring of branches
>>>>>>>>>>>> to the
>>>>>>>>>>>>       >>>>>>>>>>>>> streamsBuilder.build() also looked great for
>>>>>>>>>>>> me at
>>>>>>>>>>> first
>>>>>>>>>>>>       >>>>> glance,
>>>>>>>>>>>>       >>>>>>> but
>>>>>>>>>>>>       >>>>>>>>> ---
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>>>> available in the
>>>>>>>>>>>>       same
>>>>>>>>>>>>       >>>>>> scope
>>>>>>>>>>>>       >>>>>>> as
>>>>>>>>>>>>       >>>>>>>>> each
>>>>>>>>>>>>       >>>>>>>>>>>>> other.  That is, if we wanted to merge 
>>>>>>>>>>>> them back
>>>>>>>>>>> together
>>>>>>>>>>>>       >>>>> again
>>>>>>>>>>>>       >>>>>> I
>>>>>>>>>>>>       >>>>>>>>> don't see
>>>>>>>>>>>>       >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>       >>>>>>>>>>>>>>> You just took the words right out of my
>>>>>>>>>>>> mouth, I was
>>>>>>>>>>>>       just
>>>>>>>>>>>>       >>>>>> going
>>>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>>>       >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>       >>>>>>>>>>>>>>> Consider the example from Bill's book, p.
>>>>>>>>>>>> 101: say
>>>>>>>>>>>>       we need
>>>>>>>>>>>>       >>>>> to
>>>>>>>>>>>>       >>>>>>>>> identify
>>>>>>>>>>>>       >>>>>>>>>>>>> customers who have bought coffee and made a
>>>>>>>>>>>> purchase
>>>>>>>>>>>>       in the
>>>>>>>>>>>>       >>>>>>>>> electronics
>>>>>>>>>>>>       >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>       >>>>>>>>>>>>>>> This is the code I usually write under 
>>>>>>>>>>>> these
>>>>>>>>>>>>       circumstances
>>>>>>>>>>>>       >>>>>> using
>>>>>>>>>>>>       >>>>>>>> my
>>>>>>>>>>>>       >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>       >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>       >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> 
>>>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>       >>>>>>>>>>>>>>>       return
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>       >>>>>>>>>>>>>>>       /*In the real world the code here 
>>>>>>>>>>>> can be
>>>>>>>>>>>>       complex, so
>>>>>>>>>>>>       >>>>>>>>> creation of
>>>>>>>>>>>>       >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>>>> justified, in
>>>>>>>>>>>>       order to
>>>>>>>>>>>>       >>>>>>>> separate
>>>>>>>>>>>>       >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>       >>>>>>>>>>>>>>>  }
>>>>>>>>>>>>       >>>>>>>>>>>>>>> }
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>>>> CouponIssuer();
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>>>       >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to
>>>>>>>>>>>> wire up
>>>>>>>>>>>>       everything
>>>>>>>>>>>>       >>>>>>>> later,
>>>>>>>>>>>>       >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>       >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>>>>>>>       initialize the
>>>>>>>>>>>>       >>>>>>>>> CouponIssuer
>>>>>>>>>>>>       >>>>>>>>>>>>> we need the terminal operation to be called
>>>>>>>>>>>> before
>>>>>>>>>>>>       >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>       >>>>>>>>>>>>> is called.
>>>>>>>>>>>>       >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>>>> KIP-401 is
>>>>>>>>>>>>       essentially
>>>>>>>>>>>>       >>>>>> the
>>>>>>>>>>>>       >>>>>>>>> next
>>>>>>>>>>>>       >>>>>>>>>>>>> KIP I was going to write here. I have some
>>>>>>>>>>>> thoughts
>>>>>>>>>>>>       based on
>>>>>>>>>>>>       >>>>> my
>>>>>>>>>>>>       >>>>>>>>> experience,
>>>>>>>>>>>>       >>>>>>>>>>>>> so I will join the discussion on KIP-401 
>>>>>>>>>>>> soon.]
>>>>>>>>>>>>       >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>>>> concept of a
>>>>>>>>>>>>       fluent
>>>>>>>>>>>>       >>>>> API
>>>>>>>>>>>>       >>>>>>>> based
>>>>>>>>>>>>       >>>>>>>>>>>>> off of
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>>>       (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>>>       >>>>>> and
>>>>>>>>>>>>       >>>>>>> I
>>>>>>>>>>>>       >>>>>>>>> think
>>>>>>>>>>>>       >>>>>>>>>>>>> I
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>>>> earlier about
>>>>>>>>>>>>       >>>>>>> compatibility
>>>>>>>>>>>>       >>>>>>>>>>>>> issues,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was
>>>>>>>>>>>> unaware
>>>>>>>>>>>>       that Java
>>>>>>>>>>>>       >>>>> is
>>>>>>>>>>>>       >>>>>>>> smart
>>>>>>>>>>>>       >>>>>>>>>>>>> enough to
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    distinguish between a 
>>>>>>>>>>>> branch(varargs...)
>>>>>>>>>>>>       returning one
>>>>>>>>>>>>       >>>>>>> thing
>>>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>>>       >>>>>>>>>>>>> branch()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    with no arguments returning another 
>>>>>>>>>>>> thing.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't
>>>>>>>>>>>> actually
>>>>>>>>>>>>       need
>>>>>>>>>>>>       >>>>> it.
>>>>>>>>>>>>       >>>>>>> We
>>>>>>>>>>>>       >>>>>>>>> can
>>>>>>>>>>>>       >>>>>>>>>>>>> just
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>>>> KBranchedStream who
>>>>>>>>>>>>       shares
>>>>>>>>>>>>       >>>>>> its
>>>>>>>>>>>>       >>>>>>>>> state
>>>>>>>>>>>>       >>>>>>>>>>>>> with the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do
>>>>>>>>>>>> the
>>>>>>>>>>>>       branching.
>>>>>>>>>>>>       >>>>>>> It's
>>>>>>>>>>>>       >>>>>>>>> not
>>>>>>>>>>>>       >>>>>>>>>>>>> terribly
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>    pretty in its current form, but I 
>>>>>>>>>>>> think it
>>>>>>>>>>>>       demonstrates
>>>>>>>>>>>>       >>>>>> its
>>>>>>>>>>>>       >>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>>>> request should
>>>>>>>>>>> be
>>>>>>>>>>>>       >>>>> final
>>>>>>>>>>>>       >>>>>> or
>>>>>>>>>>>>       >>>>>>>>> even a
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> starting point if we go in this direction,
>>>>>>>>>>>> I just
>>>>>>>>>>>>       wanted to
>>>>>>>>>>>>       >>>>>> see
>>>>>>>>>>>>       >>>>>>>> how
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> challenging it would be to get the API
>>>>>>>>>>>> working.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> I will say though, that I'm not sure the
>>>>>>>>>>>> existing
>>>>>>>>>>>>       solution
>>>>>>>>>>>>       >>>>>>> could
>>>>>>>>>>>>       >>>>>>>> be
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> deprecated in favor of this, which I had
>>>>>>>>>>>> originally
>>>>>>>>>>>>       >>>>> suggested
>>>>>>>>>>>>       >>>>>>>> was a
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> possibility.  The reason is that the newly
>>>>>>>>>>>> branched
>>>>>>>>>>>>       streams
>>>>>>>>>>>>       >>>>>> are
>>>>>>>>>>>>       >>>>>>>> not
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>>>> other.  That
>>>>>>>>>>>>       is, if we
>>>>>>>>>>>>       >>>>>>> wanted
>>>>>>>>>>>>       >>>>>>>>> to
>>>>>>>>>>>>       >>>>>>>>>>>>> merge
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> them back together again I don't see a way
>>>>>>>>>>>> to do
>>>>>>>>>>>>       that.  The
>>>>>>>>>>>>       >>>>>> KIP
>>>>>>>>>>>>       >>>>>>>>>>>>> proposal
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>>>> means is that
>>>>>>>>>>> for
>>>>>>>>>>>>       >>>>>> either
>>>>>>>>>>>>       >>>>>>>>>>>>> solution,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> deprecating the existing branch(...) is
>>>>>>>>>>>> not on the
>>>>>>>>>>>>       table.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan
>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>>>>>>>       >>>>>>>>>>>>> wrote:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>>>> discussed up to
>>>>>>>>>>> this
>>>>>>>>>>>>       >>>>>> point.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed
>>>>>>>>>>>> that
>>>>>>>>>>>>       branch API
>>>>>>>>>>>>       >>>>>>> needs
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> improvement. Motivation is given in 
>>>>>>>>>>>> the KIP.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...)....
>>>>>>>>>>>> //onTopOf
>>>>>>>>>>>>       returns
>>>>>>>>>>>>       >>>>>> its
>>>>>>>>>>>>       >>>>>>>>> argument
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2)
>>>>>>>>>>>> The code
>>>>>>>>>>> won't
>>>>>>>>>>>>       >>>>> make
>>>>>>>>>>>>       >>>>>>>> sense
>>>>>>>>>>>>       >>>>>>>>>>>>> until
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> all the necessary ingredients are 
>>>>>>>>>>>> provided.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>       instance
>>>>>>>>>>>>       >>>>>>>>> contrasts the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>>>> noDefault(). Both
>>>>>>>>>>>>       >>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>       >>>>>>>>>>>>> and
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams
>>>>>>>>>>>> interface
>>>>>>>>>>> is
>>>>>>>>>>>>       >>>>>> defined.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: We need to define two terminal 
>>>>>>>>>>>> methods
>>>>>>>>>>>>       >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very
>>>>>>>>>>>> easy to
>>>>>>>>>>>>       miss the
>>>>>>>>>>>>       >>>>>> fact
>>>>>>>>>>>>       >>>>>>>>> that one
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> of the terminal methods should be called.
>>>>>>>>>>>> If these
>>>>>>>>>>>>       methods
>>>>>>>>>>>>       >>>>>> are
>>>>>>>>>>>>       >>>>>>>> not
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>>>> runtime.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can
>>>>>>>>>>>> we do
>>>>>>>>>>> better?
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> I see your point when you are talking
>>>>>>>>>>>> about
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot 
>>>>>>>>>>>> not be
>>>>>>>>>>>>       implemented the
>>>>>>>>>>>>       >>>>>>> easy
>>>>>>>>>>>>       >>>>>>>>> way.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> user could specify a terminal 
>>>>>>>>>>>> method that
>>>>>>>>>>> assumes
>>>>>>>>>>>>       >>>>> nothing
>>>>>>>>>>>>       >>>>>>>> will
>>>>>>>>>>>>       >>>>>>>>>>>>> reach
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>>>>> occurs.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be
>>>>>>>>>>>> the only
>>>>>>>>>>> option
>>>>>>>>>>>>       >>>>>> besides
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios
>>>>>>>>>>>> when we
>>>>>>>>>>>>       want to
>>>>>>>>>>>>       >>>>>> just
>>>>>>>>>>>>       >>>>>>>>> silently
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>>>>>>>       predicate. 2)
>>>>>>>>>>>>       >>>>>>> Throwing
>>>>>>>>>>>>       >>>>>>>>> an
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> exception in the middle of data flow
>>>>>>>>>>>> processing
>>>>>>>>>>>>       looks
>>>>>>>>>>>>       >>>>>> like a
>>>>>>>>>>>>       >>>>>>>> bad
>>>>>>>>>>>>       >>>>>>>>>>>>> idea.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would
>>>>>>>>>>>> prefer to
>>>>>>>>>>>>       emit a
>>>>>>>>>>>>       >>>>>>>> special
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is
>>>>>>>>>>>> exactly
>>>>>>>>>>> where
>>>>>>>>>>>>       >>>>>>> `default`
>>>>>>>>>>>>       >>>>>>>>> can
>>>>>>>>>>>>       >>>>>>>>>>>>> be
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>       >>>>> InternalTopologyBuilder
>>>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>>>       >>>>>>>>> track
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>>>>> and raise
>>>>>>>>>>>>       a clear
>>>>>>>>>>>>       >>>>>>> error
>>>>>>>>>>>>       >>>>>>>>>>>>> before it
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the
>>>>>>>>>>>> program is
>>>>>>>>>>>>       >>>>> compiled
>>>>>>>>>>>>       >>>>>>> and
>>>>>>>>>>>>       >>>>>>>>> run?
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply 
>>>>>>>>>>>> won't
>>>>>>>>>>>>       compile if
>>>>>>>>>>>>       >>>>> used
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an 
>>>>>>>>>>>> API as a
>>>>>>>>>>>>       method chain
>>>>>>>>>>>>       >>>>>>>> starting
>>>>>>>>>>>>       >>>>>>>>>>>>> from
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost
>>>>>>>>>>>> difference
>>>>>>>>>>>>       between
>>>>>>>>>>>>       >>>>>>>> runtime
>>>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure
>>>>>>>>>>>> uncovers
>>>>>>>>>>>>       >>>>> instantly
>>>>>>>>>>>>       >>>>>> on
>>>>>>>>>>>>       >>>>>>>>> unit
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> tests, it costs more for the project
>>>>>>>>>>>> than a
>>>>>>>>>>>>       compilation
>>>>>>>>>>>>       >>>>>>>> failure.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>>>> operation being
>>>>>>>>>>>>       required.
>>>>>>>>>>>>       >>>>>>> But
>>>>>>>>>>>>       >>>>>>>> is
>>>>>>>>>>>>       >>>>>>>>>>>>> that
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't
>>>>>>>>>>>> want a
>>>>>>>>>>>>       >>>>>> defaultBranch
>>>>>>>>>>>>       >>>>>>>>> they
>>>>>>>>>>>>       >>>>>>>>>>>>> can
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>>>       just as
>>>>>>>>>>>>       >>>>>>>>> easily.  In
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a
>>>>>>>>>>>> nicer API
>>>>>>>>>>> - a
>>>>>>>>>>>>       >>>>> user
>>>>>>>>>>>>       >>>>>>>> could
>>>>>>>>>>>>       >>>>>>>>>>>>> specify
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing
>>>>>>>>>>>> will reach
>>>>>>>>>>> the
>>>>>>>>>>>>       >>>>>> default
>>>>>>>>>>>>       >>>>>>>>> branch,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>>>>> occurs.
>>>>>>>>>>> That
>>>>>>>>>>>>       >>>>> seems
>>>>>>>>>>>>       >>>>>>> like
>>>>>>>>>>>>       >>>>>>>>> an
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> improvement over the current branch()
>>>>>>>>>>>> API,
>>>>>>>>>>>>       which allows
>>>>>>>>>>>>       >>>>>> for
>>>>>>>>>>>>       >>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>> more
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly 
>>>>>>>>>>>> getting
>>>>>>>>>>> dropped.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>>>> certainly has
>>>>>>>>>>>>       to be
>>>>>>>>>>>>       >>>>>> well
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>>>       >>>>> InternalTopologyBuilder
>>>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>>>       >>>>>>>>> track
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>>>>> and raise
>>>>>>>>>>>>       a clear
>>>>>>>>>>>>       >>>>>>> error
>>>>>>>>>>>>       >>>>>>>>>>>>> before it
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that
>>>>>>>>>>>> there is
>>>>>>>>>>> a
>>>>>>>>>>>>       >>>>> "build
>>>>>>>>>>>>       >>>>>>>> step"
>>>>>>>>>>>>       >>>>>>>>>>>>> where
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>>>>>>>       >>>>>> StreamsBuilder.build()
>>>>>>>>>>>>       >>>>>>> is
>>>>>>>>>>>>       >>>>>>>>>>>>> called.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>>>> argument, I
>>>>>>>>>>> agree
>>>>>>>>>>>>       >>>>> that
>>>>>>>>>>>>       >>>>>>> it's
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> allow users to do other operations on
>>>>>>>>>>>> the input
>>>>>>>>>>>>       stream.
>>>>>>>>>>>>       >>>>>>> With
>>>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same
>>>>>>>>>>>> way all
>>>>>>>>>>> other
>>>>>>>>>>>>       >>>>>>> operations
>>>>>>>>>>>>       >>>>>>>>> do -
>>>>>>>>>>>>       >>>>>>>>>>>>> if
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> want to process off the original 
>>>>>>>>>>>> KStream
>>>>>>>>>>> multiple
>>>>>>>>>>>>       >>>>> times,
>>>>>>>>>>>>       >>>>>>> you
>>>>>>>>>>>>       >>>>>>>>> just
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call
>>>>>>>>>>>> as many
>>>>>>>>>>>>       operations
>>>>>>>>>>>>       >>>>>> on
>>>>>>>>>>>>       >>>>>>> it
>>>>>>>>>>>>       >>>>>>>>> as
>>>>>>>>>>>>       >>>>>>>>>>>>> you
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan
>>>>>>>>>>>> Ponomarev <
>>>>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we
>>>>>>>>>>>> do not
>>>>>>>>>>>>       always need
>>>>>>>>>>>>       >>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>>>>>>>> operation we
>>>>>>>>>>>>       >>>>> don't
>>>>>>>>>>>>       >>>>>>>> know
>>>>>>>>>>>>       >>>>>>>>>>>>> when to
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch 
>>>>>>>>>>>> switch'.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its
>>>>>>>>>>>> argument,
>>>>>>>>>>>>       so we
>>>>>>>>>>>>       >>>>> can
>>>>>>>>>>>>       >>>>>> do
>>>>>>>>>>>>       >>>>>>>>>>>>> something
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> more with the original branch after
>>>>>>>>>>>> branching.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I understand your point that the 
>>>>>>>>>>>> need of
>>>>>>>>>>> special
>>>>>>>>>>>>       >>>>> object
>>>>>>>>>>>>       >>>>>>>>>>>>> construction
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream
>>>>>>>>>>>> methods.
>>>>>>>>>>> But
>>>>>>>>>>>>       >>>>> here
>>>>>>>>>>>>       >>>>>> we
>>>>>>>>>>>>       >>>>>>>>> have a
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to
>>>>>>>>>>>> split the
>>>>>>>>>>>>       flow,
>>>>>>>>>>>>       >>>>> so
>>>>>>>>>>>>       >>>>>> I
>>>>>>>>>>>>       >>>>>>>>> think
>>>>>>>>>>>>       >>>>>>>>>>>>> this
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve
>>>>>>>>>>>> this
>>>>>>>>>>>>       API, but I
>>>>>>>>>>>>       >>>>>> find
>>>>>>>>>>>>       >>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing 
>>>>>>>>>>>> since it
>>>>>>>>>>>>       contrasts the
>>>>>>>>>>>>       >>>>>>> fluency
>>>>>>>>>>>>       >>>>>>>>> of
>>>>>>>>>>>>       >>>>>>>>>>>>> other
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd
>>>>>>>>>>>> like to
>>>>>>>>>>>>       just call
>>>>>>>>>>>>       >>>>> a
>>>>>>>>>>>>       >>>>>>>>> method on
>>>>>>>>>>>>       >>>>>>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if
>>>>>>>>>>>> the branch
>>>>>>>>>>>>       cases
>>>>>>>>>>>>       >>>>> are
>>>>>>>>>>>>       >>>>>>>>> defined
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate,
>>>>>>>>>>>> handleCase)
>>>>>>>>>>>>       is very
>>>>>>>>>>>>       >>>>>> nice
>>>>>>>>>>>>       >>>>>>>>> and the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped
>>>>>>>>>>>> around
>>>>>>>>>>>>       how we
>>>>>>>>>>>>       >>>>>>> specify
>>>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>> source
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>>>>>>>> this::handle1)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>>>>>>>> this::handle2)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>>>       >>>>>>>> KStreamBrancher
>>>>>>>>>>>>       >>>>>>>>> or
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>>>>>>>       terminated by
>>>>>>>>>>>>       >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>>>>>>>       incompatible with
>>>>>>>>>>>>       >>>>> the
>>>>>>>>>>>>       >>>>>>>>> current
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to
>>>>>>>>>>>> have a
>>>>>>>>>>>>       different
>>>>>>>>>>>>       >>>>>> name,
>>>>>>>>>>>>       >>>>>>>> but
>>>>>>>>>>>>       >>>>>>>>> that
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we
>>>>>>>>>>>> could call it
>>>>>>>>>>>>       >>>>>> something
>>>>>>>>>>>>       >>>>>>>> like
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the
>>>>>>>>>>>> old API.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of
>>>>>>>>>>>> your
>>>>>>>>>>>>       KIP?  It
>>>>>>>>>>>>       >>>>>> seems
>>>>>>>>>>>>       >>>>>>>>> like it
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>>>> branching
>>>>>>>>>>>>       while also
>>>>>>>>>>>>       >>>>>>>> allowing
>>>>>>>>>>>>       >>>>>>>>> you
>>>>>>>>>>>>       >>>>>>>>>>>>> to
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>>>>>>>       KBranchedStreams
>>>>>>>>>>>>       >>>>>> if
>>>>>>>>>>>>       >>>>>>>>> desired.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan
>>>>>>>>>>>> Ponomarev
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>>>       ks){
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void 
>>>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>>>       String> ks){
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String,
>>>>>>>>>>>> String>()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>>>       this::handleFirstCase)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>>>       this::handleSecondCase)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>>>       >>>>> takes a
>>>>>>>>>>>>       >>>>>>>>> Consumer
>>>>>>>>>>>>       >>>>>>>>>>>>> as a
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing,
>>>>>>>>>>>> and the
>>>>>>>>>>>>       example in
>>>>>>>>>>>>       >>>>>> the
>>>>>>>>>>>>       >>>>>>>> KIP
>>>>>>>>>>>>       >>>>>>>>>>>>> shows
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a
>>>>>>>>>>>> terminal node
>>>>>>>>>>>>       >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but
>>>>>>>>>>>> how would
>>>>>>>>>>> we
>>>>>>>>>>>>       >>>>> handle
>>>>>>>>>>>>       >>>>>>> the
>>>>>>>>>>>>       >>>>>>>>> case
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>>>> wants to
>>>>>>>>>>> continue
>>>>>>>>>>>>       >>>>>>>> processing
>>>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>>>       >>>>>>>>>>>>> not
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on
>>>>>>>>>>>> the
>>>>>>>>>>> branched
>>>>>>>>>>>>       >>>>>> stream
>>>>>>>>>>>>       >>>>>>>>>>>>> immediately?
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic
>>>>>>>>>>>> as is if
>>>>>>>>>>>>       we had
>>>>>>>>>>>>       >>>>>>>> something
>>>>>>>>>>>>       >>>>>>>>> like
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] 
>>>>>>>>>>>> branches =
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM
>>>>>>>>>>>> Bill Bejeck
>>>>>>>>>>> <
>>>>>>>>>>>>       >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>>>> discussion for
>>>>>>>>>>> KIP-
>>>>>>>>>>>>       >>>>> 418.
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion 
>>>>>>>>>>>> about
>>>>>>>>>>> KIP-418.
>>>>>>>>>>>>       >>>>> Please
>>>>>>>>>>>>       >>>>>>>> take
>>>>>>>>>>>>       >>>>>>>>> a
>>>>>>>>>>>>       >>>>>>>>>>>>> look
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>>>> appreciate any
>>>>>>>>>>>>       feedback :)
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>>       >>>>>
>>>>>>>>>>>>
>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>       >>>>>>>>>>>>> 
>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>>>       >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>>>       >
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>>
>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <vv...@apache.org>.
Hi Ivan,

It looks like you missed my reply on Apr 23rd. I think it’s close, but I had a few last comments. 

Thanks,
John

On Sun, May 3, 2020, at 15:39, Ivan Ponomarev wrote:
> Hello everyone,
> 
> will someone please take a look at the reworked KIP?
> 
> I believe that now it follows design principles and takes into account 
> all the arguments discussed here.
> 
> 
> Regards,
> 
> Ivan
> 
> 
> 23.04.2020 2:45, Ivan Ponomarev пишет:
> > Hi,
> > 
> > I have read the John's "DSL design principles" and have completely 
> > rewritten the KIP, see 
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
> > 
> > 
> > 
> > This version includes all the previous discussion results and follows 
> > the design principles, with one exception.
> > 
> > The exception is
> > 
> > branch(Predicate<K,V> predicate, Branched<K,V> branched)
> > 
> > which formally violates 'no more than one parameter' rule, but I think 
> > here it is justified.
> > 
> > We must provide a predicate for a branch and don't need to provide one 
> > for the default branch. Thus for both operations we may use a single 
> > Branched parameter class, with an extra method parameter for `branch`.
> > 
> > Since predicate is a natural, necessary part of a branch, no 
> > 'proliferation of overloads, deprecations, etc.' is expected here as it 
> > is said in the rationale for the 'single parameter rule'.
> > 
> > WDYT, is this KIP mature enough to begin voting?
> > 
> > Regards,
> > 
> > Ivan
> > 
> > 21.04.2020 2:09, Matthias J. Sax пишет:
> >> Ivan,
> >>
> >> no worries about getting side tracked. Glad to have you back!
> >>
> >> The DSL improved further in the meantime and we already have a `Named`
> >> config object to name operators. It seems reasonable to me to build on 
> >> this.
> >>
> >> Furthermore, John did a writeup about "DSL design principles" that we
> >> want to follow:
> >> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar 
> >>
> >> -- might be worth to checkout.
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> >>> Hi everyone!
> >>>
> >>> Let me revive the discussion of this KIP.
> >>>
> >>> I'm very sorry for stopping my participation in the discussion in June
> >>> 2019. My project work was very intensive then and it didn't leave me
> >>> spare time. But I think I must finish this, because we invested
> >>> substantial effort into this discussion and I'm not feel entitled to
> >>> propose other things before this one is finalized.
> >>>
> >>> During these months I proceeded with writing and reviewing Kafka
> >>> Streams-related code. Every time I needed branching, Spring-Kafka's
> >>> KafkaStreamBrancher class of my invention (the original idea for this
> >>> KIP) worked for me -- that's another reason why I gave up pushing the
> >>> KIP forward. When I was coming across the problem with the scope of
> >>> branches, I worked around it this way:
> >>>
> >>> AtomicReference<KStream<...>> result = new AtomicReference<>();
> >>> new KafkaStreamBrancher<....>()
> >>>      .branch(....)
> >>>      .defaultBranch(result::set)
> >>>      .onTopOf(someStream);
> >>> result.get()...
> >>>
> >>>
> >>> And yes, of course I don't feel very happy with this approach.
> >>>
> >>> I think that Matthias came up with a bright solution in his post from
> >>> May, 24th 2019. Let me quote it:
> >>>
> >>> KStream#split() -> KBranchedStream
> >>> // branch is not easily accessible in current scope
> >>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>    -> KBranchedStream
> >>> // assign a name to the branch and
> >>> // return the sub-stream to the current scope later
> >>> //
> >>> // can be simple as `#branch(p, s->s, "name")`
> >>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >>>    -> KBranchedStream
> >>> // default branch is not easily accessible
> >>> // return map of all named sub-stream into current scope
> >>> KBranchedStream#default(Cosumer<KStream>)
> >>>    -> Map<String,KStream>
> >>> // assign custom name to default-branch
> >>> // return map of all named sub-stream into current scope
> >>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>    -> Map<String,KStream>
> >>> // assign a default name for default
> >>> // return map of all named sub-stream into current scope
> >>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>    -> Map<String,KStream>
> >>> // return map of all names sub-stream into current scope
> >>> KBranchedStream#noDefaultBranch()
> >>>    -> Map<String,KStream>
> >>>
> >>> I believe this would satisfy everyone. Optional names seems to be a good
> >>> idea: when you don't need to have the branches in the same scope, you
> >>> just don't use names and you don't risk making your code brittle. Or,
> >>> you might want to add names just for debugging purposes. Or, finally,
> >>> you might use the returned Map to have the named branches in the
> >>> original scope.
> >>>
> >>> There also was an input from John Roesler on June 4th, 2019, who
> >>> suggested using Named class. I can't comment on this. The idea seems
> >>> reasonable, but in this matter I'd rather trust people who are more
> >>> familiar with Streams API design principles than me.
> >>>
> >>> Regards,
> >>>
> >>> Ivan
> >>>
> >>>
> >>>
> >>> 08.10.2019 1:38, Matthias J. Sax пишет:
> >>>> I am moving this KIP into "inactive status". Feel free to resume the 
> >>>> KIP
> >>>> at any point.
> >>>>
> >>>> If anybody else is interested in picking up this KIP, feel free to 
> >>>> do so.
> >>>>
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> >>>>> Ivan,
> >>>>>
> >>>>> did you see my last reply? What do you think about my proposal to mix
> >>>>> both approaches and try to get best-of-both worlds?
> >>>>>
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> >>>>>> Thanks for the input John!
> >>>>>>
> >>>>>>> under your suggestion, it seems that the name is required
> >>>>>>
> >>>>>> If you want to get the `KStream` as part of the `Map` back using a
> >>>>>> `Function`, yes. If you follow the "embedded chaining" pattern 
> >>>>>> using a
> >>>>>> `Consumer`, no.
> >>>>>>
> >>>>>> Allowing for a default name via `split()` can of course be done.
> >>>>>> Similarly, using `Named` instead of `String` is possible.
> >>>>>>
> >>>>>> I wanted to sketch out a high level proposal to merge both patterns
> >>>>>> only. Your suggestions to align the new API with the existing API 
> >>>>>> make
> >>>>>> totally sense.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> One follow up question: Would `Named` be optional or required in
> >>>>>> `split()` and `branch()`? It's unclear from your example.
> >>>>>>
> >>>>>> If both are mandatory, what do we gain by it? The returned `Map` only
> >>>>>> contains the corresponding branches, so why should we prefix all of
> >>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
> >>>>>> `split()`, the same question raises?
> >>>>>>
> >>>>>> Requiring `Named` in `split()` seems only to make sense, if 
> >>>>>> `Named` is
> >>>>>> optional in `branch()` and we generate `-X` suffix using a counter 
> >>>>>> for
> >>>>>> different branch name. However, this might lead to the problem of
> >>>>>> changing names if branches are added/removed. Also, how would the 
> >>>>>> names
> >>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
> >>>>>> returned in the `Map`).
> >>>>>>
> >>>>>> If `Named` is optional for both, it could happen that a user 
> >>>>>> misses to
> >>>>>> specify a name for a branch what would lead to runtime issues.
> >>>>>>
> >>>>>>
> >>>>>> Hence, I am actually in favor to not allow a default name but keep
> >>>>>> `split()` without parameter and make `Named` in `branch()` required
> >>>>>> if a
> >>>>>> `Function` is used. This makes it explicit to the user that
> >>>>>> specifying a
> >>>>>> name is required if a `Function` is used.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> About
> >>>>>>
> >>>>>>> KBranchedStream#branch(BranchConfig)
> >>>>>>
> >>>>>> I don't think that the branching predicate is a configuration and 
> >>>>>> hence
> >>>>>> would not include it in a configuration object.
> >>>>>>
> >>>>>>>       withChain(...);
> >>>>>>
> >>>>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
> >>>>>> seem to be a configuration. We can also not prevent a user to call
> >>>>>> `withName()` in combination of `withChain()` what does not make sense
> >>>>>> IMHO. We could of course throw an RTE but not have a compile time 
> >>>>>> check
> >>>>>> seems less appealing. Also, it could happen that neither 
> >>>>>> `withChain()`
> >>>>>> not `withName()` is called and the branch is missing in the returned
> >>>>>> `Map` what lead to runtime issues, too.
> >>>>>>
> >>>>>> Hence, I don't think that we should add `BranchConfig`. A config 
> >>>>>> object
> >>>>>> is helpful if each configuration can be set independently of all
> >>>>>> others,
> >>>>>> but this seems not to be the case here. If we add new configuration
> >>>>>> later, we can also just move forward by deprecating the methods that
> >>>>>> accept `Named` and add new methods that accepted `BranchConfig` (that
> >>>>>> would of course implement `Named`).
> >>>>>>
> >>>>>>
> >>>>>> Thoughts?
> >>>>>>
> >>>>>>
> >>>>>> @Ivan, what do you think about the general idea to blend the two main
> >>>>>> approaches of returning a `Map` plus an "embedded chaining"?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
> >>>>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
> >>>>>>> everyone. Returning the map from the terminal operations also solves
> >>>>>>> the problem of merging/joining the branched streams, if we want 
> >>>>>>> to add
> >>>>>>> support for the compliment later on.
> >>>>>>>
> >>>>>>> Under your suggestion, it seems that the name is required. 
> >>>>>>> Otherwise,
> >>>>>>> we wouldn't have keys for the map to return. I this this is actually
> >>>>>>> not too bad, since experience has taught us that, although names for
> >>>>>>> operations are not required to define stream processing logic, it 
> >>>>>>> does
> >>>>>>> significantly improve the operational experience when you can map 
> >>>>>>> the
> >>>>>>> topology, logs, metrics, etc. back to the source code. Since you
> >>>>>>> wouldn't (have to) reference the name to chain extra processing onto
> >>>>>>> the branch (thanks to the second argument), you can avoid the
> >>>>>>> "unchecked name" problem that Ivan pointed out.
> >>>>>>>
> >>>>>>> In the current implementation of Branch, you can name the branch
> >>>>>>> operator itself, and then all the branches get index-suffixed names
> >>>>>>> built from the branch operator name. I guess under this proposal, we
> >>>>>>> could naturally append the branch name to the branching operator 
> >>>>>>> name,
> >>>>>>> like this:
> >>>>>>>
> >>>>>>>      stream.split(Named.withName("mysplit")) //creates node 
> >>>>>>> "mysplit"
> >>>>>>>                 .branch(..., ..., "abranch") // creates node
> >>>>>>> "mysplit-abranch"
> >>>>>>>                 .defaultBranch(...) // creates node 
> >>>>>>> "mysplit-default"
> >>>>>>>
> >>>>>>> It does make me wonder about the DSL syntax itself, though.
> >>>>>>>
> >>>>>>> We don't have a defined grammar, so there's plenty of room to debate
> >>>>>>> the "best" syntax in the context of each operation, but in general,
> >>>>>>> the KStream DSL operators follow this pattern:
> >>>>>>>
> >>>>>>>       operator(function, config_object?) OR operator(config_object)
> >>>>>>>
> >>>>>>> where config_object is often just Named in the "function" variant.
> >>>>>>> Even when the config_object isn't a Named, but some other config
> >>>>>>> class, that config class _always_ implements NamedOperation.
> >>>>>>>
> >>>>>>> Here, we're introducing a totally different pattern:
> >>>>>>>
> >>>>>>>     operator(function, function, string)
> >>>>>>>
> >>>>>>> where the string is the name.
> >>>>>>> My first question is whether the name should instead be specified 
> >>>>>>> with
> >>>>>>> the NamedOperation interface.
> >>>>>>>
> >>>>>>> My second question is whether we should just roll all these 
> >>>>>>> arguments
> >>>>>>> up into a config object like:
> >>>>>>>
> >>>>>>>      KBranchedStream#branch(BranchConfig)
> >>>>>>>
> >>>>>>>      interface BranchConfig extends NamedOperation {
> >>>>>>>       withPredicate(...);
> >>>>>>>       withChain(...);
> >>>>>>>       withName(...);
> >>>>>>>     }
> >>>>>>>
> >>>>>>> Although I guess we'd like to call BranchConfig something more like
> >>>>>>> "Branched", even if I don't particularly like that pattern.
> >>>>>>>
> >>>>>>> This makes the source code a little noisier, but it also makes us 
> >>>>>>> more
> >>>>>>> future-proof, as we can deal with a wide range of alternatives 
> >>>>>>> purely
> >>>>>>> in the config interface, and never have to deal with adding 
> >>>>>>> overloads
> >>>>>>> to the KBranchedStream if/when we decide we want the name to be
> >>>>>>> optional, or the KStream->KStream to be optional.
> >>>>>>>
> >>>>>>> WDYT?
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> -John
> >>>>>>>
> >>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> >>>>>>> <mi...@confluent.io> wrote:
> >>>>>>>>
> >>>>>>>> Matthias: I think that's pretty reasonable from my point of view.
> >>>>>>>> Good
> >>>>>>>> suggestion.
> >>>>>>>>
> >>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
> >>>>>>>> <ma...@confluent.io>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Interesting discussion.
> >>>>>>>>>
> >>>>>>>>> I am wondering, if we cannot unify the advantage of both 
> >>>>>>>>> approaches:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> KStream#split() -> KBranchedStream
> >>>>>>>>>
> >>>>>>>>> // branch is not easily accessible in current scope
> >>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
> >>>>>>>>>     -> KBranchedStream
> >>>>>>>>>
> >>>>>>>>> // assign a name to the branch and
> >>>>>>>>> // return the sub-stream to the current scope later
> >>>>>>>>> //
> >>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
> >>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
> >>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, 
> >>>>>>>>> String)
> >>>>>>>>>     -> KBranchedStream
> >>>>>>>>>
> >>>>>>>>> // default branch is not easily accessible
> >>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
> >>>>>>>>>     -> Map<String,KStream>
> >>>>>>>>>
> >>>>>>>>> // assign custom name to default-branch
> >>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
> >>>>>>>>>     -> Map<String,KStream>
> >>>>>>>>>
> >>>>>>>>> // assign a default name for default
> >>>>>>>>> // return map of all named sub-stream into current scope
> >>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >>>>>>>>>     -> Map<String,KStream>
> >>>>>>>>>
> >>>>>>>>> // return map of all names sub-stream into current scope
> >>>>>>>>> KBranchedStream#noDefaultBranch()
> >>>>>>>>>     -> Map<String,KStream>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Hence, for each sub-stream, the user can pick to add a name and
> >>>>>>>>> return
> >>>>>>>>> the branch "result" to the calling scope or not. The
> >>>>>>>>> implementation can
> >>>>>>>>> also check at runtime that all returned names are unique. The
> >>>>>>>>> returned
> >>>>>>>>> Map can be empty and it's also optional to use the Map.
> >>>>>>>>>
> >>>>>>>>> To me, it seems like a good way to get best of both worlds.
> >>>>>>>>>
> >>>>>>>>> Thoughts?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -Matthias
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
> >>>>>>>>>> Ivan,
> >>>>>>>>>>
> >>>>>>>>>> That's a very good point about the "start" operator in the
> >>>>>>>>>> dynamic case.
> >>>>>>>>>> I had no problem with "split()"; I was just questioning the
> >>>>>>>>>> necessity.
> >>>>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
> >>>>>>>>>> "split()" start operator. Thanks!
> >>>>>>>>>>
> >>>>>>>>>> Separately, I'm interested to see where the present discussion
> >>>>>>>>>> leads.
> >>>>>>>>>> I've written enough Javascript code in my life to be 
> >>>>>>>>>> suspicious of
> >>>>>>>>>> nested closures. You have a good point about using method
> >>>>>>>>>> references (or
> >>>>>>>>>> indeed function literals also work). It should be validating
> >>>>>>>>>> that this
> >>>>>>>>>> was also the JS community's first approach to flattening the
> >>>>>>>>>> logic when
> >>>>>>>>>> their nested closure situation got out of hand. Unfortunately, 
> >>>>>>>>>> it's
> >>>>>>>>>> replacing nesting with redirection, both of which disrupt code
> >>>>>>>>>> readability (but in different ways for different reasons). In 
> >>>>>>>>>> other
> >>>>>>>>>> words, I agree that function references is *the* first-order
> >>>>>>>>>> solution if
> >>>>>>>>>> the nested code does indeed become a problem.
> >>>>>>>>>>
> >>>>>>>>>> However, the history of JS also tells us that function
> >>>>>>>>>> references aren't
> >>>>>>>>>> the end of the story either, and you can see that by observing 
> >>>>>>>>>> that
> >>>>>>>>>> there have been two follow-on eras, as they continue trying to
> >>>>>>>>>> cope with
> >>>>>>>>>> the consequences of living in such a callback-heavy language.
> >>>>>>>>>> First, you
> >>>>>>>>>> have Futures/Promises, which essentially let you convert nested
> >>>>>>>>>> code to
> >>>>>>>>>> method-chained code (Observables/FP is a popular variation on
> >>>>>>>>>> this).
> >>>>>>>>>> Most lately, you have async/await, which is an effort to apply
> >>>>>>>>>> language
> >>>>>>>>>> (not just API) syntax to the problem, and offer the "flattest"
> >>>>>>>>>> possible
> >>>>>>>>>> programming style to solve the problem (because you get back to
> >>>>>>>>>> just one
> >>>>>>>>>> code block per functional unit).
> >>>>>>>>>>
> >>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
> >>>>>>>>>> nowhere
> >>>>>>>>>> near as callback heavy as JS, so I don't think we have to take
> >>>>>>>>>> the JS
> >>>>>>>>>> story for granted, but then again, I think we can derive some
> >>>>>>>>>> valuable
> >>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
> >>>>>>>>>> bringing this
> >>>>>>>>>> up to inspire further/deeper discussion. At the same time, just
> >>>>>>>>>> like JS,
> >>>>>>>>>> we can afford to take an iterative approach to the problem.
> >>>>>>>>>>
> >>>>>>>>>> Separately again, I'm interested in the post-branch merge (and
> >>>>>>>>>> I'd also
> >>>>>>>>>> add join) problem that Paul brought up. We can clearly punt on
> >>>>>>>>>> it, by
> >>>>>>>>>> terminating the nested branches with sink operators. But is
> >>>>>>>>>> there a DSL
> >>>>>>>>>> way to do it?
> >>>>>>>>>>
> >>>>>>>>>> Thanks again for your driving this,
> >>>>>>>>>> -John
> >>>>>>>>>>
> >>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
> >>>>>>>>>> <ma...@gmail.com>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>       Ivan, I’ll definitely forfeit my point on the clumsiness of
> >>>>>>>>>> the
> >>>>>>>>>>       branch(predicate, consumer) solution, I don’t see any real
> >>>>>>>>>> drawbacks
> >>>>>>>>>>       for the dynamic case.
> >>>>>>>>>>
> >>>>>>>>>>       IMO the one trade off to consider at this point is the 
> >>>>>>>>>> scope
> >>>>>>>>>>       question. I don’t know if I totally agree that “we rarely
> >>>>>>>>>> need them
> >>>>>>>>>>       in the same scope” since merging the branches back together
> >>>>>>>>>> later
> >>>>>>>>>>       seems like a perfectly plausible use case that can be a lot
> >>>>>>>>>> nicer
> >>>>>>>>>>       when the branched streams are in the same scope. That being
> >>>>>>>>>> said,
> >>>>>>>>>>       for the reasons Ivan listed, I think it is overall the 
> >>>>>>>>>> better
> >>>>>>>>>>       solution - working around the scope thing is easy enough if
> >>>>>>>>>> you need
> >>>>>>>>>>       to.
> >>>>>>>>>>
> >>>>>>>>>>       > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >>>>>>>>>>       <ip...@mail.ru.invalid> wrote:
> >>>>>>>>>>       >
> >>>>>>>>>>       > Hello everyone, thank you all for joining the discussion!
> >>>>>>>>>>       >
> >>>>>>>>>>       > Well, I don't think the idea of named branches, be it a
> >>>>>>>>>>       LinkedHashMap (no other Map will do, because order of
> >>>>>>>>>> definition
> >>>>>>>>>>       matters) or `branch` method  taking name and Consumer 
> >>>>>>>>>> has more
> >>>>>>>>>>       advantages than drawbacks.
> >>>>>>>>>>       >
> >>>>>>>>>>       > In my opinion, the only real positive outcome from 
> >>>>>>>>>> Michael's
> >>>>>>>>>>       proposal is that all the returned branches are in the same
> >>>>>>>>>> scope.
> >>>>>>>>>>       But 1) we rarely need them in the same scope 2) there is a
> >>>>>>>>>>       workaround for the scope problem, described in the KIP.
> >>>>>>>>>>       >
> >>>>>>>>>>       > 'Inlining the complex logic' is not a problem, because we
> >>>>>>>>>> can use
> >>>>>>>>>>       method references instead of lambdas. In real world
> >>>>>>>>>> scenarios you
> >>>>>>>>>>       tend to split the complex logic to methods anyway, so the
> >>>>>>>>>> code is
> >>>>>>>>>>       going to be clean.
> >>>>>>>>>>       >
> >>>>>>>>>>       > The drawbacks are strong. The cohesion between predicates
> >>>>>>>>>> and
> >>>>>>>>>>       handlers is lost. We have to define predicates in one
> >>>>>>>>>> place, and
> >>>>>>>>>>       handlers in another. This opens the door for bugs:
> >>>>>>>>>>       >
> >>>>>>>>>>       > - what if we forget to define a handler for a name? or a
> >>>>>>>>>> name for
> >>>>>>>>>>       a handler?
> >>>>>>>>>>       > - what if we misspell a name?
> >>>>>>>>>>       > - what if we copy-paste and duplicate a name?
> >>>>>>>>>>       >
> >>>>>>>>>>       > What Michael propose would have been totally OK if we had
> >>>>>>>>>> been
> >>>>>>>>>>       writing the API in Lua, Ruby or Python. In those 
> >>>>>>>>>> languages the
> >>>>>>>>>>       "dynamic naming" approach would have looked most concise 
> >>>>>>>>>> and
> >>>>>>>>>>       beautiful. But in Java we expect all the problems 
> >>>>>>>>>> related to
> >>>>>>>>>>       identifiers to be eliminated in compile time.
> >>>>>>>>>>       >
> >>>>>>>>>>       > Do we have to invent duck-typing for the Java API?
> >>>>>>>>>>       >
> >>>>>>>>>>       > And if we do, what advantage are we supposed to get
> >>>>>>>>>> besides having
> >>>>>>>>>>       all the branches in the same scope? Michael, maybe I'm
> >>>>>>>>>> missing your
> >>>>>>>>>>       point?
> >>>>>>>>>>       >
> >>>>>>>>>>       > ---
> >>>>>>>>>>       >
> >>>>>>>>>>       > Earlier in this discussion John Roesler also proposed 
> >>>>>>>>>> to do
> >>>>>>>>>>       without "start branching" operator, and later Paul
> >>>>>>>>>> mentioned that in
> >>>>>>>>>>       the case when we have to add a dynamic number of 
> >>>>>>>>>> branches, the
> >>>>>>>>>>       current KIP is 'clumsier' compared to Michael's 'Map'
> >>>>>>>>>> solution. Let
> >>>>>>>>>>       me address both comments here.
> >>>>>>>>>>       >
> >>>>>>>>>>       > 1) "Start branching" operator (I think that *split* is a
> >>>>>>>>>> good name
> >>>>>>>>>>       for it indeed) is critical when we need to do a dynamic
> >>>>>>>>>> branching,
> >>>>>>>>>>       see example below.
> >>>>>>>>>>       >
> >>>>>>>>>>       > 2) No, dynamic branching in current KIP is not clumsy at
> >>>>>>>>>> all.
> >>>>>>>>>>       Imagine a real-world scenario when you need one branch per
> >>>>>>>>>> enum
> >>>>>>>>>>       value (say, RecordType). You can have something like this:
> >>>>>>>>>>       >
> >>>>>>>>>>       > /*John:if we had to start with stream.branch(...) here,
> >>>>>>>>>> it would
> >>>>>>>>>>       have been much messier.*/
> >>>>>>>>>>       > KBranchedStream branched = stream.split();
> >>>>>>>>>>       >
> >>>>>>>>>>       > /*Not clumsy at all :-)*/
> >>>>>>>>>>       > for (RecordType recordType : RecordType.values())
> >>>>>>>>>>       >             branched = branched.branch((k, v) ->
> >>>>>>>>>> v.getRecType() ==
> >>>>>>>>>>       recordType,
> >>>>>>>>>>       >                     recordType::processRecords);
> >>>>>>>>>>       >
> >>>>>>>>>>       > Regards,
> >>>>>>>>>>       >
> >>>>>>>>>>       > Ivan
> >>>>>>>>>>       >
> >>>>>>>>>>       >
> >>>>>>>>>>       > 02.05.2019 14:40, Matthias J. Sax пишет:
> >>>>>>>>>>       >> I also agree with Michael's observation about the core
> >>>>>>>>>> problem of
> >>>>>>>>>>       >> current `branch()` implementation.
> >>>>>>>>>>       >>
> >>>>>>>>>>       >> However, I also don't like to pass in a clumsy Map
> >>>>>>>>>> object. My
> >>>>>>>>>>       thinking
> >>>>>>>>>>       >> was more aligned with Paul's proposal to just add a name
> >>>>>>>>>> to each
> >>>>>>>>>>       >> `branch()` statement and return a `Map<String,KStream>`.
> >>>>>>>>>>       >>
> >>>>>>>>>>       >> It makes the code easier to read, and also make the
> >>>>>>>>>> order of
> >>>>>>>>>>       >> `Predicates` (that is essential) easier to grasp.
> >>>>>>>>>>       >>
> >>>>>>>>>>       >>>>>> Map<String, KStream<K, V>> branches = stream.split()
> >>>>>>>>>>       >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>       >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>       >>>>>>    .defaultBranch("defaultBranch");
> >>>>>>>>>>       >> An open question is the case for which no
> >>>>>>>>>> defaultBranch() should
> >>>>>>>>> be
> >>>>>>>>>>       >> specified. Atm, `split()` and `branch()` would return
> >>>>>>>>>>       `BranchedKStream`
> >>>>>>>>>>       >> and the call to `defaultBranch()` that returns the 
> >>>>>>>>>> `Map` is
> >>>>>>>>> mandatory
> >>>>>>>>>>       >> (what is not the case atm). Or is this actually not a 
> >>>>>>>>>> real
> >>>>>>>>> problem,
> >>>>>>>>>>       >> because users can just ignore the branch returned by
> >>>>>>>>>>       `defaultBranch()`
> >>>>>>>>>>       >> in the result `Map` ?
> >>>>>>>>>>       >>
> >>>>>>>>>>       >>
> >>>>>>>>>>       >> About "inlining": So far, it seems to be a matter of
> >>>>>>>>>> personal
> >>>>>>>>>>       >> preference. I can see arguments for both, but no "killer
> >>>>>>>>>>       argument" yet
> >>>>>>>>>>       >> that clearly make the case for one or the other.
> >>>>>>>>>>       >>
> >>>>>>>>>>       >>
> >>>>>>>>>>       >> -Matthias
> >>>>>>>>>>       >>
> >>>>>>>>>>       >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>>>>>>>>>       >>> Perhaps inlining is the wrong terminology. It doesn’t
> >>>>>>>>>> require
> >>>>>>>>>>       that a lambda with the full downstream topology be defined
> >>>>>>>>>> inline -
> >>>>>>>>>>       it can be a method reference as with Ivan’s original
> >>>>>>>>>> suggestion.
> >>>>>>>>>>       The advantage of putting the predicate and its downstream
> >>>>>>>>>> logic
> >>>>>>>>>>       (Consumer) together in branch() is that they are required
> >>>>>>>>>> to be near
> >>>>>>>>>>       to each other.
> >>>>>>>>>>       >>>
> >>>>>>>>>>       >>> Ultimately the downstream code has to live somewhere,
> >>>>>>>>>> and deep
> >>>>>>>>>>       branch trees will be hard to read regardless.
> >>>>>>>>>>       >>>
> >>>>>>>>>>       >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >>>>>>>>>>       <michael.drogalis@confluent.io
> >>>>>>>>>>       <ma...@confluent.io>> wrote:
> >>>>>>>>>>       >>>>
> >>>>>>>>>>       >>>> I'm less enthusiastic about inlining the branch logic
> >>>>>>>>>> with its
> >>>>>>>>>>       downstream
> >>>>>>>>>>       >>>> functionality. Programs that have deep branch trees 
> >>>>>>>>>> will
> >>>>>>>>>>       quickly become
> >>>>>>>>>>       >>>> harder to read as a single unit.
> >>>>>>>>>>       >>>>
> >>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >>>>>>>>>>       <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Also +1 on the issues/goals as Michael outlined them,
> >>>>>>>>>> I think
> >>>>>>>>>>       that sets a
> >>>>>>>>>>       >>>>> great framework for the discussion.
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Regarding the SortedMap solution, my understanding is
> >>>>>>>>>> that the
> >>>>>>>>>>       current
> >>>>>>>>>>       >>>>> proposal in the KIP is what is in my PR which
> >>>>>>>>>> (pending naming
> >>>>>>>>>>       decisions) is
> >>>>>>>>>>       >>>>> roughly this:
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> stream.split()
> >>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >>>>>>>>>>       >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Obviously some ordering is necessary, since branching
> >>>>>>>>>> as a
> >>>>>>>>>>       construct
> >>>>>>>>>>       >>>>> doesn't work without it, but this solution seems 
> >>>>>>>>>> like it
> >>>>>>>>>>       provides as much
> >>>>>>>>>>       >>>>> associativity as the SortedMap solution, because each
> >>>>>>>>>> branch()
> >>>>>>>>>>       call
> >>>>>>>>>>       >>>>> directly associates the "conditional" with the "code
> >>>>>>>>>> block."
> >>>>>>>>>>       The value it
> >>>>>>>>>>       >>>>> provides over the KIP solution is the accessing of
> >>>>>>>>>> streams in
> >>>>>>>>>>       the same
> >>>>>>>>>>       >>>>> scope.
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> The KIP solution is less "dynamic" than the SortedMap
> >>>>>>>>>> solution
> >>>>>>>>>>       in the sense
> >>>>>>>>>>       >>>>> that it is slightly clumsier to add a dynamic 
> >>>>>>>>>> number of
> >>>>>>>>>>       branches, but it is
> >>>>>>>>>>       >>>>> certainly possible.  It seems to me like the API
> >>>>>>>>>> should favor
> >>>>>>>>>>       the "static"
> >>>>>>>>>>       >>>>> case anyway, and should make it simple and 
> >>>>>>>>>> readable to
> >>>>>>>>>>       fluently declare and
> >>>>>>>>>>       >>>>> access your branches in-line.  It also makes it
> >>>>>>>>>> impossible to
> >>>>>>>>>>       ignore a
> >>>>>>>>>>       >>>>> branch, and it is possible to build an (almost)
> >>>>>>>>>> identical
> >>>>>>>>>>       SortedMap
> >>>>>>>>>>       >>>>> solution on top of it.
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> I could also see a middle ground where instead of 
> >>>>>>>>>> a raw
> >>>>>>>>>>       SortedMap being
> >>>>>>>>>>       >>>>> taken in, branch() takes a name and not a Consumer.
> >>>>>>>>>> Something
> >>>>>>>>>>       like this:
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Map<String, KStream<K, V>> branches = stream.split()
> >>>>>>>>>>       >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>>>>>       >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>>>>>       >>>>>    .defaultBranch("defaultBranch",
> >>>>>>>>>> Consumer<KStream<K, V>>);
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Pros for that solution:
> >>>>>>>>>>       >>>>> - accessing branched KStreams in same scope
> >>>>>>>>>>       >>>>> - no double brace initialization, hopefully slightly
> >>>>>>>>>> more
> >>>>>>>>>>       readable than
> >>>>>>>>>>       >>>>> SortedMap
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Cons
> >>>>>>>>>>       >>>>> - downstream branch logic cannot be specified inline
> >>>>>>>>>> which
> >>>>>>>>>>       makes it harder
> >>>>>>>>>>       >>>>> to read top to bottom (like existing API and
> >>>>>>>>>> SortedMap, but
> >>>>>>>>>>       unlike the KIP)
> >>>>>>>>>>       >>>>> - you can forget to "handle" one of the branched
> >>>>>>>>>> streams (like
> >>>>>>>>>>       existing
> >>>>>>>>>>       >>>>> API and SortedMap, but unlike the KIP)
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> (KBranchedStreams could even work *both* ways but
> >>>>>>>>>> perhaps
> >>>>>>>>>>       that's overdoing
> >>>>>>>>>>       >>>>> it).
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Overall I'm curious how important it is to be able to
> >>>>>>>>>> easily
> >>>>>>>>>>       access the
> >>>>>>>>>>       >>>>> branched KStream in the same scope as the original.
> >>>>>>>>>> It's
> >>>>>>>>>>       possible that it
> >>>>>>>>>>       >>>>> doesn't need to be handled directly by the API, but
> >>>>>>>>>> instead
> >>>>>>>>>>       left up to the
> >>>>>>>>>>       >>>>> user.  I'm sort of in the middle on it.
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> Paul
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
> >>>>>>>>>>       <sophie@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>       >>>>> wrote:
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>       >>>>>> I'd like to +1 what Michael said about the issues
> >>>>>>>>>> with the
> >>>>>>>>>>       existing
> >>>>>>>>>>       >>>>> branch
> >>>>>>>>>>       >>>>>> method, I agree with what he's outlined and I think
> >>>>>>>>>> we should
> >>>>>>>>>>       proceed by
> >>>>>>>>>>       >>>>>> trying to alleviate these problems. Specifically it
> >>>>>>>>>> seems
> >>>>>>>>>>       important to be
> >>>>>>>>>>       >>>>>> able to cleanly access the individual branches (eg
> >>>>>>>>>> by mapping
> >>>>>>>>>>       >>>>>> name->stream), which I thought was the original
> >>>>>>>>>> intention of
> >>>>>>>>>>       this KIP.
> >>>>>>>>>>       >>>>>>
> >>>>>>>>>>       >>>>>> That said, I don't think we should so easily give in
> >>>>>>>>>> to the
> >>>>>>>>>>       double brace
> >>>>>>>>>>       >>>>>> anti-pattern or force ours users into it if at all
> >>>>>>>>>> possible to
> >>>>>>>>>>       >>>>> avoid...just
> >>>>>>>>>>       >>>>>> my two cents.
> >>>>>>>>>>       >>>>>>
> >>>>>>>>>>       >>>>>> Cheers,
> >>>>>>>>>>       >>>>>> Sophie
> >>>>>>>>>>       >>>>>>
> >>>>>>>>>>       >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
> >>>>>>>>>>       >>>>>> michael.drogalis@confluent.io
> >>>>>>>>>>       <ma...@confluent.io>> wrote:
> >>>>>>>>>>       >>>>>>
> >>>>>>>>>>       >>>>>>> I’d like to propose a different way of thinking
> >>>>>>>>>> about this.
> >>>>>>>>>>       To me,
> >>>>>>>>>>       >>>>> there
> >>>>>>>>>>       >>>>>>> are three problems with the existing branch 
> >>>>>>>>>> signature:
> >>>>>>>>>>       >>>>>>>
> >>>>>>>>>>       >>>>>>> 1. If you use it the way most people do, Java
> >>>>>>>>>> raises unsafe
> >>>>>>>>> type
> >>>>>>>>>>       >>>>>> warnings.
> >>>>>>>>>>       >>>>>>> 2. The way in which you use the stream branches is
> >>>>>>>>>>       positionally coupled
> >>>>>>>>>>       >>>>>> to
> >>>>>>>>>>       >>>>>>> the ordering of the conditionals.
> >>>>>>>>>>       >>>>>>> 3. It is brittle to extend existing branch calls 
> >>>>>>>>>> with
> >>>>>>>>>>       additional code
> >>>>>>>>>>       >>>>>>> paths.
> >>>>>>>>>>       >>>>>>>
> >>>>>>>>>>       >>>>>>> Using associative constructs instead of relying on
> >>>>>>>>>> ordered
> >>>>>>>>>>       constructs
> >>>>>>>>>>       >>>>>> would
> >>>>>>>>>>       >>>>>>> be a stronger approach. Consider a signature that
> >>>>>>>>>> instead
> >>>>>>>>>>       looks like
> >>>>>>>>>>       >>>>>> this:
> >>>>>>>>>>       >>>>>>> Map<String, KStream<K,V>>
> >>>>>>>>>> KStream#branch(SortedMap<String,
> >>>>>>>>>>       Predicate<?
> >>>>>>>>>>       >>>>>>> super K,? super V>>);
> >>>>>>>>>>       >>>>>>>
> >>>>>>>>>>       >>>>>>> Branches are given names in a map, and as a result,
> >>>>>>>>>> the API
> >>>>>>>>>>       returns a
> >>>>>>>>>>       >>>>>>> mapping of names to streams. The ordering of the
> >>>>>>>>> conditionals is
> >>>>>>>>>>       >>>>>> maintained
> >>>>>>>>>>       >>>>>>> because it’s a sorted map. Insert order determines
> >>>>>>>>>> the order
> >>>>>>>>> of
> >>>>>>>>>>       >>>>>> evaluation.
> >>>>>>>>>>       >>>>>>> This solves problem 1 because there are no more
> >>>>>>>>>> varargs. It
> >>>>>>>>>>       solves
> >>>>>>>>>>       >>>>>> problem
> >>>>>>>>>>       >>>>>>> 2 because you no longer lean on ordering to 
> >>>>>>>>>> access the
> >>>>>>>>>>       branch you’re
> >>>>>>>>>>       >>>>>>> interested in. It solves problem 3 because you can
> >>>>>>>>>> introduce
> >>>>>>>>>>       another
> >>>>>>>>>>       >>>>>>> conditional by simply attaching another name to the
> >>>>>>>>>>       structure, rather
> >>>>>>>>>>       >>>>>> than
> >>>>>>>>>>       >>>>>>> messing with the existing indices.
> >>>>>>>>>>       >>>>>>>
> >>>>>>>>>>       >>>>>>> One of the drawbacks is that creating the map
> >>>>>>>>>> inline is
> >>>>>>>>>>       historically
> >>>>>>>>>>       >>>>>>> awkward in Java. I know it’s an anti-pattern to use
> >>>>>>>>>>       voluminously, but
> >>>>>>>>>>       >>>>>>> double brace initialization would clean up the
> >>>>>>>>>> aesthetics.
> >>>>>>>>>>       >>>>>>>
> >>>>>>>>>>       >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >>>>>>>>>>       <john@confluent.io <ma...@confluent.io>>
> >>>>>>>>>>       >>>>> wrote:
> >>>>>>>>>>       >>>>>>>> Hi Ivan,
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>> Thanks for the update.
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>> FWIW, I agree with Matthias that the current 
> >>>>>>>>>> "start
> >>>>>>>>> branching"
> >>>>>>>>>>       >>>>> operator
> >>>>>>>>>>       >>>>>>> is
> >>>>>>>>>>       >>>>>>>> confusing when named the same way as the actual
> >>>>>>>>>> branches.
> >>>>>>>>>>       "Split"
> >>>>>>>>>>       >>>>> seems
> >>>>>>>>>>       >>>>>>>> like a good name. Alternatively, we can do without
> >>>>>>>>>> a "start
> >>>>>>>>>>       >>>>> branching"
> >>>>>>>>>>       >>>>>>>> operator at all, and just do:
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>> stream
> >>>>>>>>>>       >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>       >>>>>>>>      .branch(Predicate)
> >>>>>>>>>>       >>>>>>>>      .defaultBranch();
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>> Tentatively, I think that this branching operation
> >>>>>>>>>> should be
> >>>>>>>>>>       >>>>> terminal.
> >>>>>>>>>>       >>>>>>> That
> >>>>>>>>>>       >>>>>>>> way, we don't create ambiguity about how to use
> >>>>>>>>>> it. That
> >>>>>>>>>>       is, `branch`
> >>>>>>>>>>       >>>>>>>> should return `KBranchedStream`, while
> >>>>>>>>>> `defaultBranch` is
> >>>>>>>>>>       `void`, to
> >>>>>>>>>>       >>>>>>>> enforce that it comes last, and that there is only
> >>>>>>>>>> one
> >>>>>>>>>>       definition of
> >>>>>>>>>>       >>>>>> the
> >>>>>>>>>>       >>>>>>>> default branch. Potentially, we should log a
> >>>>>>>>>> warning if
> >>>>>>>>>>       there's no
> >>>>>>>>>>       >>>>>>> default,
> >>>>>>>>>>       >>>>>>>> and additionally log a warning (or throw an
> >>>>>>>>>> exception) if a
> >>>>>>>>>>       record
> >>>>>>>>>>       >>>>>> falls
> >>>>>>>>>>       >>>>>>>> though with no default.
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>> Thoughts?
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>> Thanks,
> >>>>>>>>>>       >>>>>>>> -John
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
> >>>>>>>>>>       >>>>> matthias@confluent.io <ma...@confluent.io>
> >>>>>>>>>>       >>>>>>>> wrote:
> >>>>>>>>>>       >>>>>>>>
> >>>>>>>>>>       >>>>>>>>> Thanks for updating the KIP and your answers.
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> this is to make the name similar to String#split
> >>>>>>>>>>       >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>>>       >>>>>>>>> The intend was to avoid name duplication. The
> >>>>>>>>>> return type
> >>>>>>>>>>       should
> >>>>>>>>>>       >>>>>> _not_
> >>>>>>>>>>       >>>>>>>>> be an array.
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> The current proposal is
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> stream.branch()
> >>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>       >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> IMHO, this reads a little odd, because the first
> >>>>>>>>>>       `branch()` does
> >>>>>>>>>>       >>>>> not
> >>>>>>>>>>       >>>>>>>>> take any parameters and has different semantics
> >>>>>>>>>> than the
> >>>>>>>>> later
> >>>>>>>>>>       >>>>>>>>> `branch()` calls. Note, that from the code
> >>>>>>>>>> snippet above,
> >>>>>>>>> it's
> >>>>>>>>>>       >>>>> hidden
> >>>>>>>>>>       >>>>>>>>> that the first call is `KStream#branch()` while
> >>>>>>>>>> the others
> >>>>>>>>> are
> >>>>>>>>>>       >>>>>>>>> `KBranchedStream#branch()` what makes reading the
> >>>>>>>>>> code
> >>>>>>>>> harder.
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> Because I suggested to rename `addBranch()` ->
> >>>>>>>>>> `branch()`,
> >>>>>>>>>>       I though
> >>>>>>>>>>       >>>>>> it
> >>>>>>>>>>       >>>>>>>>> might be better to also rename `KStream#branch()`
> >>>>>>>>>> to avoid
> >>>>>>>>> the
> >>>>>>>>>>       >>>>> naming
> >>>>>>>>>>       >>>>>>>>> overlap that seems to be confusing. The following
> >>>>>>>>>> reads
> >>>>>>>>> much
> >>>>>>>>>>       >>>>> cleaner
> >>>>>>>>>>       >>>>>> to
> >>>>>>>>>>       >>>>>>>> me:
> >>>>>>>>>>       >>>>>>>>> stream.split()
> >>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>>       >>>>>>>>>      .defaultBranch();
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> Maybe there is a better alternative to `split()`
> >>>>>>>>>> though to
> >>>>>>>>>>       avoid
> >>>>>>>>>>       >>>>> the
> >>>>>>>>>>       >>>>>>>>> naming overlap.
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> 'default' is, however, a reserved word, so
> >>>>>>>>>> unfortunately
> >>>>>>>>> we
> >>>>>>>>>>       >>>>> cannot
> >>>>>>>>>>       >>>>>>> have
> >>>>>>>>>>       >>>>>>>>> a method with such name :-)
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> Bummer. Didn't consider this. Maybe we can still
> >>>>>>>>>> come up
> >>>>>>>>>>       with a
> >>>>>>>>>>       >>>>> short
> >>>>>>>>>>       >>>>>>>> name?
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> Can you add the interface `KBranchedStream` to
> >>>>>>>>>> the KIP
> >>>>>>>>>>       with all
> >>>>>>>>>>       >>>>> it's
> >>>>>>>>>>       >>>>>>>>> methods? It will be part of public API and 
> >>>>>>>>>> should be
> >>>>>>>>>>       contained in
> >>>>>>>>>>       >>>>> the
> >>>>>>>>>>       >>>>>>>>> KIP. For example, it's unclear atm, what the
> >>>>>>>>>> return type of
> >>>>>>>>>>       >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> You did not comment on the idea to add a
> >>>>>>>>>>       `KBranchedStream#get(int
> >>>>>>>>>>       >>>>>>> index)
> >>>>>>>>>>       >>>>>>>>> -> KStream` method to get the individually
> >>>>>>>>>>       branched-KStreams. Would
> >>>>>>>>>>       >>>>>> be
> >>>>>>>>>>       >>>>>>>>> nice to get your feedback about it. It seems you
> >>>>>>>>>> suggest
> >>>>>>>>>>       that users
> >>>>>>>>>>       >>>>>>>>> would need to write custom utility code
> >>>>>>>>>> otherwise, to
> >>>>>>>>>>       access them.
> >>>>>>>>>>       >>>>> We
> >>>>>>>>>>       >>>>>>>>> should discuss the pros and cons of both
> >>>>>>>>>> approaches. It
> >>>>>>>>> feels
> >>>>>>>>>>       >>>>>>>>> "incomplete" to me atm, if the API has no
> >>>>>>>>>> built-in support
> >>>>>>>>>>       to get
> >>>>>>>>>>       >>>>> the
> >>>>>>>>>>       >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>> -Matthias
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>>       >>>>>>>>>> Hi all!
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> I have updated the KIP-418 according to the new
> >>>>>>>>>> vision.
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>>       >>>>>>>>>> I can see your point: this is to make the name
> >>>>>>>>>> similar to
> >>>>>>>>>>       >>>>>>> String#split
> >>>>>>>>>>       >>>>>>>>>> that also returns an array, right? But is it
> >>>>>>>>>> worth the
> >>>>>>>>>>       loss of
> >>>>>>>>>>       >>>>>>>> backwards
> >>>>>>>>>>       >>>>>>>>>> compatibility? We can have overloaded branch()
> >>>>>>>>>> as well
> >>>>>>>>>>       without
> >>>>>>>>>>       >>>>>>>> affecting
> >>>>>>>>>>       >>>>>>>>>> the existing code. Maybe the old array-based
> >>>>>>>>>> `branch`
> >>>>>>>>> method
> >>>>>>>>>>       >>>>> should
> >>>>>>>>>>       >>>>>>> be
> >>>>>>>>>>       >>>>>>>>>> deprecated, but this is a subject for 
> >>>>>>>>>> discussion.
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>>>>>>>       >>>>> BranchingKStream#branch(),
> >>>>>>>>>>       >>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>> BranchingKStream#default()
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> Totally agree with 'addBranch->branch' rename.
> >>>>>>>>>> 'default'
> >>>>>>>>> is,
> >>>>>>>>>>       >>>>>>> however, a
> >>>>>>>>>>       >>>>>>>>>> reserved word, so unfortunately we cannot have a
> >>>>>>>>>> method
> >>>>>>>>>>       with such
> >>>>>>>>>>       >>>>>>> name
> >>>>>>>>>>       >>>>>>>>> :-)
> >>>>>>>>>>       >>>>>>>>>>> defaultBranch() does take an `Predicate` as
> >>>>>>>>>> argument,
> >>>>>>>>> but I
> >>>>>>>>>>       >>>>> think
> >>>>>>>>>>       >>>>>>> that
> >>>>>>>>>>       >>>>>>>>>> is not required?
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> Absolutely! I think that was just copy-paste
> >>>>>>>>>> error or
> >>>>>>>>>>       something.
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> please revise the new version of the KIP and
> >>>>>>>>>> Paul's PR
> >>>>>>>>>>       >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> Ivan
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>       >>>>>>>>>>> Thanks for driving the discussion of this KIP.
> >>>>>>>>>> It seems
> >>>>>>>>> that
> >>>>>>>>>>       >>>>>>> everybody
> >>>>>>>>>>       >>>>>>>>>>> agrees that the current branch() method using
> >>>>>>>>>> arrays is
> >>>>>>>>> not
> >>>>>>>>>>       >>>>>> optimal.
> >>>>>>>>>>       >>>>>>>>>>> I had a quick look into the PR and I like the
> >>>>>>>>>> overall
> >>>>>>>>>>       proposal.
> >>>>>>>>>>       >>>>>>> There
> >>>>>>>>>>       >>>>>>>>>>> are some minor things we need to consider. I 
> >>>>>>>>>> would
> >>>>>>>>>>       recommend the
> >>>>>>>>>>       >>>>>>>>>>> following renaming:
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>       >>>>>>>>>>> KBranchedStream#addBranch() ->
> >>>>>>>>>> BranchingKStream#branch()
> >>>>>>>>>>       >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >>>>>>>>>>       BranchingKStream#default()
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> It's just a suggestion to get slightly shorter
> >>>>>>>>>> method
> >>>>>>>>> names.
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> In the current PR, defaultBranch() does take an
> >>>>>>>>>>       `Predicate` as
> >>>>>>>>>>       >>>>>>>> argument,
> >>>>>>>>>>       >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> Also, we should consider KIP-307, that was
> >>>>>>>>>> recently
> >>>>>>>>>>       accepted and
> >>>>>>>>>>       >>>>>> is
> >>>>>>>>>>       >>>>>>>>>>> currently implemented:
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>
> >>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL 
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> Ie, we should add overloads that accepted a
> >>>>>>>>>> `Named`
> >>>>>>>>>>       parameter.
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> For the issue that the created `KStream` object
> >>>>>>>>>> are in
> >>>>>>>>>>       different
> >>>>>>>>>>       >>>>>>>> scopes:
> >>>>>>>>>>       >>>>>>>>>>> could we extend `KBranchedStream` with a 
> >>>>>>>>>> `get(int
> >>>>>>>>>>       index)` method
> >>>>>>>>>>       >>>>>>> that
> >>>>>>>>>>       >>>>>>>>>>> returns the corresponding "branched" result
> >>>>>>>>>> `KStream`
> >>>>>>>>>>       object?
> >>>>>>>>>>       >>>>>> Maybe,
> >>>>>>>>>>       >>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>> second argument of `addBranch()` should not 
> >>>>>>>>>> be a
> >>>>>>>>>>       >>>>>> `Consumer<KStream>`
> >>>>>>>>>>       >>>>>>>> but
> >>>>>>>>>>       >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could
> >>>>>>>>>> return
> >>>>>>>>>>       whatever
> >>>>>>>>>>       >>>>>> the
> >>>>>>>>>>       >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> Finally, I would also suggest to update the KIP
> >>>>>>>>>> with the
> >>>>>>>>>>       current
> >>>>>>>>>>       >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>> -Matthias
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>       >>>>>>>>>>>> Ivan,
> >>>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>> I'm a bit of a novice here as well, but I
> >>>>>>>>>> think it
> >>>>>>>>>>       makes sense
> >>>>>>>>>>       >>>>>> for
> >>>>>>>>>>       >>>>>>>> you
> >>>>>>>>>>       >>>>>>>>> to
> >>>>>>>>>>       >>>>>>>>>>>> revise the KIP and continue the discussion.
> >>>>>>>>>> Obviously
> >>>>>>>>>>       we'll
> >>>>>>>>>>       >>>>> need
> >>>>>>>>>>       >>>>>>>> some
> >>>>>>>>>>       >>>>>>>>>>>> buy-in from committers that have actual
> >>>>>>>>>> binding votes on
> >>>>>>>>>>       >>>>> whether
> >>>>>>>>>>       >>>>>>> the
> >>>>>>>>>>       >>>>>>>>> KIP
> >>>>>>>>>>       >>>>>>>>>>>> could be adopted.  It would be great to hear
> >>>>>>>>>> if they
> >>>>>>>>>>       think this
> >>>>>>>>>>       >>>>>> is
> >>>>>>>>>>       >>>>>>> a
> >>>>>>>>>>       >>>>>>>>> good
> >>>>>>>>>>       >>>>>>>>>>>> idea overall.  I'm not sure if that happens
> >>>>>>>>>> just by
> >>>>>>>>>>       starting a
> >>>>>>>>>>       >>>>>>> vote,
> >>>>>>>>>>       >>>>>>>>> or if
> >>>>>>>>>>       >>>>>>>>>>>> there is generally some indication of interest
> >>>>>>>>> beforehand.
> >>>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>> That being said, I'll continue the discussion
> >>>>>>>>>> a bit:
> >>>>>>>>>>       assuming
> >>>>>>>>>>       >>>>> we
> >>>>>>>>>>       >>>>>> do
> >>>>>>>>>>       >>>>>>>>> move
> >>>>>>>>>>       >>>>>>>>>>>> forward the solution of "stream.branch() 
> >>>>>>>>>> returns
> >>>>>>>>>>       >>>>>> KBranchedStream",
> >>>>>>>>>>       >>>>>>> do
> >>>>>>>>>>       >>>>>>>>> we
> >>>>>>>>>>       >>>>>>>>>>>> deprecate "stream.branch(...) returns
> >>>>>>>>>> KStream[]"?  I
> >>>>>>>>> would
> >>>>>>>>>>       >>>>> favor
> >>>>>>>>>>       >>>>>>>>>>>> deprecating, since having two mutually
> >>>>>>>>>> exclusive APIs
> >>>>>>>>> that
> >>>>>>>>>>       >>>>>>> accomplish
> >>>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>> same thing is confusing, especially when
> >>>>>>>>>> they're fairly
> >>>>>>>>>>       similar
> >>>>>>>>>>       >>>>>>>>> anyway.  We
> >>>>>>>>>>       >>>>>>>>>>>> just need to be sure we're not making 
> >>>>>>>>>> something
> >>>>>>>>>>       >>>>>>> impossible/difficult
> >>>>>>>>>>       >>>>>>>>> that
> >>>>>>>>>>       >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>> Regarding my PR - I think the general
> >>>>>>>>>> structure would
> >>>>>>>>> work,
> >>>>>>>>>>       >>>>> it's
> >>>>>>>>>>       >>>>>>>> just a
> >>>>>>>>>>       >>>>>>>>>>>> little sloppy overall in terms of naming and
> >>>>>>>>>> clarity. In
> >>>>>>>>>>       >>>>>>> particular,
> >>>>>>>>>>       >>>>>>>>>>>> passing in the "predicates" and "children"
> >>>>>>>>>> lists which
> >>>>>>>>> get
> >>>>>>>>>>       >>>>>> modified
> >>>>>>>>>>       >>>>>>>> in
> >>>>>>>>>>       >>>>>>>>>>>> KBranchedStream but read from all the way
> >>>>>>>>>>       KStreamLazyBranch is
> >>>>>>>>>>       >>>>> a
> >>>>>>>>>>       >>>>>>> bit
> >>>>>>>>>>       >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>> Thanks,
> >>>>>>>>>>       >>>>>>>>>>>> Paul
> >>>>>>>>>>       >>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan 
> >>>>>>>>>> Ponomarev <
> >>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>       >>>>>>>>> wrote:
> >>>>>>>>>>       >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>> I read your code carefully and now I am fully
> >>>>>>>>>>       convinced: your
> >>>>>>>>>>       >>>>>>>> proposal
> >>>>>>>>>>       >>>>>>>>>>>>> looks better and should work. We just have to
> >>>>>>>>>> document
> >>>>>>>>> the
> >>>>>>>>>>       >>>>>> crucial
> >>>>>>>>>>       >>>>>>>>> fact
> >>>>>>>>>>       >>>>>>>>>>>>> that KStream consumers are invoked as they're
> >>>>>>>>>> added.
> >>>>>>>>>>       And then
> >>>>>>>>>>       >>>>>> it's
> >>>>>>>>>>       >>>>>>>> all
> >>>>>>>>>>       >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>> What shall we do now? I should re-write the
> >>>>>>>>>> KIP and
> >>>>>>>>>>       resume the
> >>>>>>>>>>       >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>> Why are you telling that your PR 'should not
> >>>>>>>>>> be even a
> >>>>>>>>>>       >>>>> starting
> >>>>>>>>>>       >>>>>>>> point
> >>>>>>>>>>       >>>>>>>>> if
> >>>>>>>>>>       >>>>>>>>>>>>> we go in this direction'? To me it looks like
> >>>>>>>>>> a good
> >>>>>>>>>>       starting
> >>>>>>>>>>       >>>>>>> point.
> >>>>>>>>>>       >>>>>>>>> But
> >>>>>>>>>>       >>>>>>>>>>>>> as a novice in this project I might miss some
> >>>>>>>>>> important
> >>>>>>>>>>       >>>>> details.
> >>>>>>>>>>       >>>>>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>> Ivan
> >>>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>       >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>       >>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>> Maybe I’m missing the point, but I 
> >>>>>>>>>> believe the
> >>>>>>>>>>       >>>>> stream.branch()
> >>>>>>>>>>       >>>>>>>>> solution
> >>>>>>>>>>       >>>>>>>>>>>>> supports this. The couponIssuer::set*
> >>>>>>>>>> consumers will be
> >>>>>>>>>>       >>>>> invoked
> >>>>>>>>>>       >>>>>> as
> >>>>>>>>>>       >>>>>>>>> they’re
> >>>>>>>>>>       >>>>>>>>>>>>> added, not during streamsBuilder.build(). So
> >>>>>>>>>> the user
> >>>>>>>>>>       still
> >>>>>>>>>>       >>>>>> ought
> >>>>>>>>>>       >>>>>>> to
> >>>>>>>>>>       >>>>>>>>> be
> >>>>>>>>>>       >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward
> >>>>>>>>>> and
> >>>>>>>>>>       depend on
> >>>>>>>>>>       >>>>> the
> >>>>>>>>>>       >>>>>>>>> branched
> >>>>>>>>>>       >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>       >>>>>>>>>>>>>> The issue I mean to point out is that it is
> >>>>>>>>>> hard to
> >>>>>>>>>>       access
> >>>>>>>>>>       >>>>> the
> >>>>>>>>>>       >>>>>>>>> branched
> >>>>>>>>>>       >>>>>>>>>>>>> streams in the same scope as the original
> >>>>>>>>>> stream (that
> >>>>>>>>>>       is, not
> >>>>>>>>>>       >>>>>>>> inside
> >>>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>> couponIssuer), which is a problem with both
> >>>>>>>>>> proposed
> >>>>>>>>>>       >>>>> solutions.
> >>>>>>>>>>       >>>>>> It
> >>>>>>>>>>       >>>>>>>>> can be
> >>>>>>>>>>       >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>       >>>>>>>>>>>>>> [Also, great to hear additional interest in
> >>>>>>>>>> 401, I’m
> >>>>>>>>>>       excited
> >>>>>>>>>>       >>>>> to
> >>>>>>>>>>       >>>>>>>> hear
> >>>>>>>>>>       >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>       >>>>>>>>>>>>>> Paul
> >>>>>>>>>>       >>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan 
> >>>>>>>>>> Ponomarev <
> >>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>       >>>>>>>>> wrote:
> >>>>>>>>>>       >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> The idea to postpone the wiring of branches
> >>>>>>>>>> to the
> >>>>>>>>>>       >>>>>>>>>>>>> streamsBuilder.build() also looked great for
> >>>>>>>>>> me at
> >>>>>>>>> first
> >>>>>>>>>>       >>>>> glance,
> >>>>>>>>>>       >>>>>>> but
> >>>>>>>>>>       >>>>>>>>> ---
> >>>>>>>>>>       >>>>>>>>>>>>>>>> the newly branched streams are not
> >>>>>>>>>> available in the
> >>>>>>>>>>       same
> >>>>>>>>>>       >>>>>> scope
> >>>>>>>>>>       >>>>>>> as
> >>>>>>>>>>       >>>>>>>>> each
> >>>>>>>>>>       >>>>>>>>>>>>> other.  That is, if we wanted to merge 
> >>>>>>>>>> them back
> >>>>>>>>> together
> >>>>>>>>>>       >>>>> again
> >>>>>>>>>>       >>>>>> I
> >>>>>>>>>>       >>>>>>>>> don't see
> >>>>>>>>>>       >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>       >>>>>>>>>>>>>>> You just took the words right out of my
> >>>>>>>>>> mouth, I was
> >>>>>>>>>>       just
> >>>>>>>>>>       >>>>>> going
> >>>>>>>>>>       >>>>>>> to
> >>>>>>>>>>       >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>       >>>>>>>>>>>>>>> Consider the example from Bill's book, p.
> >>>>>>>>>> 101: say
> >>>>>>>>>>       we need
> >>>>>>>>>>       >>>>> to
> >>>>>>>>>>       >>>>>>>>> identify
> >>>>>>>>>>       >>>>>>>>>>>>> customers who have bought coffee and made a
> >>>>>>>>>> purchase
> >>>>>>>>>>       in the
> >>>>>>>>>>       >>>>>>>>> electronics
> >>>>>>>>>>       >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>       >>>>>>>>>>>>>>> This is the code I usually write under 
> >>>>>>>>>> these
> >>>>>>>>>>       circumstances
> >>>>>>>>>>       >>>>>> using
> >>>>>>>>>>       >>>>>>>> my
> >>>>>>>>>>       >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>       >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>       >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
> >>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> 
> >>>>>>>>>> electronicsPurchases;
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>       >>>>>>>>>>>>>>>       return
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>       >>>>>>>>>>>>>>>       /*In the real world the code here 
> >>>>>>>>>> can be
> >>>>>>>>>>       complex, so
> >>>>>>>>>>       >>>>>>>>> creation of
> >>>>>>>>>>       >>>>>>>>>>>>> a separate CouponIssuer class is fully
> >>>>>>>>>> justified, in
> >>>>>>>>>>       order to
> >>>>>>>>>>       >>>>>>>> separate
> >>>>>>>>>>       >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>       >>>>>>>>>>>>>>>  }
> >>>>>>>>>>       >>>>>>>>>>>>>>> }
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
> >>>>>>>>>> CouponIssuer();
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate1,
> >>>>>>>>> couponIssuer::setCoffePurchases)
> >>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>>>>>>       >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>       >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to
> >>>>>>>>>> wire up
> >>>>>>>>>>       everything
> >>>>>>>>>>       >>>>>>>> later,
> >>>>>>>>>>       >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>       >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> Does this make sense?  In order to properly
> >>>>>>>>>>       initialize the
> >>>>>>>>>>       >>>>>>>>> CouponIssuer
> >>>>>>>>>>       >>>>>>>>>>>>> we need the terminal operation to be called
> >>>>>>>>>> before
> >>>>>>>>>>       >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>       >>>>>>>>>>>>> is called.
> >>>>>>>>>>       >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
> >>>>>>>>>> KIP-401 is
> >>>>>>>>>>       essentially
> >>>>>>>>>>       >>>>>> the
> >>>>>>>>>>       >>>>>>>>> next
> >>>>>>>>>>       >>>>>>>>>>>>> KIP I was going to write here. I have some
> >>>>>>>>>> thoughts
> >>>>>>>>>>       based on
> >>>>>>>>>>       >>>>> my
> >>>>>>>>>>       >>>>>>>>> experience,
> >>>>>>>>>>       >>>>>>>>>>>>> so I will join the discussion on KIP-401 
> >>>>>>>>>> soon.]
> >>>>>>>>>>       >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>       >>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>       >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>       >>>>>>>>>>>>>>>> I tried to make a very rough proof of
> >>>>>>>>>> concept of a
> >>>>>>>>>>       fluent
> >>>>>>>>>>       >>>>> API
> >>>>>>>>>>       >>>>>>>> based
> >>>>>>>>>>       >>>>>>>>>>>>> off of
> >>>>>>>>>>       >>>>>>>>>>>>>>>> KStream here
> >>>>>>>>>>       (https://github.com/apache/kafka/pull/6512),
> >>>>>>>>>>       >>>>>> and
> >>>>>>>>>>       >>>>>>> I
> >>>>>>>>>>       >>>>>>>>> think
> >>>>>>>>>>       >>>>>>>>>>>>> I
> >>>>>>>>>>       >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
> >>>>>>>>>> earlier about
> >>>>>>>>>>       >>>>>>> compatibility
> >>>>>>>>>>       >>>>>>>>>>>>> issues,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was
> >>>>>>>>>> unaware
> >>>>>>>>>>       that Java
> >>>>>>>>>>       >>>>> is
> >>>>>>>>>>       >>>>>>>> smart
> >>>>>>>>>>       >>>>>>>>>>>>> enough to
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    distinguish between a 
> >>>>>>>>>> branch(varargs...)
> >>>>>>>>>>       returning one
> >>>>>>>>>>       >>>>>>> thing
> >>>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>>       >>>>>>>>>>>>> branch()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    with no arguments returning another 
> >>>>>>>>>> thing.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't
> >>>>>>>>>> actually
> >>>>>>>>>>       need
> >>>>>>>>>>       >>>>> it.
> >>>>>>>>>>       >>>>>>> We
> >>>>>>>>>>       >>>>>>>>> can
> >>>>>>>>>>       >>>>>>>>>>>>> just
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    build up the branches in the
> >>>>>>>>>> KBranchedStream who
> >>>>>>>>>>       shares
> >>>>>>>>>>       >>>>>> its
> >>>>>>>>>>       >>>>>>>>> state
> >>>>>>>>>>       >>>>>>>>>>>>> with the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do
> >>>>>>>>>> the
> >>>>>>>>>>       branching.
> >>>>>>>>>>       >>>>>>> It's
> >>>>>>>>>>       >>>>>>>>> not
> >>>>>>>>>>       >>>>>>>>>>>>> terribly
> >>>>>>>>>>       >>>>>>>>>>>>>>>>    pretty in its current form, but I 
> >>>>>>>>>> think it
> >>>>>>>>>>       demonstrates
> >>>>>>>>>>       >>>>>> its
> >>>>>>>>>>       >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>       >>>>>>>>>>>>>>>> To be clear, I don't think that pull
> >>>>>>>>>> request should
> >>>>>>>>> be
> >>>>>>>>>>       >>>>> final
> >>>>>>>>>>       >>>>>> or
> >>>>>>>>>>       >>>>>>>>> even a
> >>>>>>>>>>       >>>>>>>>>>>>>>>> starting point if we go in this direction,
> >>>>>>>>>> I just
> >>>>>>>>>>       wanted to
> >>>>>>>>>>       >>>>>> see
> >>>>>>>>>>       >>>>>>>> how
> >>>>>>>>>>       >>>>>>>>>>>>>>>> challenging it would be to get the API
> >>>>>>>>>> working.
> >>>>>>>>>>       >>>>>>>>>>>>>>>> I will say though, that I'm not sure the
> >>>>>>>>>> existing
> >>>>>>>>>>       solution
> >>>>>>>>>>       >>>>>>> could
> >>>>>>>>>>       >>>>>>>> be
> >>>>>>>>>>       >>>>>>>>>>>>>>>> deprecated in favor of this, which I had
> >>>>>>>>>> originally
> >>>>>>>>>>       >>>>> suggested
> >>>>>>>>>>       >>>>>>>> was a
> >>>>>>>>>>       >>>>>>>>>>>>>>>> possibility.  The reason is that the newly
> >>>>>>>>>> branched
> >>>>>>>>>>       streams
> >>>>>>>>>>       >>>>>> are
> >>>>>>>>>>       >>>>>>>> not
> >>>>>>>>>>       >>>>>>>>>>>>>>>> available in the same scope as each
> >>>>>>>>>> other.  That
> >>>>>>>>>>       is, if we
> >>>>>>>>>>       >>>>>>> wanted
> >>>>>>>>>>       >>>>>>>>> to
> >>>>>>>>>>       >>>>>>>>>>>>> merge
> >>>>>>>>>>       >>>>>>>>>>>>>>>> them back together again I don't see a way
> >>>>>>>>>> to do
> >>>>>>>>>>       that.  The
> >>>>>>>>>>       >>>>>> KIP
> >>>>>>>>>>       >>>>>>>>>>>>> proposal
> >>>>>>>>>>       >>>>>>>>>>>>>>>> has the same issue, though - all this
> >>>>>>>>>> means is that
> >>>>>>>>> for
> >>>>>>>>>>       >>>>>> either
> >>>>>>>>>>       >>>>>>>>>>>>> solution,
> >>>>>>>>>>       >>>>>>>>>>>>>>>> deprecating the existing branch(...) is
> >>>>>>>>>> not on the
> >>>>>>>>>>       table.
> >>>>>>>>>>       >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>       >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan
> >>>>>>>>>> Ponomarev <
> >>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
> >>>>>>>>>>       >>>>>>>>>>>>> wrote:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> OK, let me summarize what we have
> >>>>>>>>>> discussed up to
> >>>>>>>>> this
> >>>>>>>>>>       >>>>>> point.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed
> >>>>>>>>>> that
> >>>>>>>>>>       branch API
> >>>>>>>>>>       >>>>>>> needs
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> improvement. Motivation is given in 
> >>>>>>>>>> the KIP.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> There are two potential ways to do it:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...)....
> >>>>>>>>>> //onTopOf
> >>>>>>>>>>       returns
> >>>>>>>>>>       >>>>>> its
> >>>>>>>>>>       >>>>>>>>> argument
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2)
> >>>>>>>>>> The code
> >>>>>>>>> won't
> >>>>>>>>>>       >>>>> make
> >>>>>>>>>>       >>>>>>>> sense
> >>>>>>>>>>       >>>>>>>>>>>>> until
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> all the necessary ingredients are 
> >>>>>>>>>> provided.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: The need to create a
> >>>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>       instance
> >>>>>>>>>>       >>>>>>>>> contrasts the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
> >>>>>>>>>> noDefault(). Both
> >>>>>>>>>>       >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>       >>>>>>>>>>>>> and
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams
> >>>>>>>>>> interface
> >>>>>>>>> is
> >>>>>>>>>>       >>>>>> defined.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: We need to define two terminal 
> >>>>>>>>>> methods
> >>>>>>>>>>       >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very
> >>>>>>>>>> easy to
> >>>>>>>>>>       miss the
> >>>>>>>>>>       >>>>>> fact
> >>>>>>>>>>       >>>>>>>>> that one
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> of the terminal methods should be called.
> >>>>>>>>>> If these
> >>>>>>>>>>       methods
> >>>>>>>>>>       >>>>>> are
> >>>>>>>>>>       >>>>>>>> not
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> called, we can throw an exception in
> >>>>>>>>>> runtime.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can
> >>>>>>>>>> we do
> >>>>>>>>> better?
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> I see your point when you are talking
> >>>>>>>>>> about
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot 
> >>>>>>>>>> not be
> >>>>>>>>>>       implemented the
> >>>>>>>>>>       >>>>>>> easy
> >>>>>>>>>>       >>>>>>>>> way.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> user could specify a terminal 
> >>>>>>>>>> method that
> >>>>>>>>> assumes
> >>>>>>>>>>       >>>>> nothing
> >>>>>>>>>>       >>>>>>>> will
> >>>>>>>>>>       >>>>>>>>>>>>> reach
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> throwing an exception if such a case
> >>>>>>>>>> occurs.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be
> >>>>>>>>>> the only
> >>>>>>>>> option
> >>>>>>>>>>       >>>>>> besides
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios
> >>>>>>>>>> when we
> >>>>>>>>>>       want to
> >>>>>>>>>>       >>>>>> just
> >>>>>>>>>>       >>>>>>>>> silently
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
> >>>>>>>>>>       predicate. 2)
> >>>>>>>>>>       >>>>>>> Throwing
> >>>>>>>>>>       >>>>>>>>> an
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> exception in the middle of data flow
> >>>>>>>>>> processing
> >>>>>>>>>>       looks
> >>>>>>>>>>       >>>>>> like a
> >>>>>>>>>>       >>>>>>>> bad
> >>>>>>>>>>       >>>>>>>>>>>>> idea.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would
> >>>>>>>>>> prefer to
> >>>>>>>>>>       emit a
> >>>>>>>>>>       >>>>>>>> special
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is
> >>>>>>>>>> exactly
> >>>>>>>>> where
> >>>>>>>>>>       >>>>>>> `default`
> >>>>>>>>>>       >>>>>>>>> can
> >>>>>>>>>>       >>>>>>>>>>>>> be
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>       >>>>> InternalTopologyBuilder
> >>>>>>>>>>       >>>>>>> to
> >>>>>>>>>>       >>>>>>>>> track
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> branches that haven't been terminated
> >>>>>>>>>> and raise
> >>>>>>>>>>       a clear
> >>>>>>>>>>       >>>>>>> error
> >>>>>>>>>>       >>>>>>>>>>>>> before it
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the
> >>>>>>>>>> program is
> >>>>>>>>>>       >>>>> compiled
> >>>>>>>>>>       >>>>>>> and
> >>>>>>>>>>       >>>>>>>>> run?
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply 
> >>>>>>>>>> won't
> >>>>>>>>>>       compile if
> >>>>>>>>>>       >>>>> used
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an 
> >>>>>>>>>> API as a
> >>>>>>>>>>       method chain
> >>>>>>>>>>       >>>>>>>> starting
> >>>>>>>>>>       >>>>>>>>>>>>> from
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost
> >>>>>>>>>> difference
> >>>>>>>>>>       between
> >>>>>>>>>>       >>>>>>>> runtime
> >>>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure
> >>>>>>>>>> uncovers
> >>>>>>>>>>       >>>>> instantly
> >>>>>>>>>>       >>>>>> on
> >>>>>>>>>>       >>>>>>>>> unit
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> tests, it costs more for the project
> >>>>>>>>>> than a
> >>>>>>>>>>       compilation
> >>>>>>>>>>       >>>>>>>> failure.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Good point about the terminal
> >>>>>>>>>> operation being
> >>>>>>>>>>       required.
> >>>>>>>>>>       >>>>>>> But
> >>>>>>>>>>       >>>>>>>> is
> >>>>>>>>>>       >>>>>>>>>>>>> that
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't
> >>>>>>>>>> want a
> >>>>>>>>>>       >>>>>> defaultBranch
> >>>>>>>>>>       >>>>>>>>> they
> >>>>>>>>>>       >>>>>>>>>>>>> can
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> some other terminal method
> >>>>>>>>>> (noDefaultBranch()?)
> >>>>>>>>>>       just as
> >>>>>>>>>>       >>>>>>>>> easily.  In
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a
> >>>>>>>>>> nicer API
> >>>>>>>>> - a
> >>>>>>>>>>       >>>>> user
> >>>>>>>>>>       >>>>>>>> could
> >>>>>>>>>>       >>>>>>>>>>>>> specify
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing
> >>>>>>>>>> will reach
> >>>>>>>>> the
> >>>>>>>>>>       >>>>>> default
> >>>>>>>>>>       >>>>>>>>> branch,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case
> >>>>>>>>>> occurs.
> >>>>>>>>> That
> >>>>>>>>>>       >>>>> seems
> >>>>>>>>>>       >>>>>>> like
> >>>>>>>>>>       >>>>>>>>> an
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> improvement over the current branch()
> >>>>>>>>>> API,
> >>>>>>>>>>       which allows
> >>>>>>>>>>       >>>>>> for
> >>>>>>>>>>       >>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>> more
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly 
> >>>>>>>>>> getting
> >>>>>>>>> dropped.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
> >>>>>>>>>> certainly has
> >>>>>>>>>>       to be
> >>>>>>>>>>       >>>>>> well
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>>>>>>>       >>>>> InternalTopologyBuilder
> >>>>>>>>>>       >>>>>>> to
> >>>>>>>>>>       >>>>>>>>> track
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated
> >>>>>>>>>> and raise
> >>>>>>>>>>       a clear
> >>>>>>>>>>       >>>>>>> error
> >>>>>>>>>>       >>>>>>>>>>>>> before it
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that
> >>>>>>>>>> there is
> >>>>>>>>> a
> >>>>>>>>>>       >>>>> "build
> >>>>>>>>>>       >>>>>>>> step"
> >>>>>>>>>>       >>>>>>>>>>>>> where
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
> >>>>>>>>>>       >>>>>> StreamsBuilder.build()
> >>>>>>>>>>       >>>>>>> is
> >>>>>>>>>>       >>>>>>>>>>>>> called.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
> >>>>>>>>>> argument, I
> >>>>>>>>> agree
> >>>>>>>>>>       >>>>> that
> >>>>>>>>>>       >>>>>>> it's
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> allow users to do other operations on
> >>>>>>>>>> the input
> >>>>>>>>>>       stream.
> >>>>>>>>>>       >>>>>>> With
> >>>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same
> >>>>>>>>>> way all
> >>>>>>>>> other
> >>>>>>>>>>       >>>>>>> operations
> >>>>>>>>>>       >>>>>>>>> do -
> >>>>>>>>>>       >>>>>>>>>>>>> if
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> want to process off the original 
> >>>>>>>>>> KStream
> >>>>>>>>> multiple
> >>>>>>>>>>       >>>>> times,
> >>>>>>>>>>       >>>>>>> you
> >>>>>>>>>>       >>>>>>>>> just
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call
> >>>>>>>>>> as many
> >>>>>>>>>>       operations
> >>>>>>>>>>       >>>>>> on
> >>>>>>>>>>       >>>>>>> it
> >>>>>>>>>>       >>>>>>>>> as
> >>>>>>>>>>       >>>>>>>>>>>>> you
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan
> >>>>>>>>>> Ponomarev <
> >>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we
> >>>>>>>>>> do not
> >>>>>>>>>>       always need
> >>>>>>>>>>       >>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
> >>>>>>>>> operation we
> >>>>>>>>>>       >>>>> don't
> >>>>>>>>>>       >>>>>>>> know
> >>>>>>>>>>       >>>>>>>>>>>>> when to
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch 
> >>>>>>>>>> switch'.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its
> >>>>>>>>>> argument,
> >>>>>>>>>>       so we
> >>>>>>>>>>       >>>>> can
> >>>>>>>>>>       >>>>>> do
> >>>>>>>>>>       >>>>>>>>>>>>> something
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> more with the original branch after
> >>>>>>>>>> branching.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I understand your point that the 
> >>>>>>>>>> need of
> >>>>>>>>> special
> >>>>>>>>>>       >>>>> object
> >>>>>>>>>>       >>>>>>>>>>>>> construction
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream
> >>>>>>>>>> methods.
> >>>>>>>>> But
> >>>>>>>>>>       >>>>> here
> >>>>>>>>>>       >>>>>> we
> >>>>>>>>>>       >>>>>>>>> have a
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to
> >>>>>>>>>> split the
> >>>>>>>>>>       flow,
> >>>>>>>>>>       >>>>> so
> >>>>>>>>>>       >>>>>> I
> >>>>>>>>>>       >>>>>>>>> think
> >>>>>>>>>>       >>>>>>>>>>>>> this
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve
> >>>>>>>>>> this
> >>>>>>>>>>       API, but I
> >>>>>>>>>>       >>>>>> find
> >>>>>>>>>>       >>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing 
> >>>>>>>>>> since it
> >>>>>>>>>>       contrasts the
> >>>>>>>>>>       >>>>>>> fluency
> >>>>>>>>>>       >>>>>>>>> of
> >>>>>>>>>>       >>>>>>>>>>>>> other
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd
> >>>>>>>>>> like to
> >>>>>>>>>>       just call
> >>>>>>>>>>       >>>>> a
> >>>>>>>>>>       >>>>>>>>> method on
> >>>>>>>>>>       >>>>>>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if
> >>>>>>>>>> the branch
> >>>>>>>>>>       cases
> >>>>>>>>>>       >>>>> are
> >>>>>>>>>>       >>>>>>>>> defined
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate,
> >>>>>>>>>> handleCase)
> >>>>>>>>>>       is very
> >>>>>>>>>>       >>>>>> nice
> >>>>>>>>>>       >>>>>>>>> and the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped
> >>>>>>>>>> around
> >>>>>>>>>>       how we
> >>>>>>>>>>       >>>>>>> specify
> >>>>>>>>>>       >>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>> source
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
> >>>>>>>>> this::handle1)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
> >>>>>>>>> this::handle2)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> .defaultBranch(this::handleDefault);
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
> >>>>>>>>>> KBranchedStreams or
> >>>>>>>>>>       >>>>>>>> KStreamBrancher
> >>>>>>>>>>       >>>>>>>>> or
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
> >>>>>>>>>>       terminated by
> >>>>>>>>>>       >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
> >>>>>>>>>>       incompatible with
> >>>>>>>>>>       >>>>> the
> >>>>>>>>>>       >>>>>>>>> current
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to
> >>>>>>>>>> have a
> >>>>>>>>>>       different
> >>>>>>>>>>       >>>>>> name,
> >>>>>>>>>>       >>>>>>>> but
> >>>>>>>>>>       >>>>>>>>> that
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we
> >>>>>>>>>> could call it
> >>>>>>>>>>       >>>>>> something
> >>>>>>>>>>       >>>>>>>> like
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the
> >>>>>>>>>> old API.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of
> >>>>>>>>>> your
> >>>>>>>>>>       KIP?  It
> >>>>>>>>>>       >>>>>> seems
> >>>>>>>>>>       >>>>>>>>> like it
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
> >>>>>>>>>> branching
> >>>>>>>>>>       while also
> >>>>>>>>>>       >>>>>>>> allowing
> >>>>>>>>>>       >>>>>>>>> you
> >>>>>>>>>>       >>>>>>>>>>>>> to
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
> >>>>>>>>>>       KBranchedStreams
> >>>>>>>>>>       >>>>>> if
> >>>>>>>>>>       >>>>>>>>> desired.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan
> >>>>>>>>>> Ponomarev
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void
> >>>>>>>>>> handleFirstCase(KStream<String, String>
> >>>>>>>>>>       ks){
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> ks.filter(....).mapValues(...)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void 
> >>>>>>>>>> handleSecondCase(KStream<String,
> >>>>>>>>>>       String> ks){
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> ks.selectKey(...).groupByKey()...
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String,
> >>>>>>>>>> String>()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >>>>>>>>>>       this::handleFirstCase)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >>>>>>>>>>       this::handleSecondCase)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> >>>>>>>>> KafkaStreamsBrancher
> >>>>>>>>>>       >>>>> takes a
> >>>>>>>>>>       >>>>>>>>> Consumer
> >>>>>>>>>>       >>>>>>>>>>>>> as a
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing,
> >>>>>>>>>> and the
> >>>>>>>>>>       example in
> >>>>>>>>>>       >>>>>> the
> >>>>>>>>>>       >>>>>>>> KIP
> >>>>>>>>>>       >>>>>>>>>>>>> shows
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a
> >>>>>>>>>> terminal node
> >>>>>>>>>>       >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but
> >>>>>>>>>> how would
> >>>>>>>>> we
> >>>>>>>>>>       >>>>> handle
> >>>>>>>>>>       >>>>>>> the
> >>>>>>>>>>       >>>>>>>>> case
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
> >>>>>>>>>> wants to
> >>>>>>>>> continue
> >>>>>>>>>>       >>>>>>>> processing
> >>>>>>>>>>       >>>>>>>>> and
> >>>>>>>>>>       >>>>>>>>>>>>> not
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on
> >>>>>>>>>> the
> >>>>>>>>> branched
> >>>>>>>>>>       >>>>>> stream
> >>>>>>>>>>       >>>>>>>>>>>>> immediately?
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic
> >>>>>>>>>> as is if
> >>>>>>>>>>       we had
> >>>>>>>>>>       >>>>>>>> something
> >>>>>>>>>>       >>>>>>>>> like
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] 
> >>>>>>>>>> branches =
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM
> >>>>>>>>>> Bill Bejeck
> >>>>>>>>> <
> >>>>>>>>>>       >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
> >>>>>>>>>> discussion for
> >>>>>>>>> KIP-
> >>>>>>>>>>       >>>>> 418.
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion 
> >>>>>>>>>> about
> >>>>>>>>> KIP-418.
> >>>>>>>>>>       >>>>> Please
> >>>>>>>>>>       >>>>>>>> take
> >>>>>>>>>>       >>>>>>>>> a
> >>>>>>>>>>       >>>>>>>>>>>>> look
> >>>>>>>>>>       >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
> >>>>>>>>>> appreciate any
> >>>>>>>>>>       feedback :)
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>>>>>>       >>>>>
> >>>>>>>>>>
> >>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>       >>>>>>>>>>>>> 
> >>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>>>>>>>       >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>       >>>>>>>>>
> >>>>>>>>>>       >
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> > 
> 
> 
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hello everyone,

will someone please take a look at the reworked KIP?

I believe that now it follows design principles and takes into account 
all the arguments discussed here.


Regards,

Ivan


23.04.2020 2:45, Ivan Ponomarev пишет:
> Hi,
> 
> I have read the John's "DSL design principles" and have completely 
> rewritten the KIP, see 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
> 
> 
> 
> This version includes all the previous discussion results and follows 
> the design principles, with one exception.
> 
> The exception is
> 
> branch(Predicate<K,V> predicate, Branched<K,V> branched)
> 
> which formally violates 'no more than one parameter' rule, but I think 
> here it is justified.
> 
> We must provide a predicate for a branch and don't need to provide one 
> for the default branch. Thus for both operations we may use a single 
> Branched parameter class, with an extra method parameter for `branch`.
> 
> Since predicate is a natural, necessary part of a branch, no 
> 'proliferation of overloads, deprecations, etc.' is expected here as it 
> is said in the rationale for the 'single parameter rule'.
> 
> WDYT, is this KIP mature enough to begin voting?
> 
> Regards,
> 
> Ivan
> 
> 21.04.2020 2:09, Matthias J. Sax пишет:
>> Ivan,
>>
>> no worries about getting side tracked. Glad to have you back!
>>
>> The DSL improved further in the meantime and we already have a `Named`
>> config object to name operators. It seems reasonable to me to build on 
>> this.
>>
>> Furthermore, John did a writeup about "DSL design principles" that we
>> want to follow:
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar 
>>
>> -- might be worth to checkout.
>>
>>
>> -Matthias
>>
>>
>> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>>> Hi everyone!
>>>
>>> Let me revive the discussion of this KIP.
>>>
>>> I'm very sorry for stopping my participation in the discussion in June
>>> 2019. My project work was very intensive then and it didn't leave me
>>> spare time. But I think I must finish this, because we invested
>>> substantial effort into this discussion and I'm not feel entitled to
>>> propose other things before this one is finalized.
>>>
>>> During these months I proceeded with writing and reviewing Kafka
>>> Streams-related code. Every time I needed branching, Spring-Kafka's
>>> KafkaStreamBrancher class of my invention (the original idea for this
>>> KIP) worked for me -- that's another reason why I gave up pushing the
>>> KIP forward. When I was coming across the problem with the scope of
>>> branches, I worked around it this way:
>>>
>>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>>> new KafkaStreamBrancher<....>()
>>>      .branch(....)
>>>      .defaultBranch(result::set)
>>>      .onTopOf(someStream);
>>> result.get()...
>>>
>>>
>>> And yes, of course I don't feel very happy with this approach.
>>>
>>> I think that Matthias came up with a bright solution in his post from
>>> May, 24th 2019. Let me quote it:
>>>
>>> KStream#split() -> KBranchedStream
>>> // branch is not easily accessible in current scope
>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>    -> KBranchedStream
>>> // assign a name to the branch and
>>> // return the sub-stream to the current scope later
>>> //
>>> // can be simple as `#branch(p, s->s, "name")`
>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>    -> KBranchedStream
>>> // default branch is not easily accessible
>>> // return map of all named sub-stream into current scope
>>> KBranchedStream#default(Cosumer<KStream>)
>>>    -> Map<String,KStream>
>>> // assign custom name to default-branch
>>> // return map of all named sub-stream into current scope
>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>    -> Map<String,KStream>
>>> // assign a default name for default
>>> // return map of all named sub-stream into current scope
>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>    -> Map<String,KStream>
>>> // return map of all names sub-stream into current scope
>>> KBranchedStream#noDefaultBranch()
>>>    -> Map<String,KStream>
>>>
>>> I believe this would satisfy everyone. Optional names seems to be a good
>>> idea: when you don't need to have the branches in the same scope, you
>>> just don't use names and you don't risk making your code brittle. Or,
>>> you might want to add names just for debugging purposes. Or, finally,
>>> you might use the returned Map to have the named branches in the
>>> original scope.
>>>
>>> There also was an input from John Roesler on June 4th, 2019, who
>>> suggested using Named class. I can't comment on this. The idea seems
>>> reasonable, but in this matter I'd rather trust people who are more
>>> familiar with Streams API design principles than me.
>>>
>>> Regards,
>>>
>>> Ivan
>>>
>>>
>>>
>>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>>> I am moving this KIP into "inactive status". Feel free to resume the 
>>>> KIP
>>>> at any point.
>>>>
>>>> If anybody else is interested in picking up this KIP, feel free to 
>>>> do so.
>>>>
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>>> Ivan,
>>>>>
>>>>> did you see my last reply? What do you think about my proposal to mix
>>>>> both approaches and try to get best-of-both worlds?
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>>> Thanks for the input John!
>>>>>>
>>>>>>> under your suggestion, it seems that the name is required
>>>>>>
>>>>>> If you want to get the `KStream` as part of the `Map` back using a
>>>>>> `Function`, yes. If you follow the "embedded chaining" pattern 
>>>>>> using a
>>>>>> `Consumer`, no.
>>>>>>
>>>>>> Allowing for a default name via `split()` can of course be done.
>>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>>
>>>>>> I wanted to sketch out a high level proposal to merge both patterns
>>>>>> only. Your suggestions to align the new API with the existing API 
>>>>>> make
>>>>>> totally sense.
>>>>>>
>>>>>>
>>>>>>
>>>>>> One follow up question: Would `Named` be optional or required in
>>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>>
>>>>>> If both are mandatory, what do we gain by it? The returned `Map` only
>>>>>> contains the corresponding branches, so why should we prefix all of
>>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>>> `split()`, the same question raises?
>>>>>>
>>>>>> Requiring `Named` in `split()` seems only to make sense, if 
>>>>>> `Named` is
>>>>>> optional in `branch()` and we generate `-X` suffix using a counter 
>>>>>> for
>>>>>> different branch name. However, this might lead to the problem of
>>>>>> changing names if branches are added/removed. Also, how would the 
>>>>>> names
>>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>>> returned in the `Map`).
>>>>>>
>>>>>> If `Named` is optional for both, it could happen that a user 
>>>>>> misses to
>>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>>
>>>>>>
>>>>>> Hence, I am actually in favor to not allow a default name but keep
>>>>>> `split()` without parameter and make `Named` in `branch()` required
>>>>>> if a
>>>>>> `Function` is used. This makes it explicit to the user that
>>>>>> specifying a
>>>>>> name is required if a `Function` is used.
>>>>>>
>>>>>>
>>>>>>
>>>>>> About
>>>>>>
>>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>>
>>>>>> I don't think that the branching predicate is a configuration and 
>>>>>> hence
>>>>>> would not include it in a configuration object.
>>>>>>
>>>>>>>       withChain(...);
>>>>>>
>>>>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>>>>>> seem to be a configuration. We can also not prevent a user to call
>>>>>> `withName()` in combination of `withChain()` what does not make sense
>>>>>> IMHO. We could of course throw an RTE but not have a compile time 
>>>>>> check
>>>>>> seems less appealing. Also, it could happen that neither 
>>>>>> `withChain()`
>>>>>> not `withName()` is called and the branch is missing in the returned
>>>>>> `Map` what lead to runtime issues, too.
>>>>>>
>>>>>> Hence, I don't think that we should add `BranchConfig`. A config 
>>>>>> object
>>>>>> is helpful if each configuration can be set independently of all
>>>>>> others,
>>>>>> but this seems not to be the case here. If we add new configuration
>>>>>> later, we can also just move forward by deprecating the methods that
>>>>>> accept `Named` and add new methods that accepted `BranchConfig` (that
>>>>>> would of course implement `Named`).
>>>>>>
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>>
>>>>>> @Ivan, what do you think about the general idea to blend the two main
>>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>>>>>> everyone. Returning the map from the terminal operations also solves
>>>>>>> the problem of merging/joining the branched streams, if we want 
>>>>>>> to add
>>>>>>> support for the compliment later on.
>>>>>>>
>>>>>>> Under your suggestion, it seems that the name is required. 
>>>>>>> Otherwise,
>>>>>>> we wouldn't have keys for the map to return. I this this is actually
>>>>>>> not too bad, since experience has taught us that, although names for
>>>>>>> operations are not required to define stream processing logic, it 
>>>>>>> does
>>>>>>> significantly improve the operational experience when you can map 
>>>>>>> the
>>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>>> wouldn't (have to) reference the name to chain extra processing onto
>>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>>
>>>>>>> In the current implementation of Branch, you can name the branch
>>>>>>> operator itself, and then all the branches get index-suffixed names
>>>>>>> built from the branch operator name. I guess under this proposal, we
>>>>>>> could naturally append the branch name to the branching operator 
>>>>>>> name,
>>>>>>> like this:
>>>>>>>
>>>>>>>      stream.split(Named.withName("mysplit")) //creates node 
>>>>>>> "mysplit"
>>>>>>>                 .branch(..., ..., "abranch") // creates node
>>>>>>> "mysplit-abranch"
>>>>>>>                 .defaultBranch(...) // creates node 
>>>>>>> "mysplit-default"
>>>>>>>
>>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>>
>>>>>>> We don't have a defined grammar, so there's plenty of room to debate
>>>>>>> the "best" syntax in the context of each operation, but in general,
>>>>>>> the KStream DSL operators follow this pattern:
>>>>>>>
>>>>>>>       operator(function, config_object?) OR operator(config_object)
>>>>>>>
>>>>>>> where config_object is often just Named in the "function" variant.
>>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>>
>>>>>>> Here, we're introducing a totally different pattern:
>>>>>>>
>>>>>>>     operator(function, function, string)
>>>>>>>
>>>>>>> where the string is the name.
>>>>>>> My first question is whether the name should instead be specified 
>>>>>>> with
>>>>>>> the NamedOperation interface.
>>>>>>>
>>>>>>> My second question is whether we should just roll all these 
>>>>>>> arguments
>>>>>>> up into a config object like:
>>>>>>>
>>>>>>>      KBranchedStream#branch(BranchConfig)
>>>>>>>
>>>>>>>      interface BranchConfig extends NamedOperation {
>>>>>>>       withPredicate(...);
>>>>>>>       withChain(...);
>>>>>>>       withName(...);
>>>>>>>     }
>>>>>>>
>>>>>>> Although I guess we'd like to call BranchConfig something more like
>>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>>
>>>>>>> This makes the source code a little noisier, but it also makes us 
>>>>>>> more
>>>>>>> future-proof, as we can deal with a wide range of alternatives 
>>>>>>> purely
>>>>>>> in the config interface, and never have to deal with adding 
>>>>>>> overloads
>>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>>
>>>>>>> WDYT?
>>>>>>>
>>>>>>> Thanks,
>>>>>>> -John
>>>>>>>
>>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>>> <mi...@confluent.io> wrote:
>>>>>>>>
>>>>>>>> Matthias: I think that's pretty reasonable from my point of view.
>>>>>>>> Good
>>>>>>>> suggestion.
>>>>>>>>
>>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>>> <ma...@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Interesting discussion.
>>>>>>>>>
>>>>>>>>> I am wondering, if we cannot unify the advantage of both 
>>>>>>>>> approaches:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>>
>>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>>     -> KBranchedStream
>>>>>>>>>
>>>>>>>>> // assign a name to the branch and
>>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>>> //
>>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, 
>>>>>>>>> String)
>>>>>>>>>     -> KBranchedStream
>>>>>>>>>
>>>>>>>>> // default branch is not easily accessible
>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>
>>>>>>>>> // assign custom name to default-branch
>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>
>>>>>>>>> // assign a default name for default
>>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>
>>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>>     -> Map<String,KStream>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Hence, for each sub-stream, the user can pick to add a name and
>>>>>>>>> return
>>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>>> implementation can
>>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>>> returned
>>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>>
>>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>>
>>>>>>>>> Thoughts?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>>> Ivan,
>>>>>>>>>>
>>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>>> dynamic case.
>>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>>> necessity.
>>>>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>>
>>>>>>>>>> Separately, I'm interested to see where the present discussion
>>>>>>>>>> leads.
>>>>>>>>>> I've written enough Javascript code in my life to be 
>>>>>>>>>> suspicious of
>>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>>> references (or
>>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>>> that this
>>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>>> logic when
>>>>>>>>>> their nested closure situation got out of hand. Unfortunately, 
>>>>>>>>>> it's
>>>>>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>>>>>> readability (but in different ways for different reasons). In 
>>>>>>>>>> other
>>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>>> solution if
>>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>>
>>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>>> references aren't
>>>>>>>>>> the end of the story either, and you can see that by observing 
>>>>>>>>>> that
>>>>>>>>>> there have been two follow-on eras, as they continue trying to
>>>>>>>>>> cope with
>>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>>> First, you
>>>>>>>>>> have Futures/Promises, which essentially let you convert nested
>>>>>>>>>> code to
>>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>>> this).
>>>>>>>>>> Most lately, you have async/await, which is an effort to apply
>>>>>>>>>> language
>>>>>>>>>> (not just API) syntax to the problem, and offer the "flattest"
>>>>>>>>>> possible
>>>>>>>>>> programming style to solve the problem (because you get back to
>>>>>>>>>> just one
>>>>>>>>>> code block per functional unit).
>>>>>>>>>>
>>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>>> nowhere
>>>>>>>>>> near as callback heavy as JS, so I don't think we have to take
>>>>>>>>>> the JS
>>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>>> valuable
>>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>>> bringing this
>>>>>>>>>> up to inspire further/deeper discussion. At the same time, just
>>>>>>>>>> like JS,
>>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>>
>>>>>>>>>> Separately again, I'm interested in the post-branch merge (and
>>>>>>>>>> I'd also
>>>>>>>>>> add join) problem that Paul brought up. We can clearly punt on
>>>>>>>>>> it, by
>>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>>> there a DSL
>>>>>>>>>> way to do it?
>>>>>>>>>>
>>>>>>>>>> Thanks again for your driving this,
>>>>>>>>>> -John
>>>>>>>>>>
>>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>>
>>>>>>>>>>       Ivan, I’ll definitely forfeit my point on the clumsiness of
>>>>>>>>>> the
>>>>>>>>>>       branch(predicate, consumer) solution, I don’t see any real
>>>>>>>>>> drawbacks
>>>>>>>>>>       for the dynamic case.
>>>>>>>>>>
>>>>>>>>>>       IMO the one trade off to consider at this point is the 
>>>>>>>>>> scope
>>>>>>>>>>       question. I don’t know if I totally agree that “we rarely
>>>>>>>>>> need them
>>>>>>>>>>       in the same scope” since merging the branches back together
>>>>>>>>>> later
>>>>>>>>>>       seems like a perfectly plausible use case that can be a lot
>>>>>>>>>> nicer
>>>>>>>>>>       when the branched streams are in the same scope. That being
>>>>>>>>>> said,
>>>>>>>>>>       for the reasons Ivan listed, I think it is overall the 
>>>>>>>>>> better
>>>>>>>>>>       solution - working around the scope thing is easy enough if
>>>>>>>>>> you need
>>>>>>>>>>       to.
>>>>>>>>>>
>>>>>>>>>>       > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>>       <ip...@mail.ru.invalid> wrote:
>>>>>>>>>>       >
>>>>>>>>>>       > Hello everyone, thank you all for joining the discussion!
>>>>>>>>>>       >
>>>>>>>>>>       > Well, I don't think the idea of named branches, be it a
>>>>>>>>>>       LinkedHashMap (no other Map will do, because order of
>>>>>>>>>> definition
>>>>>>>>>>       matters) or `branch` method  taking name and Consumer 
>>>>>>>>>> has more
>>>>>>>>>>       advantages than drawbacks.
>>>>>>>>>>       >
>>>>>>>>>>       > In my opinion, the only real positive outcome from 
>>>>>>>>>> Michael's
>>>>>>>>>>       proposal is that all the returned branches are in the same
>>>>>>>>>> scope.
>>>>>>>>>>       But 1) we rarely need them in the same scope 2) there is a
>>>>>>>>>>       workaround for the scope problem, described in the KIP.
>>>>>>>>>>       >
>>>>>>>>>>       > 'Inlining the complex logic' is not a problem, because we
>>>>>>>>>> can use
>>>>>>>>>>       method references instead of lambdas. In real world
>>>>>>>>>> scenarios you
>>>>>>>>>>       tend to split the complex logic to methods anyway, so the
>>>>>>>>>> code is
>>>>>>>>>>       going to be clean.
>>>>>>>>>>       >
>>>>>>>>>>       > The drawbacks are strong. The cohesion between predicates
>>>>>>>>>> and
>>>>>>>>>>       handlers is lost. We have to define predicates in one
>>>>>>>>>> place, and
>>>>>>>>>>       handlers in another. This opens the door for bugs:
>>>>>>>>>>       >
>>>>>>>>>>       > - what if we forget to define a handler for a name? or a
>>>>>>>>>> name for
>>>>>>>>>>       a handler?
>>>>>>>>>>       > - what if we misspell a name?
>>>>>>>>>>       > - what if we copy-paste and duplicate a name?
>>>>>>>>>>       >
>>>>>>>>>>       > What Michael propose would have been totally OK if we had
>>>>>>>>>> been
>>>>>>>>>>       writing the API in Lua, Ruby or Python. In those 
>>>>>>>>>> languages the
>>>>>>>>>>       "dynamic naming" approach would have looked most concise 
>>>>>>>>>> and
>>>>>>>>>>       beautiful. But in Java we expect all the problems 
>>>>>>>>>> related to
>>>>>>>>>>       identifiers to be eliminated in compile time.
>>>>>>>>>>       >
>>>>>>>>>>       > Do we have to invent duck-typing for the Java API?
>>>>>>>>>>       >
>>>>>>>>>>       > And if we do, what advantage are we supposed to get
>>>>>>>>>> besides having
>>>>>>>>>>       all the branches in the same scope? Michael, maybe I'm
>>>>>>>>>> missing your
>>>>>>>>>>       point?
>>>>>>>>>>       >
>>>>>>>>>>       > ---
>>>>>>>>>>       >
>>>>>>>>>>       > Earlier in this discussion John Roesler also proposed 
>>>>>>>>>> to do
>>>>>>>>>>       without "start branching" operator, and later Paul
>>>>>>>>>> mentioned that in
>>>>>>>>>>       the case when we have to add a dynamic number of 
>>>>>>>>>> branches, the
>>>>>>>>>>       current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>>> solution. Let
>>>>>>>>>>       me address both comments here.
>>>>>>>>>>       >
>>>>>>>>>>       > 1) "Start branching" operator (I think that *split* is a
>>>>>>>>>> good name
>>>>>>>>>>       for it indeed) is critical when we need to do a dynamic
>>>>>>>>>> branching,
>>>>>>>>>>       see example below.
>>>>>>>>>>       >
>>>>>>>>>>       > 2) No, dynamic branching in current KIP is not clumsy at
>>>>>>>>>> all.
>>>>>>>>>>       Imagine a real-world scenario when you need one branch per
>>>>>>>>>> enum
>>>>>>>>>>       value (say, RecordType). You can have something like this:
>>>>>>>>>>       >
>>>>>>>>>>       > /*John:if we had to start with stream.branch(...) here,
>>>>>>>>>> it would
>>>>>>>>>>       have been much messier.*/
>>>>>>>>>>       > KBranchedStream branched = stream.split();
>>>>>>>>>>       >
>>>>>>>>>>       > /*Not clumsy at all :-)*/
>>>>>>>>>>       > for (RecordType recordType : RecordType.values())
>>>>>>>>>>       >             branched = branched.branch((k, v) ->
>>>>>>>>>> v.getRecType() ==
>>>>>>>>>>       recordType,
>>>>>>>>>>       >                     recordType::processRecords);
>>>>>>>>>>       >
>>>>>>>>>>       > Regards,
>>>>>>>>>>       >
>>>>>>>>>>       > Ivan
>>>>>>>>>>       >
>>>>>>>>>>       >
>>>>>>>>>>       > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>>       >> I also agree with Michael's observation about the core
>>>>>>>>>> problem of
>>>>>>>>>>       >> current `branch()` implementation.
>>>>>>>>>>       >>
>>>>>>>>>>       >> However, I also don't like to pass in a clumsy Map
>>>>>>>>>> object. My
>>>>>>>>>>       thinking
>>>>>>>>>>       >> was more aligned with Paul's proposal to just add a name
>>>>>>>>>> to each
>>>>>>>>>>       >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>>>>>       >>
>>>>>>>>>>       >> It makes the code easier to read, and also make the
>>>>>>>>>> order of
>>>>>>>>>>       >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>>       >>
>>>>>>>>>>       >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>>       >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>       >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>       >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>>       >> An open question is the case for which no
>>>>>>>>>> defaultBranch() should
>>>>>>>>> be
>>>>>>>>>>       >> specified. Atm, `split()` and `branch()` would return
>>>>>>>>>>       `BranchedKStream`
>>>>>>>>>>       >> and the call to `defaultBranch()` that returns the 
>>>>>>>>>> `Map` is
>>>>>>>>> mandatory
>>>>>>>>>>       >> (what is not the case atm). Or is this actually not a 
>>>>>>>>>> real
>>>>>>>>> problem,
>>>>>>>>>>       >> because users can just ignore the branch returned by
>>>>>>>>>>       `defaultBranch()`
>>>>>>>>>>       >> in the result `Map` ?
>>>>>>>>>>       >>
>>>>>>>>>>       >>
>>>>>>>>>>       >> About "inlining": So far, it seems to be a matter of
>>>>>>>>>> personal
>>>>>>>>>>       >> preference. I can see arguments for both, but no "killer
>>>>>>>>>>       argument" yet
>>>>>>>>>>       >> that clearly make the case for one or the other.
>>>>>>>>>>       >>
>>>>>>>>>>       >>
>>>>>>>>>>       >> -Matthias
>>>>>>>>>>       >>
>>>>>>>>>>       >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>>       >>> Perhaps inlining is the wrong terminology. It doesn’t
>>>>>>>>>> require
>>>>>>>>>>       that a lambda with the full downstream topology be defined
>>>>>>>>>> inline -
>>>>>>>>>>       it can be a method reference as with Ivan’s original
>>>>>>>>>> suggestion.
>>>>>>>>>>       The advantage of putting the predicate and its downstream
>>>>>>>>>> logic
>>>>>>>>>>       (Consumer) together in branch() is that they are required
>>>>>>>>>> to be near
>>>>>>>>>>       to each other.
>>>>>>>>>>       >>>
>>>>>>>>>>       >>> Ultimately the downstream code has to live somewhere,
>>>>>>>>>> and deep
>>>>>>>>>>       branch trees will be hard to read regardless.
>>>>>>>>>>       >>>
>>>>>>>>>>       >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>>       <michael.drogalis@confluent.io
>>>>>>>>>>       <ma...@confluent.io>> wrote:
>>>>>>>>>>       >>>>
>>>>>>>>>>       >>>> I'm less enthusiastic about inlining the branch logic
>>>>>>>>>> with its
>>>>>>>>>>       downstream
>>>>>>>>>>       >>>> functionality. Programs that have deep branch trees 
>>>>>>>>>> will
>>>>>>>>>>       quickly become
>>>>>>>>>>       >>>> harder to read as a single unit.
>>>>>>>>>>       >>>>
>>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>>       <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Also +1 on the issues/goals as Michael outlined them,
>>>>>>>>>> I think
>>>>>>>>>>       that sets a
>>>>>>>>>>       >>>>> great framework for the discussion.
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Regarding the SortedMap solution, my understanding is
>>>>>>>>>> that the
>>>>>>>>>>       current
>>>>>>>>>>       >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>>> (pending naming
>>>>>>>>>>       decisions) is
>>>>>>>>>>       >>>>> roughly this:
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> stream.split()
>>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>>       >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Obviously some ordering is necessary, since branching
>>>>>>>>>> as a
>>>>>>>>>>       construct
>>>>>>>>>>       >>>>> doesn't work without it, but this solution seems 
>>>>>>>>>> like it
>>>>>>>>>>       provides as much
>>>>>>>>>>       >>>>> associativity as the SortedMap solution, because each
>>>>>>>>>> branch()
>>>>>>>>>>       call
>>>>>>>>>>       >>>>> directly associates the "conditional" with the "code
>>>>>>>>>> block."
>>>>>>>>>>       The value it
>>>>>>>>>>       >>>>> provides over the KIP solution is the accessing of
>>>>>>>>>> streams in
>>>>>>>>>>       the same
>>>>>>>>>>       >>>>> scope.
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> The KIP solution is less "dynamic" than the SortedMap
>>>>>>>>>> solution
>>>>>>>>>>       in the sense
>>>>>>>>>>       >>>>> that it is slightly clumsier to add a dynamic 
>>>>>>>>>> number of
>>>>>>>>>>       branches, but it is
>>>>>>>>>>       >>>>> certainly possible.  It seems to me like the API
>>>>>>>>>> should favor
>>>>>>>>>>       the "static"
>>>>>>>>>>       >>>>> case anyway, and should make it simple and 
>>>>>>>>>> readable to
>>>>>>>>>>       fluently declare and
>>>>>>>>>>       >>>>> access your branches in-line.  It also makes it
>>>>>>>>>> impossible to
>>>>>>>>>>       ignore a
>>>>>>>>>>       >>>>> branch, and it is possible to build an (almost)
>>>>>>>>>> identical
>>>>>>>>>>       SortedMap
>>>>>>>>>>       >>>>> solution on top of it.
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> I could also see a middle ground where instead of 
>>>>>>>>>> a raw
>>>>>>>>>>       SortedMap being
>>>>>>>>>>       >>>>> taken in, branch() takes a name and not a Consumer.
>>>>>>>>>> Something
>>>>>>>>>>       like this:
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>>       >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>>       >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>>       >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Pros for that solution:
>>>>>>>>>>       >>>>> - accessing branched KStreams in same scope
>>>>>>>>>>       >>>>> - no double brace initialization, hopefully slightly
>>>>>>>>>> more
>>>>>>>>>>       readable than
>>>>>>>>>>       >>>>> SortedMap
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Cons
>>>>>>>>>>       >>>>> - downstream branch logic cannot be specified inline
>>>>>>>>>> which
>>>>>>>>>>       makes it harder
>>>>>>>>>>       >>>>> to read top to bottom (like existing API and
>>>>>>>>>> SortedMap, but
>>>>>>>>>>       unlike the KIP)
>>>>>>>>>>       >>>>> - you can forget to "handle" one of the branched
>>>>>>>>>> streams (like
>>>>>>>>>>       existing
>>>>>>>>>>       >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> (KBranchedStreams could even work *both* ways but
>>>>>>>>>> perhaps
>>>>>>>>>>       that's overdoing
>>>>>>>>>>       >>>>> it).
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Overall I'm curious how important it is to be able to
>>>>>>>>>> easily
>>>>>>>>>>       access the
>>>>>>>>>>       >>>>> branched KStream in the same scope as the original.
>>>>>>>>>> It's
>>>>>>>>>>       possible that it
>>>>>>>>>>       >>>>> doesn't need to be handled directly by the API, but
>>>>>>>>>> instead
>>>>>>>>>>       left up to the
>>>>>>>>>>       >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> Paul
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>>>>>       <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>>       >>>>> wrote:
>>>>>>>>>>       >>>>>
>>>>>>>>>>       >>>>>> I'd like to +1 what Michael said about the issues
>>>>>>>>>> with the
>>>>>>>>>>       existing
>>>>>>>>>>       >>>>> branch
>>>>>>>>>>       >>>>>> method, I agree with what he's outlined and I think
>>>>>>>>>> we should
>>>>>>>>>>       proceed by
>>>>>>>>>>       >>>>>> trying to alleviate these problems. Specifically it
>>>>>>>>>> seems
>>>>>>>>>>       important to be
>>>>>>>>>>       >>>>>> able to cleanly access the individual branches (eg
>>>>>>>>>> by mapping
>>>>>>>>>>       >>>>>> name->stream), which I thought was the original
>>>>>>>>>> intention of
>>>>>>>>>>       this KIP.
>>>>>>>>>>       >>>>>>
>>>>>>>>>>       >>>>>> That said, I don't think we should so easily give in
>>>>>>>>>> to the
>>>>>>>>>>       double brace
>>>>>>>>>>       >>>>>> anti-pattern or force ours users into it if at all
>>>>>>>>>> possible to
>>>>>>>>>>       >>>>> avoid...just
>>>>>>>>>>       >>>>>> my two cents.
>>>>>>>>>>       >>>>>>
>>>>>>>>>>       >>>>>> Cheers,
>>>>>>>>>>       >>>>>> Sophie
>>>>>>>>>>       >>>>>>
>>>>>>>>>>       >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>>>>>       >>>>>> michael.drogalis@confluent.io
>>>>>>>>>>       <ma...@confluent.io>> wrote:
>>>>>>>>>>       >>>>>>
>>>>>>>>>>       >>>>>>> I’d like to propose a different way of thinking
>>>>>>>>>> about this.
>>>>>>>>>>       To me,
>>>>>>>>>>       >>>>> there
>>>>>>>>>>       >>>>>>> are three problems with the existing branch 
>>>>>>>>>> signature:
>>>>>>>>>>       >>>>>>>
>>>>>>>>>>       >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>>> raises unsafe
>>>>>>>>> type
>>>>>>>>>>       >>>>>> warnings.
>>>>>>>>>>       >>>>>>> 2. The way in which you use the stream branches is
>>>>>>>>>>       positionally coupled
>>>>>>>>>>       >>>>>> to
>>>>>>>>>>       >>>>>>> the ordering of the conditionals.
>>>>>>>>>>       >>>>>>> 3. It is brittle to extend existing branch calls 
>>>>>>>>>> with
>>>>>>>>>>       additional code
>>>>>>>>>>       >>>>>>> paths.
>>>>>>>>>>       >>>>>>>
>>>>>>>>>>       >>>>>>> Using associative constructs instead of relying on
>>>>>>>>>> ordered
>>>>>>>>>>       constructs
>>>>>>>>>>       >>>>>> would
>>>>>>>>>>       >>>>>>> be a stronger approach. Consider a signature that
>>>>>>>>>> instead
>>>>>>>>>>       looks like
>>>>>>>>>>       >>>>>> this:
>>>>>>>>>>       >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>>       Predicate<?
>>>>>>>>>>       >>>>>>> super K,? super V>>);
>>>>>>>>>>       >>>>>>>
>>>>>>>>>>       >>>>>>> Branches are given names in a map, and as a result,
>>>>>>>>>> the API
>>>>>>>>>>       returns a
>>>>>>>>>>       >>>>>>> mapping of names to streams. The ordering of the
>>>>>>>>> conditionals is
>>>>>>>>>>       >>>>>> maintained
>>>>>>>>>>       >>>>>>> because it’s a sorted map. Insert order determines
>>>>>>>>>> the order
>>>>>>>>> of
>>>>>>>>>>       >>>>>> evaluation.
>>>>>>>>>>       >>>>>>> This solves problem 1 because there are no more
>>>>>>>>>> varargs. It
>>>>>>>>>>       solves
>>>>>>>>>>       >>>>>> problem
>>>>>>>>>>       >>>>>>> 2 because you no longer lean on ordering to 
>>>>>>>>>> access the
>>>>>>>>>>       branch you’re
>>>>>>>>>>       >>>>>>> interested in. It solves problem 3 because you can
>>>>>>>>>> introduce
>>>>>>>>>>       another
>>>>>>>>>>       >>>>>>> conditional by simply attaching another name to the
>>>>>>>>>>       structure, rather
>>>>>>>>>>       >>>>>> than
>>>>>>>>>>       >>>>>>> messing with the existing indices.
>>>>>>>>>>       >>>>>>>
>>>>>>>>>>       >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>>> inline is
>>>>>>>>>>       historically
>>>>>>>>>>       >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>>>>>       voluminously, but
>>>>>>>>>>       >>>>>>> double brace initialization would clean up the
>>>>>>>>>> aesthetics.
>>>>>>>>>>       >>>>>>>
>>>>>>>>>>       >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>>       <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>>       >>>>> wrote:
>>>>>>>>>>       >>>>>>>> Hi Ivan,
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>> Thanks for the update.
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>> FWIW, I agree with Matthias that the current 
>>>>>>>>>> "start
>>>>>>>>> branching"
>>>>>>>>>>       >>>>> operator
>>>>>>>>>>       >>>>>>> is
>>>>>>>>>>       >>>>>>>> confusing when named the same way as the actual
>>>>>>>>>> branches.
>>>>>>>>>>       "Split"
>>>>>>>>>>       >>>>> seems
>>>>>>>>>>       >>>>>>>> like a good name. Alternatively, we can do without
>>>>>>>>>> a "start
>>>>>>>>>>       >>>>> branching"
>>>>>>>>>>       >>>>>>>> operator at all, and just do:
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>> stream
>>>>>>>>>>       >>>>>>>>      .branch(Predicate)
>>>>>>>>>>       >>>>>>>>      .branch(Predicate)
>>>>>>>>>>       >>>>>>>>      .defaultBranch();
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>> Tentatively, I think that this branching operation
>>>>>>>>>> should be
>>>>>>>>>>       >>>>> terminal.
>>>>>>>>>>       >>>>>>> That
>>>>>>>>>>       >>>>>>>> way, we don't create ambiguity about how to use
>>>>>>>>>> it. That
>>>>>>>>>>       is, `branch`
>>>>>>>>>>       >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>>> `defaultBranch` is
>>>>>>>>>>       `void`, to
>>>>>>>>>>       >>>>>>>> enforce that it comes last, and that there is only
>>>>>>>>>> one
>>>>>>>>>>       definition of
>>>>>>>>>>       >>>>>> the
>>>>>>>>>>       >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>>> warning if
>>>>>>>>>>       there's no
>>>>>>>>>>       >>>>>>> default,
>>>>>>>>>>       >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>>> exception) if a
>>>>>>>>>>       record
>>>>>>>>>>       >>>>>> falls
>>>>>>>>>>       >>>>>>>> though with no default.
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>> Thoughts?
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>> Thanks,
>>>>>>>>>>       >>>>>>>> -John
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>>>>>       >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>>>>>       >>>>>>>> wrote:
>>>>>>>>>>       >>>>>>>>
>>>>>>>>>>       >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> this is to make the name similar to String#split
>>>>>>>>>>       >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>>       >>>>>>>>> The intend was to avoid name duplication. The
>>>>>>>>>> return type
>>>>>>>>>>       should
>>>>>>>>>>       >>>>>> _not_
>>>>>>>>>>       >>>>>>>>> be an array.
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> The current proposal is
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> stream.branch()
>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>       >>>>>>>>>      .defaultBranch();
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>>>>>       `branch()` does
>>>>>>>>>>       >>>>> not
>>>>>>>>>>       >>>>>>>>> take any parameters and has different semantics
>>>>>>>>>> than the
>>>>>>>>> later
>>>>>>>>>>       >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>>> snippet above,
>>>>>>>>> it's
>>>>>>>>>>       >>>>> hidden
>>>>>>>>>>       >>>>>>>>> that the first call is `KStream#branch()` while
>>>>>>>>>> the others
>>>>>>>>> are
>>>>>>>>>>       >>>>>>>>> `KBranchedStream#branch()` what makes reading the
>>>>>>>>>> code
>>>>>>>>> harder.
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> Because I suggested to rename `addBranch()` ->
>>>>>>>>>> `branch()`,
>>>>>>>>>>       I though
>>>>>>>>>>       >>>>>> it
>>>>>>>>>>       >>>>>>>>> might be better to also rename `KStream#branch()`
>>>>>>>>>> to avoid
>>>>>>>>> the
>>>>>>>>>>       >>>>> naming
>>>>>>>>>>       >>>>>>>>> overlap that seems to be confusing. The following
>>>>>>>>>> reads
>>>>>>>>> much
>>>>>>>>>>       >>>>> cleaner
>>>>>>>>>>       >>>>>> to
>>>>>>>>>>       >>>>>>>> me:
>>>>>>>>>>       >>>>>>>>> stream.split()
>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>>       >>>>>>>>>      .defaultBranch();
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> Maybe there is a better alternative to `split()`
>>>>>>>>>> though to
>>>>>>>>>>       avoid
>>>>>>>>>>       >>>>> the
>>>>>>>>>>       >>>>>>>>> naming overlap.
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>>> unfortunately
>>>>>>>>> we
>>>>>>>>>>       >>>>> cannot
>>>>>>>>>>       >>>>>>> have
>>>>>>>>>>       >>>>>>>>> a method with such name :-)
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> Bummer. Didn't consider this. Maybe we can still
>>>>>>>>>> come up
>>>>>>>>>>       with a
>>>>>>>>>>       >>>>> short
>>>>>>>>>>       >>>>>>>> name?
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> Can you add the interface `KBranchedStream` to
>>>>>>>>>> the KIP
>>>>>>>>>>       with all
>>>>>>>>>>       >>>>> it's
>>>>>>>>>>       >>>>>>>>> methods? It will be part of public API and 
>>>>>>>>>> should be
>>>>>>>>>>       contained in
>>>>>>>>>>       >>>>> the
>>>>>>>>>>       >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>>> return type of
>>>>>>>>>>       >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>>       `KBranchedStream#get(int
>>>>>>>>>>       >>>>>>> index)
>>>>>>>>>>       >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>>       branched-KStreams. Would
>>>>>>>>>>       >>>>>> be
>>>>>>>>>>       >>>>>>>>> nice to get your feedback about it. It seems you
>>>>>>>>>> suggest
>>>>>>>>>>       that users
>>>>>>>>>>       >>>>>>>>> would need to write custom utility code
>>>>>>>>>> otherwise, to
>>>>>>>>>>       access them.
>>>>>>>>>>       >>>>> We
>>>>>>>>>>       >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>>> approaches. It
>>>>>>>>> feels
>>>>>>>>>>       >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>>> built-in support
>>>>>>>>>>       to get
>>>>>>>>>>       >>>>> the
>>>>>>>>>>       >>>>>>>>> branched-KStreams directly.
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>> -Matthias
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>>       >>>>>>>>>> Hi all!
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> I have updated the KIP-418 according to the new
>>>>>>>>>> vision.
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>>       >>>>>>>>>> I can see your point: this is to make the name
>>>>>>>>>> similar to
>>>>>>>>>>       >>>>>>> String#split
>>>>>>>>>>       >>>>>>>>>> that also returns an array, right? But is it
>>>>>>>>>> worth the
>>>>>>>>>>       loss of
>>>>>>>>>>       >>>>>>>> backwards
>>>>>>>>>>       >>>>>>>>>> compatibility? We can have overloaded branch()
>>>>>>>>>> as well
>>>>>>>>>>       without
>>>>>>>>>>       >>>>>>>> affecting
>>>>>>>>>>       >>>>>>>>>> the existing code. Maybe the old array-based
>>>>>>>>>> `branch`
>>>>>>>>> method
>>>>>>>>>>       >>>>> should
>>>>>>>>>>       >>>>>>> be
>>>>>>>>>>       >>>>>>>>>> deprecated, but this is a subject for 
>>>>>>>>>> discussion.
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>>       >>>>> BranchingKStream#branch(),
>>>>>>>>>>       >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>> BranchingKStream#default()
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> Totally agree with 'addBranch->branch' rename.
>>>>>>>>>> 'default'
>>>>>>>>> is,
>>>>>>>>>>       >>>>>>> however, a
>>>>>>>>>>       >>>>>>>>>> reserved word, so unfortunately we cannot have a
>>>>>>>>>> method
>>>>>>>>>>       with such
>>>>>>>>>>       >>>>>>> name
>>>>>>>>>>       >>>>>>>>> :-)
>>>>>>>>>>       >>>>>>>>>>> defaultBranch() does take an `Predicate` as
>>>>>>>>>> argument,
>>>>>>>>> but I
>>>>>>>>>>       >>>>> think
>>>>>>>>>>       >>>>>>> that
>>>>>>>>>>       >>>>>>>>>> is not required?
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> Absolutely! I think that was just copy-paste
>>>>>>>>>> error or
>>>>>>>>>>       something.
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> Dear colleagues,
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> please revise the new version of the KIP and
>>>>>>>>>> Paul's PR
>>>>>>>>>>       >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> Ivan
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>       >>>>>>>>>>> Thanks for driving the discussion of this KIP.
>>>>>>>>>> It seems
>>>>>>>>> that
>>>>>>>>>>       >>>>>>> everybody
>>>>>>>>>>       >>>>>>>>>>> agrees that the current branch() method using
>>>>>>>>>> arrays is
>>>>>>>>> not
>>>>>>>>>>       >>>>>> optimal.
>>>>>>>>>>       >>>>>>>>>>> I had a quick look into the PR and I like the
>>>>>>>>>> overall
>>>>>>>>>>       proposal.
>>>>>>>>>>       >>>>>>> There
>>>>>>>>>>       >>>>>>>>>>> are some minor things we need to consider. I 
>>>>>>>>>> would
>>>>>>>>>>       recommend the
>>>>>>>>>>       >>>>>>>>>>> following renaming:
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>       >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>>       >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>>       BranchingKStream#default()
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> It's just a suggestion to get slightly shorter
>>>>>>>>>> method
>>>>>>>>> names.
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>>>>>       `Predicate` as
>>>>>>>>>>       >>>>>>>> argument,
>>>>>>>>>>       >>>>>>>>>>> but I think that is not required?
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>>> recently
>>>>>>>>>>       accepted and
>>>>>>>>>>       >>>>>> is
>>>>>>>>>>       >>>>>>>>>>> currently implemented:
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>
>>>>>>>>>>
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL 
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> Ie, we should add overloads that accepted a
>>>>>>>>>> `Named`
>>>>>>>>>>       parameter.
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> For the issue that the created `KStream` object
>>>>>>>>>> are in
>>>>>>>>>>       different
>>>>>>>>>>       >>>>>>>> scopes:
>>>>>>>>>>       >>>>>>>>>>> could we extend `KBranchedStream` with a 
>>>>>>>>>> `get(int
>>>>>>>>>>       index)` method
>>>>>>>>>>       >>>>>>> that
>>>>>>>>>>       >>>>>>>>>>> returns the corresponding "branched" result
>>>>>>>>>> `KStream`
>>>>>>>>>>       object?
>>>>>>>>>>       >>>>>> Maybe,
>>>>>>>>>>       >>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>> second argument of `addBranch()` should not 
>>>>>>>>>> be a
>>>>>>>>>>       >>>>>> `Consumer<KStream>`
>>>>>>>>>>       >>>>>>>> but
>>>>>>>>>>       >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could
>>>>>>>>>> return
>>>>>>>>>>       whatever
>>>>>>>>>>       >>>>>> the
>>>>>>>>>>       >>>>>>>>>>> `Function` returns?
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> Finally, I would also suggest to update the KIP
>>>>>>>>>> with the
>>>>>>>>>>       current
>>>>>>>>>>       >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>> -Matthias
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>       >>>>>>>>>>>> Ivan,
>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>>> think it
>>>>>>>>>>       makes sense
>>>>>>>>>>       >>>>>> for
>>>>>>>>>>       >>>>>>>> you
>>>>>>>>>>       >>>>>>>>> to
>>>>>>>>>>       >>>>>>>>>>>> revise the KIP and continue the discussion.
>>>>>>>>>> Obviously
>>>>>>>>>>       we'll
>>>>>>>>>>       >>>>> need
>>>>>>>>>>       >>>>>>>> some
>>>>>>>>>>       >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>>> binding votes on
>>>>>>>>>>       >>>>> whether
>>>>>>>>>>       >>>>>>> the
>>>>>>>>>>       >>>>>>>>> KIP
>>>>>>>>>>       >>>>>>>>>>>> could be adopted.  It would be great to hear
>>>>>>>>>> if they
>>>>>>>>>>       think this
>>>>>>>>>>       >>>>>> is
>>>>>>>>>>       >>>>>>> a
>>>>>>>>>>       >>>>>>>>> good
>>>>>>>>>>       >>>>>>>>>>>> idea overall.  I'm not sure if that happens
>>>>>>>>>> just by
>>>>>>>>>>       starting a
>>>>>>>>>>       >>>>>>> vote,
>>>>>>>>>>       >>>>>>>>> or if
>>>>>>>>>>       >>>>>>>>>>>> there is generally some indication of interest
>>>>>>>>> beforehand.
>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>> That being said, I'll continue the discussion
>>>>>>>>>> a bit:
>>>>>>>>>>       assuming
>>>>>>>>>>       >>>>> we
>>>>>>>>>>       >>>>>> do
>>>>>>>>>>       >>>>>>>>> move
>>>>>>>>>>       >>>>>>>>>>>> forward the solution of "stream.branch() 
>>>>>>>>>> returns
>>>>>>>>>>       >>>>>> KBranchedStream",
>>>>>>>>>>       >>>>>>> do
>>>>>>>>>>       >>>>>>>>> we
>>>>>>>>>>       >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>>> KStream[]"?  I
>>>>>>>>> would
>>>>>>>>>>       >>>>> favor
>>>>>>>>>>       >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>>> exclusive APIs
>>>>>>>>> that
>>>>>>>>>>       >>>>>>> accomplish
>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>>> they're fairly
>>>>>>>>>>       similar
>>>>>>>>>>       >>>>>>>>> anyway.  We
>>>>>>>>>>       >>>>>>>>>>>> just need to be sure we're not making 
>>>>>>>>>> something
>>>>>>>>>>       >>>>>>> impossible/difficult
>>>>>>>>>>       >>>>>>>>> that
>>>>>>>>>>       >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>>> structure would
>>>>>>>>> work,
>>>>>>>>>>       >>>>> it's
>>>>>>>>>>       >>>>>>>> just a
>>>>>>>>>>       >>>>>>>>>>>> little sloppy overall in terms of naming and
>>>>>>>>>> clarity. In
>>>>>>>>>>       >>>>>>> particular,
>>>>>>>>>>       >>>>>>>>>>>> passing in the "predicates" and "children"
>>>>>>>>>> lists which
>>>>>>>>> get
>>>>>>>>>>       >>>>>> modified
>>>>>>>>>>       >>>>>>>> in
>>>>>>>>>>       >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>>       KStreamLazyBranch is
>>>>>>>>>>       >>>>> a
>>>>>>>>>>       >>>>>>> bit
>>>>>>>>>>       >>>>>>>>>>>> complicated to follow.
>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>> Thanks,
>>>>>>>>>>       >>>>>>>>>>>> Paul
>>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan 
>>>>>>>>>> Ponomarev <
>>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>       >>>>>>>>> wrote:
>>>>>>>>>>       >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>>>>>       convinced: your
>>>>>>>>>>       >>>>>>>> proposal
>>>>>>>>>>       >>>>>>>>>>>>> looks better and should work. We just have to
>>>>>>>>>> document
>>>>>>>>> the
>>>>>>>>>>       >>>>>> crucial
>>>>>>>>>>       >>>>>>>>> fact
>>>>>>>>>>       >>>>>>>>>>>>> that KStream consumers are invoked as they're
>>>>>>>>>> added.
>>>>>>>>>>       And then
>>>>>>>>>>       >>>>>> it's
>>>>>>>>>>       >>>>>>>> all
>>>>>>>>>>       >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>> What shall we do now? I should re-write the
>>>>>>>>>> KIP and
>>>>>>>>>>       resume the
>>>>>>>>>>       >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>> Why are you telling that your PR 'should not
>>>>>>>>>> be even a
>>>>>>>>>>       >>>>> starting
>>>>>>>>>>       >>>>>>>> point
>>>>>>>>>>       >>>>>>>>> if
>>>>>>>>>>       >>>>>>>>>>>>> we go in this direction'? To me it looks like
>>>>>>>>>> a good
>>>>>>>>>>       starting
>>>>>>>>>>       >>>>>>> point.
>>>>>>>>>>       >>>>>>>>> But
>>>>>>>>>>       >>>>>>>>>>>>> as a novice in this project I might miss some
>>>>>>>>>> important
>>>>>>>>>>       >>>>> details.
>>>>>>>>>>       >>>>>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>> Ivan
>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>       >>>>>>>>>>>>>> Ivan,
>>>>>>>>>>       >>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>> Maybe I’m missing the point, but I 
>>>>>>>>>> believe the
>>>>>>>>>>       >>>>> stream.branch()
>>>>>>>>>>       >>>>>>>>> solution
>>>>>>>>>>       >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>>> consumers will be
>>>>>>>>>>       >>>>> invoked
>>>>>>>>>>       >>>>>> as
>>>>>>>>>>       >>>>>>>>> they’re
>>>>>>>>>>       >>>>>>>>>>>>> added, not during streamsBuilder.build(). So
>>>>>>>>>> the user
>>>>>>>>>>       still
>>>>>>>>>>       >>>>>> ought
>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>       >>>>>>>>> be
>>>>>>>>>>       >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward
>>>>>>>>>> and
>>>>>>>>>>       depend on
>>>>>>>>>>       >>>>> the
>>>>>>>>>>       >>>>>>>>> branched
>>>>>>>>>>       >>>>>>>>>>>>> streams having been set.
>>>>>>>>>>       >>>>>>>>>>>>>> The issue I mean to point out is that it is
>>>>>>>>>> hard to
>>>>>>>>>>       access
>>>>>>>>>>       >>>>> the
>>>>>>>>>>       >>>>>>>>> branched
>>>>>>>>>>       >>>>>>>>>>>>> streams in the same scope as the original
>>>>>>>>>> stream (that
>>>>>>>>>>       is, not
>>>>>>>>>>       >>>>>>>> inside
>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>> couponIssuer), which is a problem with both
>>>>>>>>>> proposed
>>>>>>>>>>       >>>>> solutions.
>>>>>>>>>>       >>>>>> It
>>>>>>>>>>       >>>>>>>>> can be
>>>>>>>>>>       >>>>>>>>>>>>> worked around though.
>>>>>>>>>>       >>>>>>>>>>>>>> [Also, great to hear additional interest in
>>>>>>>>>> 401, I’m
>>>>>>>>>>       excited
>>>>>>>>>>       >>>>> to
>>>>>>>>>>       >>>>>>>> hear
>>>>>>>>>>       >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>       >>>>>>>>>>>>>> Paul
>>>>>>>>>>       >>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan 
>>>>>>>>>> Ponomarev <
>>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>       >>>>>>>>> wrote:
>>>>>>>>>>       >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> The idea to postpone the wiring of branches
>>>>>>>>>> to the
>>>>>>>>>>       >>>>>>>>>>>>> streamsBuilder.build() also looked great for
>>>>>>>>>> me at
>>>>>>>>> first
>>>>>>>>>>       >>>>> glance,
>>>>>>>>>>       >>>>>>> but
>>>>>>>>>>       >>>>>>>>> ---
>>>>>>>>>>       >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>>> available in the
>>>>>>>>>>       same
>>>>>>>>>>       >>>>>> scope
>>>>>>>>>>       >>>>>>> as
>>>>>>>>>>       >>>>>>>>> each
>>>>>>>>>>       >>>>>>>>>>>>> other.  That is, if we wanted to merge 
>>>>>>>>>> them back
>>>>>>>>> together
>>>>>>>>>>       >>>>> again
>>>>>>>>>>       >>>>>> I
>>>>>>>>>>       >>>>>>>>> don't see
>>>>>>>>>>       >>>>>>>>>>>>> a way to do that.
>>>>>>>>>>       >>>>>>>>>>>>>>> You just took the words right out of my
>>>>>>>>>> mouth, I was
>>>>>>>>>>       just
>>>>>>>>>>       >>>>>> going
>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>       >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>       >>>>>>>>>>>>>>> Consider the example from Bill's book, p.
>>>>>>>>>> 101: say
>>>>>>>>>>       we need
>>>>>>>>>>       >>>>> to
>>>>>>>>>>       >>>>>>>>> identify
>>>>>>>>>>       >>>>>>>>>>>>> customers who have bought coffee and made a
>>>>>>>>>> purchase
>>>>>>>>>>       in the
>>>>>>>>>>       >>>>>>>>> electronics
>>>>>>>>>>       >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>       >>>>>>>>>>>>>>> This is the code I usually write under 
>>>>>>>>>> these
>>>>>>>>>>       circumstances
>>>>>>>>>>       >>>>>> using
>>>>>>>>>>       >>>>>>>> my
>>>>>>>>>>       >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>       >>>>>>>>>>>>>>> @Setter
>>>>>>>>>>       >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> 
>>>>>>>>>> electronicsPurchases;
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>       >>>>>>>>>>>>>>>       return
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>       >>>>>>>>>>>>>>>       /*In the real world the code here 
>>>>>>>>>> can be
>>>>>>>>>>       complex, so
>>>>>>>>>>       >>>>>>>>> creation of
>>>>>>>>>>       >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>>> justified, in
>>>>>>>>>>       order to
>>>>>>>>>>       >>>>>>>> separate
>>>>>>>>>>       >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>       >>>>>>>>>>>>>>>  }
>>>>>>>>>>       >>>>>>>>>>>>>>> }
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>>> CouponIssuer();
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>>       >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>       >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to
>>>>>>>>>> wire up
>>>>>>>>>>       everything
>>>>>>>>>>       >>>>>>>> later,
>>>>>>>>>>       >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>       >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>>>>>       initialize the
>>>>>>>>>>       >>>>>>>>> CouponIssuer
>>>>>>>>>>       >>>>>>>>>>>>> we need the terminal operation to be called
>>>>>>>>>> before
>>>>>>>>>>       >>>>>>>>> streamsBuilder.build()
>>>>>>>>>>       >>>>>>>>>>>>> is called.
>>>>>>>>>>       >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>>> KIP-401 is
>>>>>>>>>>       essentially
>>>>>>>>>>       >>>>>> the
>>>>>>>>>>       >>>>>>>>> next
>>>>>>>>>>       >>>>>>>>>>>>> KIP I was going to write here. I have some
>>>>>>>>>> thoughts
>>>>>>>>>>       based on
>>>>>>>>>>       >>>>> my
>>>>>>>>>>       >>>>>>>>> experience,
>>>>>>>>>>       >>>>>>>>>>>>> so I will join the discussion on KIP-401 
>>>>>>>>>> soon.]
>>>>>>>>>>       >>>>>>>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> Ivan
>>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>       >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>       >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>>> concept of a
>>>>>>>>>>       fluent
>>>>>>>>>>       >>>>> API
>>>>>>>>>>       >>>>>>>> based
>>>>>>>>>>       >>>>>>>>>>>>> off of
>>>>>>>>>>       >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>>       (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>>       >>>>>> and
>>>>>>>>>>       >>>>>>> I
>>>>>>>>>>       >>>>>>>>> think
>>>>>>>>>>       >>>>>>>>>>>>> I
>>>>>>>>>>       >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>       >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>>> earlier about
>>>>>>>>>>       >>>>>>> compatibility
>>>>>>>>>>       >>>>>>>>>>>>> issues,
>>>>>>>>>>       >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was
>>>>>>>>>> unaware
>>>>>>>>>>       that Java
>>>>>>>>>>       >>>>> is
>>>>>>>>>>       >>>>>>>> smart
>>>>>>>>>>       >>>>>>>>>>>>> enough to
>>>>>>>>>>       >>>>>>>>>>>>>>>>    distinguish between a 
>>>>>>>>>> branch(varargs...)
>>>>>>>>>>       returning one
>>>>>>>>>>       >>>>>>> thing
>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>       >>>>>>>>>>>>> branch()
>>>>>>>>>>       >>>>>>>>>>>>>>>>    with no arguments returning another 
>>>>>>>>>> thing.
>>>>>>>>>>       >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't
>>>>>>>>>> actually
>>>>>>>>>>       need
>>>>>>>>>>       >>>>> it.
>>>>>>>>>>       >>>>>>> We
>>>>>>>>>>       >>>>>>>>> can
>>>>>>>>>>       >>>>>>>>>>>>> just
>>>>>>>>>>       >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>>> KBranchedStream who
>>>>>>>>>>       shares
>>>>>>>>>>       >>>>>> its
>>>>>>>>>>       >>>>>>>>> state
>>>>>>>>>>       >>>>>>>>>>>>> with the
>>>>>>>>>>       >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do
>>>>>>>>>> the
>>>>>>>>>>       branching.
>>>>>>>>>>       >>>>>>> It's
>>>>>>>>>>       >>>>>>>>> not
>>>>>>>>>>       >>>>>>>>>>>>> terribly
>>>>>>>>>>       >>>>>>>>>>>>>>>>    pretty in its current form, but I 
>>>>>>>>>> think it
>>>>>>>>>>       demonstrates
>>>>>>>>>>       >>>>>> its
>>>>>>>>>>       >>>>>>>>>>>>> feasibility.
>>>>>>>>>>       >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>>> request should
>>>>>>>>> be
>>>>>>>>>>       >>>>> final
>>>>>>>>>>       >>>>>> or
>>>>>>>>>>       >>>>>>>>> even a
>>>>>>>>>>       >>>>>>>>>>>>>>>> starting point if we go in this direction,
>>>>>>>>>> I just
>>>>>>>>>>       wanted to
>>>>>>>>>>       >>>>>> see
>>>>>>>>>>       >>>>>>>> how
>>>>>>>>>>       >>>>>>>>>>>>>>>> challenging it would be to get the API
>>>>>>>>>> working.
>>>>>>>>>>       >>>>>>>>>>>>>>>> I will say though, that I'm not sure the
>>>>>>>>>> existing
>>>>>>>>>>       solution
>>>>>>>>>>       >>>>>>> could
>>>>>>>>>>       >>>>>>>> be
>>>>>>>>>>       >>>>>>>>>>>>>>>> deprecated in favor of this, which I had
>>>>>>>>>> originally
>>>>>>>>>>       >>>>> suggested
>>>>>>>>>>       >>>>>>>> was a
>>>>>>>>>>       >>>>>>>>>>>>>>>> possibility.  The reason is that the newly
>>>>>>>>>> branched
>>>>>>>>>>       streams
>>>>>>>>>>       >>>>>> are
>>>>>>>>>>       >>>>>>>> not
>>>>>>>>>>       >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>>> other.  That
>>>>>>>>>>       is, if we
>>>>>>>>>>       >>>>>>> wanted
>>>>>>>>>>       >>>>>>>>> to
>>>>>>>>>>       >>>>>>>>>>>>> merge
>>>>>>>>>>       >>>>>>>>>>>>>>>> them back together again I don't see a way
>>>>>>>>>> to do
>>>>>>>>>>       that.  The
>>>>>>>>>>       >>>>>> KIP
>>>>>>>>>>       >>>>>>>>>>>>> proposal
>>>>>>>>>>       >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>>> means is that
>>>>>>>>> for
>>>>>>>>>>       >>>>>> either
>>>>>>>>>>       >>>>>>>>>>>>> solution,
>>>>>>>>>>       >>>>>>>>>>>>>>>> deprecating the existing branch(...) is
>>>>>>>>>> not on the
>>>>>>>>>>       table.
>>>>>>>>>>       >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>       >>>>>>>>>>>>>>>> Paul
>>>>>>>>>>       >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan
>>>>>>>>>> Ponomarev <
>>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>>>>>       >>>>>>>>>>>>> wrote:
>>>>>>>>>>       >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>>> discussed up to
>>>>>>>>> this
>>>>>>>>>>       >>>>>> point.
>>>>>>>>>>       >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed
>>>>>>>>>> that
>>>>>>>>>>       branch API
>>>>>>>>>>       >>>>>>> needs
>>>>>>>>>>       >>>>>>>>>>>>>>>>> improvement. Motivation is given in 
>>>>>>>>>> the KIP.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...)....
>>>>>>>>>> //onTopOf
>>>>>>>>>>       returns
>>>>>>>>>>       >>>>>> its
>>>>>>>>>>       >>>>>>>>> argument
>>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2)
>>>>>>>>>> The code
>>>>>>>>> won't
>>>>>>>>>>       >>>>> make
>>>>>>>>>>       >>>>>>>> sense
>>>>>>>>>>       >>>>>>>>>>>>> until
>>>>>>>>>>       >>>>>>>>>>>>>>>>> all the necessary ingredients are 
>>>>>>>>>> provided.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>       instance
>>>>>>>>>>       >>>>>>>>> contrasts the
>>>>>>>>>>       >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> stream
>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>>> noDefault(). Both
>>>>>>>>>>       >>>>>>>>> defaultBranch(..)
>>>>>>>>>>       >>>>>>>>>>>>> and
>>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams
>>>>>>>>>> interface
>>>>>>>>> is
>>>>>>>>>>       >>>>>> defined.
>>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: We need to define two terminal 
>>>>>>>>>> methods
>>>>>>>>>>       >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very
>>>>>>>>>> easy to
>>>>>>>>>>       miss the
>>>>>>>>>>       >>>>>> fact
>>>>>>>>>>       >>>>>>>>> that one
>>>>>>>>>>       >>>>>>>>>>>>>>>>> of the terminal methods should be called.
>>>>>>>>>> If these
>>>>>>>>>>       methods
>>>>>>>>>>       >>>>>> are
>>>>>>>>>>       >>>>>>>> not
>>>>>>>>>>       >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>>> runtime.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can
>>>>>>>>>> we do
>>>>>>>>> better?
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> I see your point when you are talking
>>>>>>>>>> about
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot 
>>>>>>>>>> not be
>>>>>>>>>>       implemented the
>>>>>>>>>>       >>>>>>> easy
>>>>>>>>>>       >>>>>>>>> way.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> user could specify a terminal 
>>>>>>>>>> method that
>>>>>>>>> assumes
>>>>>>>>>>       >>>>> nothing
>>>>>>>>>>       >>>>>>>> will
>>>>>>>>>>       >>>>>>>>>>>>> reach
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>>> occurs.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be
>>>>>>>>>> the only
>>>>>>>>> option
>>>>>>>>>>       >>>>>> besides
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios
>>>>>>>>>> when we
>>>>>>>>>>       want to
>>>>>>>>>>       >>>>>> just
>>>>>>>>>>       >>>>>>>>> silently
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>>>>>       predicate. 2)
>>>>>>>>>>       >>>>>>> Throwing
>>>>>>>>>>       >>>>>>>>> an
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> exception in the middle of data flow
>>>>>>>>>> processing
>>>>>>>>>>       looks
>>>>>>>>>>       >>>>>> like a
>>>>>>>>>>       >>>>>>>> bad
>>>>>>>>>>       >>>>>>>>>>>>> idea.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would
>>>>>>>>>> prefer to
>>>>>>>>>>       emit a
>>>>>>>>>>       >>>>>>>> special
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is
>>>>>>>>>> exactly
>>>>>>>>> where
>>>>>>>>>>       >>>>>>> `default`
>>>>>>>>>>       >>>>>>>>> can
>>>>>>>>>>       >>>>>>>>>>>>> be
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>       >>>>> InternalTopologyBuilder
>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>       >>>>>>>>> track
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>>> and raise
>>>>>>>>>>       a clear
>>>>>>>>>>       >>>>>>> error
>>>>>>>>>>       >>>>>>>>>>>>> before it
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the
>>>>>>>>>> program is
>>>>>>>>>>       >>>>> compiled
>>>>>>>>>>       >>>>>>> and
>>>>>>>>>>       >>>>>>>>> run?
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply 
>>>>>>>>>> won't
>>>>>>>>>>       compile if
>>>>>>>>>>       >>>>> used
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an 
>>>>>>>>>> API as a
>>>>>>>>>>       method chain
>>>>>>>>>>       >>>>>>>> starting
>>>>>>>>>>       >>>>>>>>>>>>> from
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost
>>>>>>>>>> difference
>>>>>>>>>>       between
>>>>>>>>>>       >>>>>>>> runtime
>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure
>>>>>>>>>> uncovers
>>>>>>>>>>       >>>>> instantly
>>>>>>>>>>       >>>>>> on
>>>>>>>>>>       >>>>>>>>> unit
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> tests, it costs more for the project
>>>>>>>>>> than a
>>>>>>>>>>       compilation
>>>>>>>>>>       >>>>>>>> failure.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>>> operation being
>>>>>>>>>>       required.
>>>>>>>>>>       >>>>>>> But
>>>>>>>>>>       >>>>>>>> is
>>>>>>>>>>       >>>>>>>>>>>>> that
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't
>>>>>>>>>> want a
>>>>>>>>>>       >>>>>> defaultBranch
>>>>>>>>>>       >>>>>>>>> they
>>>>>>>>>>       >>>>>>>>>>>>> can
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>>       just as
>>>>>>>>>>       >>>>>>>>> easily.  In
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a
>>>>>>>>>> nicer API
>>>>>>>>> - a
>>>>>>>>>>       >>>>> user
>>>>>>>>>>       >>>>>>>> could
>>>>>>>>>>       >>>>>>>>>>>>> specify
>>>>>>>>>>       >>>>>>>>>>>>>>>>> a
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing
>>>>>>>>>> will reach
>>>>>>>>> the
>>>>>>>>>>       >>>>>> default
>>>>>>>>>>       >>>>>>>>> branch,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>>> occurs.
>>>>>>>>> That
>>>>>>>>>>       >>>>> seems
>>>>>>>>>>       >>>>>>> like
>>>>>>>>>>       >>>>>>>>> an
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> improvement over the current branch()
>>>>>>>>>> API,
>>>>>>>>>>       which allows
>>>>>>>>>>       >>>>>> for
>>>>>>>>>>       >>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>> more
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly 
>>>>>>>>>> getting
>>>>>>>>> dropped.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>>> certainly has
>>>>>>>>>>       to be
>>>>>>>>>>       >>>>>> well
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>>       >>>>> InternalTopologyBuilder
>>>>>>>>>>       >>>>>>> to
>>>>>>>>>>       >>>>>>>>> track
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>>> and raise
>>>>>>>>>>       a clear
>>>>>>>>>>       >>>>>>> error
>>>>>>>>>>       >>>>>>>>>>>>> before it
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that
>>>>>>>>>> there is
>>>>>>>>> a
>>>>>>>>>>       >>>>> "build
>>>>>>>>>>       >>>>>>>> step"
>>>>>>>>>>       >>>>>>>>>>>>> where
>>>>>>>>>>       >>>>>>>>>>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>>>>>       >>>>>> StreamsBuilder.build()
>>>>>>>>>>       >>>>>>> is
>>>>>>>>>>       >>>>>>>>>>>>> called.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>>> argument, I
>>>>>>>>> agree
>>>>>>>>>>       >>>>> that
>>>>>>>>>>       >>>>>>> it's
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> allow users to do other operations on
>>>>>>>>>> the input
>>>>>>>>>>       stream.
>>>>>>>>>>       >>>>>>> With
>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same
>>>>>>>>>> way all
>>>>>>>>> other
>>>>>>>>>>       >>>>>>> operations
>>>>>>>>>>       >>>>>>>>> do -
>>>>>>>>>>       >>>>>>>>>>>>> if
>>>>>>>>>>       >>>>>>>>>>>>>>>>> you
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> want to process off the original 
>>>>>>>>>> KStream
>>>>>>>>> multiple
>>>>>>>>>>       >>>>> times,
>>>>>>>>>>       >>>>>>> you
>>>>>>>>>>       >>>>>>>>> just
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call
>>>>>>>>>> as many
>>>>>>>>>>       operations
>>>>>>>>>>       >>>>>> on
>>>>>>>>>>       >>>>>>> it
>>>>>>>>>>       >>>>>>>>> as
>>>>>>>>>>       >>>>>>>>>>>>> you
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan
>>>>>>>>>> Ponomarev <
>>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we
>>>>>>>>>> do not
>>>>>>>>>>       always need
>>>>>>>>>>       >>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>>>>>> operation we
>>>>>>>>>>       >>>>> don't
>>>>>>>>>>       >>>>>>>> know
>>>>>>>>>>       >>>>>>>>>>>>> when to
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch 
>>>>>>>>>> switch'.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its
>>>>>>>>>> argument,
>>>>>>>>>>       so we
>>>>>>>>>>       >>>>> can
>>>>>>>>>>       >>>>>> do
>>>>>>>>>>       >>>>>>>>>>>>> something
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> more with the original branch after
>>>>>>>>>> branching.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I understand your point that the 
>>>>>>>>>> need of
>>>>>>>>> special
>>>>>>>>>>       >>>>> object
>>>>>>>>>>       >>>>>>>>>>>>> construction
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream
>>>>>>>>>> methods.
>>>>>>>>> But
>>>>>>>>>>       >>>>> here
>>>>>>>>>>       >>>>>> we
>>>>>>>>>>       >>>>>>>>> have a
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to
>>>>>>>>>> split the
>>>>>>>>>>       flow,
>>>>>>>>>>       >>>>> so
>>>>>>>>>>       >>>>>> I
>>>>>>>>>>       >>>>>>>>> think
>>>>>>>>>>       >>>>>>>>>>>>> this
>>>>>>>>>>       >>>>>>>>>>>>>>>>> is
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve
>>>>>>>>>> this
>>>>>>>>>>       API, but I
>>>>>>>>>>       >>>>>> find
>>>>>>>>>>       >>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing 
>>>>>>>>>> since it
>>>>>>>>>>       contrasts the
>>>>>>>>>>       >>>>>>> fluency
>>>>>>>>>>       >>>>>>>>> of
>>>>>>>>>>       >>>>>>>>>>>>> other
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd
>>>>>>>>>> like to
>>>>>>>>>>       just call
>>>>>>>>>>       >>>>> a
>>>>>>>>>>       >>>>>>>>> method on
>>>>>>>>>>       >>>>>>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if
>>>>>>>>>> the branch
>>>>>>>>>>       cases
>>>>>>>>>>       >>>>> are
>>>>>>>>>>       >>>>>>>>> defined
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate,
>>>>>>>>>> handleCase)
>>>>>>>>>>       is very
>>>>>>>>>>       >>>>>> nice
>>>>>>>>>>       >>>>>>>>> and the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped
>>>>>>>>>> around
>>>>>>>>>>       how we
>>>>>>>>>>       >>>>>>> specify
>>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>> source
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>>>>>> this::handle1)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>>>>>> this::handle2)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>>> KBranchedStreams or
>>>>>>>>>>       >>>>>>>> KStreamBrancher
>>>>>>>>>>       >>>>>>>>> or
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>>>>>       terminated by
>>>>>>>>>>       >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>>>>>       incompatible with
>>>>>>>>>>       >>>>> the
>>>>>>>>>>       >>>>>>>>> current
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to
>>>>>>>>>> have a
>>>>>>>>>>       different
>>>>>>>>>>       >>>>>> name,
>>>>>>>>>>       >>>>>>>> but
>>>>>>>>>>       >>>>>>>>> that
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we
>>>>>>>>>> could call it
>>>>>>>>>>       >>>>>> something
>>>>>>>>>>       >>>>>>>> like
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the
>>>>>>>>>> old API.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of
>>>>>>>>>> your
>>>>>>>>>>       KIP?  It
>>>>>>>>>>       >>>>>> seems
>>>>>>>>>>       >>>>>>>>> like it
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>>> branching
>>>>>>>>>>       while also
>>>>>>>>>>       >>>>>>>> allowing
>>>>>>>>>>       >>>>>>>>> you
>>>>>>>>>>       >>>>>>>>>>>>> to
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>>>>>       KBranchedStreams
>>>>>>>>>>       >>>>>> if
>>>>>>>>>>       >>>>>>>>> desired.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan
>>>>>>>>>> Ponomarev
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>>       ks){
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void 
>>>>>>>>>> handleSecondCase(KStream<String,
>>>>>>>>>>       String> ks){
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String,
>>>>>>>>>> String>()
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>>       this::handleFirstCase)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>>       this::handleSecondCase)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>>       >>>>> takes a
>>>>>>>>>>       >>>>>>>>> Consumer
>>>>>>>>>>       >>>>>>>>>>>>> as a
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing,
>>>>>>>>>> and the
>>>>>>>>>>       example in
>>>>>>>>>>       >>>>>> the
>>>>>>>>>>       >>>>>>>> KIP
>>>>>>>>>>       >>>>>>>>>>>>> shows
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a
>>>>>>>>>> terminal node
>>>>>>>>>>       >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but
>>>>>>>>>> how would
>>>>>>>>> we
>>>>>>>>>>       >>>>> handle
>>>>>>>>>>       >>>>>>> the
>>>>>>>>>>       >>>>>>>>> case
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>>> wants to
>>>>>>>>> continue
>>>>>>>>>>       >>>>>>>> processing
>>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>>       >>>>>>>>>>>>> not
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on
>>>>>>>>>> the
>>>>>>>>> branched
>>>>>>>>>>       >>>>>> stream
>>>>>>>>>>       >>>>>>>>>>>>> immediately?
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic
>>>>>>>>>> as is if
>>>>>>>>>>       we had
>>>>>>>>>>       >>>>>>>> something
>>>>>>>>>>       >>>>>>>>> like
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] 
>>>>>>>>>> branches =
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM
>>>>>>>>>> Bill Bejeck
>>>>>>>>> <
>>>>>>>>>>       >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>>> discussion for
>>>>>>>>> KIP-
>>>>>>>>>>       >>>>> 418.
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion 
>>>>>>>>>> about
>>>>>>>>> KIP-418.
>>>>>>>>>>       >>>>> Please
>>>>>>>>>>       >>>>>>>> take
>>>>>>>>>>       >>>>>>>>> a
>>>>>>>>>>       >>>>>>>>>>>>> look
>>>>>>>>>>       >>>>>>>>>>>>>>>>> at
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>>> appreciate any
>>>>>>>>>>       feedback :)
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>       >>>>>
>>>>>>>>>>
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream 
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>       >>>>>>>>>>>>> 
>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>>       >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>       >>>>>>>>>
>>>>>>>>>>       >
>>>>>>>>>>
>>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 



Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi,

I have read the John's "DSL design principles" and have completely 
rewritten the KIP, see 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream


This version includes all the previous discussion results and follows 
the design principles, with one exception.

The exception is

branch(Predicate<K,V> predicate, Branched<K,V> branched)

which formally violates 'no more than one parameter' rule, but I think 
here it is justified.

We must provide a predicate for a branch and don't need to provide one 
for the default branch. Thus for both operations we may use a single 
Branched parameter class, with an extra method parameter for `branch`.

Since predicate is a natural, necessary part of a branch, no 
'proliferation of overloads, deprecations, etc.' is expected here as it 
is said in the rationale for the 'single parameter rule'.

WDYT, is this KIP mature enough to begin voting?

Regards,

Ivan

21.04.2020 2:09, Matthias J. Sax пишет:
> Ivan,
> 
> no worries about getting side tracked. Glad to have you back!
> 
> The DSL improved further in the meantime and we already have a `Named`
> config object to name operators. It seems reasonable to me to build on this.
> 
> Furthermore, John did a writeup about "DSL design principles" that we
> want to follow:
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
> -- might be worth to checkout.
> 
> 
> -Matthias
> 
> 
> On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
>> Hi everyone!
>>
>> Let me revive the discussion of this KIP.
>>
>> I'm very sorry for stopping my participation in the discussion in June
>> 2019. My project work was very intensive then and it didn't leave me
>> spare time. But I think I must finish this, because we invested
>> substantial effort into this discussion and I'm not feel entitled to
>> propose other things before this one is finalized.
>>
>> During these months I proceeded with writing and reviewing Kafka
>> Streams-related code. Every time I needed branching, Spring-Kafka's
>> KafkaStreamBrancher class of my invention (the original idea for this
>> KIP) worked for me -- that's another reason why I gave up pushing the
>> KIP forward. When I was coming across the problem with the scope of
>> branches, I worked around it this way:
>>
>> AtomicReference<KStream<...>> result = new AtomicReference<>();
>> new KafkaStreamBrancher<....>()
>>      .branch(....)
>>      .defaultBranch(result::set)
>>      .onTopOf(someStream);
>> result.get()...
>>
>>
>> And yes, of course I don't feel very happy with this approach.
>>
>> I think that Matthias came up with a bright solution in his post from
>> May, 24th 2019. Let me quote it:
>>
>> KStream#split() -> KBranchedStream
>> // branch is not easily accessible in current scope
>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>    -> KBranchedStream
>> // assign a name to the branch and
>> // return the sub-stream to the current scope later
>> //
>> // can be simple as `#branch(p, s->s, "name")`
>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>    -> KBranchedStream
>> // default branch is not easily accessible
>> // return map of all named sub-stream into current scope
>> KBranchedStream#default(Cosumer<KStream>)
>>    -> Map<String,KStream>
>> // assign custom name to default-branch
>> // return map of all named sub-stream into current scope
>> KBranchedStream#default(Function<KStream,KStream>, String)
>>    -> Map<String,KStream>
>> // assign a default name for default
>> // return map of all named sub-stream into current scope
>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>    -> Map<String,KStream>
>> // return map of all names sub-stream into current scope
>> KBranchedStream#noDefaultBranch()
>>    -> Map<String,KStream>
>>
>> I believe this would satisfy everyone. Optional names seems to be a good
>> idea: when you don't need to have the branches in the same scope, you
>> just don't use names and you don't risk making your code brittle. Or,
>> you might want to add names just for debugging purposes. Or, finally,
>> you might use the returned Map to have the named branches in the
>> original scope.
>>
>> There also was an input from John Roesler on June 4th, 2019, who
>> suggested using Named class. I can't comment on this. The idea seems
>> reasonable, but in this matter I'd rather trust people who are more
>> familiar with Streams API design principles than me.
>>
>> Regards,
>>
>> Ivan
>>
>>
>>
>> 08.10.2019 1:38, Matthias J. Sax пишет:
>>> I am moving this KIP into "inactive status". Feel free to resume the KIP
>>> at any point.
>>>
>>> If anybody else is interested in picking up this KIP, feel free to do so.
>>>
>>>
>>>
>>> -Matthias
>>>
>>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>>> Ivan,
>>>>
>>>> did you see my last reply? What do you think about my proposal to mix
>>>> both approaches and try to get best-of-both worlds?
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>>> Thanks for the input John!
>>>>>
>>>>>> under your suggestion, it seems that the name is required
>>>>>
>>>>> If you want to get the `KStream` as part of the `Map` back using a
>>>>> `Function`, yes. If you follow the "embedded chaining" pattern using a
>>>>> `Consumer`, no.
>>>>>
>>>>> Allowing for a default name via `split()` can of course be done.
>>>>> Similarly, using `Named` instead of `String` is possible.
>>>>>
>>>>> I wanted to sketch out a high level proposal to merge both patterns
>>>>> only. Your suggestions to align the new API with the existing API make
>>>>> totally sense.
>>>>>
>>>>>
>>>>>
>>>>> One follow up question: Would `Named` be optional or required in
>>>>> `split()` and `branch()`? It's unclear from your example.
>>>>>
>>>>> If both are mandatory, what do we gain by it? The returned `Map` only
>>>>> contains the corresponding branches, so why should we prefix all of
>>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>>> `split()`, the same question raises?
>>>>>
>>>>> Requiring `Named` in `split()` seems only to make sense, if `Named` is
>>>>> optional in `branch()` and we generate `-X` suffix using a counter for
>>>>> different branch name. However, this might lead to the problem of
>>>>> changing names if branches are added/removed. Also, how would the names
>>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>>> returned in the `Map`).
>>>>>
>>>>> If `Named` is optional for both, it could happen that a user misses to
>>>>> specify a name for a branch what would lead to runtime issues.
>>>>>
>>>>>
>>>>> Hence, I am actually in favor to not allow a default name but keep
>>>>> `split()` without parameter and make `Named` in `branch()` required
>>>>> if a
>>>>> `Function` is used. This makes it explicit to the user that
>>>>> specifying a
>>>>> name is required if a `Function` is used.
>>>>>
>>>>>
>>>>>
>>>>> About
>>>>>
>>>>>> KBranchedStream#branch(BranchConfig)
>>>>>
>>>>> I don't think that the branching predicate is a configuration and hence
>>>>> would not include it in a configuration object.
>>>>>
>>>>>>       withChain(...);
>>>>>
>>>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>>>>> seem to be a configuration. We can also not prevent a user to call
>>>>> `withName()` in combination of `withChain()` what does not make sense
>>>>> IMHO. We could of course throw an RTE but not have a compile time check
>>>>> seems less appealing. Also, it could happen that neither `withChain()`
>>>>> not `withName()` is called and the branch is missing in the returned
>>>>> `Map` what lead to runtime issues, too.
>>>>>
>>>>> Hence, I don't think that we should add `BranchConfig`. A config object
>>>>> is helpful if each configuration can be set independently of all
>>>>> others,
>>>>> but this seems not to be the case here. If we add new configuration
>>>>> later, we can also just move forward by deprecating the methods that
>>>>> accept `Named` and add new methods that accepted `BranchConfig` (that
>>>>> would of course implement `Named`).
>>>>>
>>>>>
>>>>> Thoughts?
>>>>>
>>>>>
>>>>> @Ivan, what do you think about the general idea to blend the two main
>>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>>
>>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>>>>> everyone. Returning the map from the terminal operations also solves
>>>>>> the problem of merging/joining the branched streams, if we want to add
>>>>>> support for the compliment later on.
>>>>>>
>>>>>> Under your suggestion, it seems that the name is required. Otherwise,
>>>>>> we wouldn't have keys for the map to return. I this this is actually
>>>>>> not too bad, since experience has taught us that, although names for
>>>>>> operations are not required to define stream processing logic, it does
>>>>>> significantly improve the operational experience when you can map the
>>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>>> wouldn't (have to) reference the name to chain extra processing onto
>>>>>> the branch (thanks to the second argument), you can avoid the
>>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>>
>>>>>> In the current implementation of Branch, you can name the branch
>>>>>> operator itself, and then all the branches get index-suffixed names
>>>>>> built from the branch operator name. I guess under this proposal, we
>>>>>> could naturally append the branch name to the branching operator name,
>>>>>> like this:
>>>>>>
>>>>>>      stream.split(Named.withName("mysplit")) //creates node "mysplit"
>>>>>>                 .branch(..., ..., "abranch") // creates node
>>>>>> "mysplit-abranch"
>>>>>>                 .defaultBranch(...) // creates node "mysplit-default"
>>>>>>
>>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>>
>>>>>> We don't have a defined grammar, so there's plenty of room to debate
>>>>>> the "best" syntax in the context of each operation, but in general,
>>>>>> the KStream DSL operators follow this pattern:
>>>>>>
>>>>>>       operator(function, config_object?) OR operator(config_object)
>>>>>>
>>>>>> where config_object is often just Named in the "function" variant.
>>>>>> Even when the config_object isn't a Named, but some other config
>>>>>> class, that config class _always_ implements NamedOperation.
>>>>>>
>>>>>> Here, we're introducing a totally different pattern:
>>>>>>
>>>>>>     operator(function, function, string)
>>>>>>
>>>>>> where the string is the name.
>>>>>> My first question is whether the name should instead be specified with
>>>>>> the NamedOperation interface.
>>>>>>
>>>>>> My second question is whether we should just roll all these arguments
>>>>>> up into a config object like:
>>>>>>
>>>>>>      KBranchedStream#branch(BranchConfig)
>>>>>>
>>>>>>      interface BranchConfig extends NamedOperation {
>>>>>>       withPredicate(...);
>>>>>>       withChain(...);
>>>>>>       withName(...);
>>>>>>     }
>>>>>>
>>>>>> Although I guess we'd like to call BranchConfig something more like
>>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>>
>>>>>> This makes the source code a little noisier, but it also makes us more
>>>>>> future-proof, as we can deal with a wide range of alternatives purely
>>>>>> in the config interface, and never have to deal with adding overloads
>>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>>> optional, or the KStream->KStream to be optional.
>>>>>>
>>>>>> WDYT?
>>>>>>
>>>>>> Thanks,
>>>>>> -John
>>>>>>
>>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>>> <mi...@confluent.io> wrote:
>>>>>>>
>>>>>>> Matthias: I think that's pretty reasonable from my point of view.
>>>>>>> Good
>>>>>>> suggestion.
>>>>>>>
>>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>>> <ma...@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Interesting discussion.
>>>>>>>>
>>>>>>>> I am wondering, if we cannot unify the advantage of both approaches:
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>>
>>>>>>>> // branch is not easily accessible in current scope
>>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>>     -> KBranchedStream
>>>>>>>>
>>>>>>>> // assign a name to the branch and
>>>>>>>> // return the sub-stream to the current scope later
>>>>>>>> //
>>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>>     -> KBranchedStream
>>>>>>>>
>>>>>>>> // default branch is not easily accessible
>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>>     -> Map<String,KStream>
>>>>>>>>
>>>>>>>> // assign custom name to default-branch
>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>>     -> Map<String,KStream>
>>>>>>>>
>>>>>>>> // assign a default name for default
>>>>>>>> // return map of all named sub-stream into current scope
>>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>>     -> Map<String,KStream>
>>>>>>>>
>>>>>>>> // return map of all names sub-stream into current scope
>>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>>     -> Map<String,KStream>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Hence, for each sub-stream, the user can pick to add a name and
>>>>>>>> return
>>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>>> implementation can
>>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>>> returned
>>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>>
>>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>>
>>>>>>>> Thoughts?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>>> Ivan,
>>>>>>>>>
>>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>>> dynamic case.
>>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>>> necessity.
>>>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>>
>>>>>>>>> Separately, I'm interested to see where the present discussion
>>>>>>>>> leads.
>>>>>>>>> I've written enough Javascript code in my life to be suspicious of
>>>>>>>>> nested closures. You have a good point about using method
>>>>>>>>> references (or
>>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>>> that this
>>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>>> logic when
>>>>>>>>> their nested closure situation got out of hand. Unfortunately, it's
>>>>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>>>>> readability (but in different ways for different reasons). In other
>>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>>> solution if
>>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>>
>>>>>>>>> However, the history of JS also tells us that function
>>>>>>>>> references aren't
>>>>>>>>> the end of the story either, and you can see that by observing that
>>>>>>>>> there have been two follow-on eras, as they continue trying to
>>>>>>>>> cope with
>>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>>> First, you
>>>>>>>>> have Futures/Promises, which essentially let you convert nested
>>>>>>>>> code to
>>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>>> this).
>>>>>>>>> Most lately, you have async/await, which is an effort to apply
>>>>>>>>> language
>>>>>>>>> (not just API) syntax to the problem, and offer the "flattest"
>>>>>>>>> possible
>>>>>>>>> programming style to solve the problem (because you get back to
>>>>>>>>> just one
>>>>>>>>> code block per functional unit).
>>>>>>>>>
>>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>>> nowhere
>>>>>>>>> near as callback heavy as JS, so I don't think we have to take
>>>>>>>>> the JS
>>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>>> valuable
>>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>>> bringing this
>>>>>>>>> up to inspire further/deeper discussion. At the same time, just
>>>>>>>>> like JS,
>>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>>
>>>>>>>>> Separately again, I'm interested in the post-branch merge (and
>>>>>>>>> I'd also
>>>>>>>>> add join) problem that Paul brought up. We can clearly punt on
>>>>>>>>> it, by
>>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>>> there a DSL
>>>>>>>>> way to do it?
>>>>>>>>>
>>>>>>>>> Thanks again for your driving this,
>>>>>>>>> -John
>>>>>>>>>
>>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>>
>>>>>>>>>       Ivan, I’ll definitely forfeit my point on the clumsiness of
>>>>>>>>> the
>>>>>>>>>       branch(predicate, consumer) solution, I don’t see any real
>>>>>>>>> drawbacks
>>>>>>>>>       for the dynamic case.
>>>>>>>>>
>>>>>>>>>       IMO the one trade off to consider at this point is the scope
>>>>>>>>>       question. I don’t know if I totally agree that “we rarely
>>>>>>>>> need them
>>>>>>>>>       in the same scope” since merging the branches back together
>>>>>>>>> later
>>>>>>>>>       seems like a perfectly plausible use case that can be a lot
>>>>>>>>> nicer
>>>>>>>>>       when the branched streams are in the same scope. That being
>>>>>>>>> said,
>>>>>>>>>       for the reasons Ivan listed, I think it is overall the better
>>>>>>>>>       solution - working around the scope thing is easy enough if
>>>>>>>>> you need
>>>>>>>>>       to.
>>>>>>>>>
>>>>>>>>>       > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>>       <ip...@mail.ru.invalid> wrote:
>>>>>>>>>       >
>>>>>>>>>       > Hello everyone, thank you all for joining the discussion!
>>>>>>>>>       >
>>>>>>>>>       > Well, I don't think the idea of named branches, be it a
>>>>>>>>>       LinkedHashMap (no other Map will do, because order of
>>>>>>>>> definition
>>>>>>>>>       matters) or `branch` method  taking name and Consumer has more
>>>>>>>>>       advantages than drawbacks.
>>>>>>>>>       >
>>>>>>>>>       > In my opinion, the only real positive outcome from Michael's
>>>>>>>>>       proposal is that all the returned branches are in the same
>>>>>>>>> scope.
>>>>>>>>>       But 1) we rarely need them in the same scope 2) there is a
>>>>>>>>>       workaround for the scope problem, described in the KIP.
>>>>>>>>>       >
>>>>>>>>>       > 'Inlining the complex logic' is not a problem, because we
>>>>>>>>> can use
>>>>>>>>>       method references instead of lambdas. In real world
>>>>>>>>> scenarios you
>>>>>>>>>       tend to split the complex logic to methods anyway, so the
>>>>>>>>> code is
>>>>>>>>>       going to be clean.
>>>>>>>>>       >
>>>>>>>>>       > The drawbacks are strong. The cohesion between predicates
>>>>>>>>> and
>>>>>>>>>       handlers is lost. We have to define predicates in one
>>>>>>>>> place, and
>>>>>>>>>       handlers in another. This opens the door for bugs:
>>>>>>>>>       >
>>>>>>>>>       > - what if we forget to define a handler for a name? or a
>>>>>>>>> name for
>>>>>>>>>       a handler?
>>>>>>>>>       > - what if we misspell a name?
>>>>>>>>>       > - what if we copy-paste and duplicate a name?
>>>>>>>>>       >
>>>>>>>>>       > What Michael propose would have been totally OK if we had
>>>>>>>>> been
>>>>>>>>>       writing the API in Lua, Ruby or Python. In those languages the
>>>>>>>>>       "dynamic naming" approach would have looked most concise and
>>>>>>>>>       beautiful. But in Java we expect all the problems related to
>>>>>>>>>       identifiers to be eliminated in compile time.
>>>>>>>>>       >
>>>>>>>>>       > Do we have to invent duck-typing for the Java API?
>>>>>>>>>       >
>>>>>>>>>       > And if we do, what advantage are we supposed to get
>>>>>>>>> besides having
>>>>>>>>>       all the branches in the same scope? Michael, maybe I'm
>>>>>>>>> missing your
>>>>>>>>>       point?
>>>>>>>>>       >
>>>>>>>>>       > ---
>>>>>>>>>       >
>>>>>>>>>       > Earlier in this discussion John Roesler also proposed to do
>>>>>>>>>       without "start branching" operator, and later Paul
>>>>>>>>> mentioned that in
>>>>>>>>>       the case when we have to add a dynamic number of branches, the
>>>>>>>>>       current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>>> solution. Let
>>>>>>>>>       me address both comments here.
>>>>>>>>>       >
>>>>>>>>>       > 1) "Start branching" operator (I think that *split* is a
>>>>>>>>> good name
>>>>>>>>>       for it indeed) is critical when we need to do a dynamic
>>>>>>>>> branching,
>>>>>>>>>       see example below.
>>>>>>>>>       >
>>>>>>>>>       > 2) No, dynamic branching in current KIP is not clumsy at
>>>>>>>>> all.
>>>>>>>>>       Imagine a real-world scenario when you need one branch per
>>>>>>>>> enum
>>>>>>>>>       value (say, RecordType). You can have something like this:
>>>>>>>>>       >
>>>>>>>>>       > /*John:if we had to start with stream.branch(...) here,
>>>>>>>>> it would
>>>>>>>>>       have been much messier.*/
>>>>>>>>>       > KBranchedStream branched = stream.split();
>>>>>>>>>       >
>>>>>>>>>       > /*Not clumsy at all :-)*/
>>>>>>>>>       > for (RecordType recordType : RecordType.values())
>>>>>>>>>       >             branched = branched.branch((k, v) ->
>>>>>>>>> v.getRecType() ==
>>>>>>>>>       recordType,
>>>>>>>>>       >                     recordType::processRecords);
>>>>>>>>>       >
>>>>>>>>>       > Regards,
>>>>>>>>>       >
>>>>>>>>>       > Ivan
>>>>>>>>>       >
>>>>>>>>>       >
>>>>>>>>>       > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>>       >> I also agree with Michael's observation about the core
>>>>>>>>> problem of
>>>>>>>>>       >> current `branch()` implementation.
>>>>>>>>>       >>
>>>>>>>>>       >> However, I also don't like to pass in a clumsy Map
>>>>>>>>> object. My
>>>>>>>>>       thinking
>>>>>>>>>       >> was more aligned with Paul's proposal to just add a name
>>>>>>>>> to each
>>>>>>>>>       >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>>>>       >>
>>>>>>>>>       >> It makes the code easier to read, and also make the
>>>>>>>>> order of
>>>>>>>>>       >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>>       >>
>>>>>>>>>       >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>       >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>       >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>       >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>>       >> An open question is the case for which no
>>>>>>>>> defaultBranch() should
>>>>>>>> be
>>>>>>>>>       >> specified. Atm, `split()` and `branch()` would return
>>>>>>>>>       `BranchedKStream`
>>>>>>>>>       >> and the call to `defaultBranch()` that returns the `Map` is
>>>>>>>> mandatory
>>>>>>>>>       >> (what is not the case atm). Or is this actually not a real
>>>>>>>> problem,
>>>>>>>>>       >> because users can just ignore the branch returned by
>>>>>>>>>       `defaultBranch()`
>>>>>>>>>       >> in the result `Map` ?
>>>>>>>>>       >>
>>>>>>>>>       >>
>>>>>>>>>       >> About "inlining": So far, it seems to be a matter of
>>>>>>>>> personal
>>>>>>>>>       >> preference. I can see arguments for both, but no "killer
>>>>>>>>>       argument" yet
>>>>>>>>>       >> that clearly make the case for one or the other.
>>>>>>>>>       >>
>>>>>>>>>       >>
>>>>>>>>>       >> -Matthias
>>>>>>>>>       >>
>>>>>>>>>       >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>>       >>> Perhaps inlining is the wrong terminology. It doesn’t
>>>>>>>>> require
>>>>>>>>>       that a lambda with the full downstream topology be defined
>>>>>>>>> inline -
>>>>>>>>>       it can be a method reference as with Ivan’s original
>>>>>>>>> suggestion.
>>>>>>>>>       The advantage of putting the predicate and its downstream
>>>>>>>>> logic
>>>>>>>>>       (Consumer) together in branch() is that they are required
>>>>>>>>> to be near
>>>>>>>>>       to each other.
>>>>>>>>>       >>>
>>>>>>>>>       >>> Ultimately the downstream code has to live somewhere,
>>>>>>>>> and deep
>>>>>>>>>       branch trees will be hard to read regardless.
>>>>>>>>>       >>>
>>>>>>>>>       >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>>       <michael.drogalis@confluent.io
>>>>>>>>>       <ma...@confluent.io>> wrote:
>>>>>>>>>       >>>>
>>>>>>>>>       >>>> I'm less enthusiastic about inlining the branch logic
>>>>>>>>> with its
>>>>>>>>>       downstream
>>>>>>>>>       >>>> functionality. Programs that have deep branch trees will
>>>>>>>>>       quickly become
>>>>>>>>>       >>>> harder to read as a single unit.
>>>>>>>>>       >>>>
>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>>       <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Also +1 on the issues/goals as Michael outlined them,
>>>>>>>>> I think
>>>>>>>>>       that sets a
>>>>>>>>>       >>>>> great framework for the discussion.
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Regarding the SortedMap solution, my understanding is
>>>>>>>>> that the
>>>>>>>>>       current
>>>>>>>>>       >>>>> proposal in the KIP is what is in my PR which
>>>>>>>>> (pending naming
>>>>>>>>>       decisions) is
>>>>>>>>>       >>>>> roughly this:
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> stream.split()
>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>       >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>>       >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Obviously some ordering is necessary, since branching
>>>>>>>>> as a
>>>>>>>>>       construct
>>>>>>>>>       >>>>> doesn't work without it, but this solution seems like it
>>>>>>>>>       provides as much
>>>>>>>>>       >>>>> associativity as the SortedMap solution, because each
>>>>>>>>> branch()
>>>>>>>>>       call
>>>>>>>>>       >>>>> directly associates the "conditional" with the "code
>>>>>>>>> block."
>>>>>>>>>       The value it
>>>>>>>>>       >>>>> provides over the KIP solution is the accessing of
>>>>>>>>> streams in
>>>>>>>>>       the same
>>>>>>>>>       >>>>> scope.
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> The KIP solution is less "dynamic" than the SortedMap
>>>>>>>>> solution
>>>>>>>>>       in the sense
>>>>>>>>>       >>>>> that it is slightly clumsier to add a dynamic number of
>>>>>>>>>       branches, but it is
>>>>>>>>>       >>>>> certainly possible.  It seems to me like the API
>>>>>>>>> should favor
>>>>>>>>>       the "static"
>>>>>>>>>       >>>>> case anyway, and should make it simple and readable to
>>>>>>>>>       fluently declare and
>>>>>>>>>       >>>>> access your branches in-line.  It also makes it
>>>>>>>>> impossible to
>>>>>>>>>       ignore a
>>>>>>>>>       >>>>> branch, and it is possible to build an (almost)
>>>>>>>>> identical
>>>>>>>>>       SortedMap
>>>>>>>>>       >>>>> solution on top of it.
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> I could also see a middle ground where instead of a raw
>>>>>>>>>       SortedMap being
>>>>>>>>>       >>>>> taken in, branch() takes a name and not a Consumer.
>>>>>>>>> Something
>>>>>>>>>       like this:
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>>       >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>>       >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>>       >>>>>    .defaultBranch("defaultBranch",
>>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Pros for that solution:
>>>>>>>>>       >>>>> - accessing branched KStreams in same scope
>>>>>>>>>       >>>>> - no double brace initialization, hopefully slightly
>>>>>>>>> more
>>>>>>>>>       readable than
>>>>>>>>>       >>>>> SortedMap
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Cons
>>>>>>>>>       >>>>> - downstream branch logic cannot be specified inline
>>>>>>>>> which
>>>>>>>>>       makes it harder
>>>>>>>>>       >>>>> to read top to bottom (like existing API and
>>>>>>>>> SortedMap, but
>>>>>>>>>       unlike the KIP)
>>>>>>>>>       >>>>> - you can forget to "handle" one of the branched
>>>>>>>>> streams (like
>>>>>>>>>       existing
>>>>>>>>>       >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> (KBranchedStreams could even work *both* ways but
>>>>>>>>> perhaps
>>>>>>>>>       that's overdoing
>>>>>>>>>       >>>>> it).
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Overall I'm curious how important it is to be able to
>>>>>>>>> easily
>>>>>>>>>       access the
>>>>>>>>>       >>>>> branched KStream in the same scope as the original.
>>>>>>>>> It's
>>>>>>>>>       possible that it
>>>>>>>>>       >>>>> doesn't need to be handled directly by the API, but
>>>>>>>>> instead
>>>>>>>>>       left up to the
>>>>>>>>>       >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> Paul
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>>>>       <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>>       >>>>> wrote:
>>>>>>>>>       >>>>>
>>>>>>>>>       >>>>>> I'd like to +1 what Michael said about the issues
>>>>>>>>> with the
>>>>>>>>>       existing
>>>>>>>>>       >>>>> branch
>>>>>>>>>       >>>>>> method, I agree with what he's outlined and I think
>>>>>>>>> we should
>>>>>>>>>       proceed by
>>>>>>>>>       >>>>>> trying to alleviate these problems. Specifically it
>>>>>>>>> seems
>>>>>>>>>       important to be
>>>>>>>>>       >>>>>> able to cleanly access the individual branches (eg
>>>>>>>>> by mapping
>>>>>>>>>       >>>>>> name->stream), which I thought was the original
>>>>>>>>> intention of
>>>>>>>>>       this KIP.
>>>>>>>>>       >>>>>>
>>>>>>>>>       >>>>>> That said, I don't think we should so easily give in
>>>>>>>>> to the
>>>>>>>>>       double brace
>>>>>>>>>       >>>>>> anti-pattern or force ours users into it if at all
>>>>>>>>> possible to
>>>>>>>>>       >>>>> avoid...just
>>>>>>>>>       >>>>>> my two cents.
>>>>>>>>>       >>>>>>
>>>>>>>>>       >>>>>> Cheers,
>>>>>>>>>       >>>>>> Sophie
>>>>>>>>>       >>>>>>
>>>>>>>>>       >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>>>>       >>>>>> michael.drogalis@confluent.io
>>>>>>>>>       <ma...@confluent.io>> wrote:
>>>>>>>>>       >>>>>>
>>>>>>>>>       >>>>>>> I’d like to propose a different way of thinking
>>>>>>>>> about this.
>>>>>>>>>       To me,
>>>>>>>>>       >>>>> there
>>>>>>>>>       >>>>>>> are three problems with the existing branch signature:
>>>>>>>>>       >>>>>>>
>>>>>>>>>       >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>>> raises unsafe
>>>>>>>> type
>>>>>>>>>       >>>>>> warnings.
>>>>>>>>>       >>>>>>> 2. The way in which you use the stream branches is
>>>>>>>>>       positionally coupled
>>>>>>>>>       >>>>>> to
>>>>>>>>>       >>>>>>> the ordering of the conditionals.
>>>>>>>>>       >>>>>>> 3. It is brittle to extend existing branch calls with
>>>>>>>>>       additional code
>>>>>>>>>       >>>>>>> paths.
>>>>>>>>>       >>>>>>>
>>>>>>>>>       >>>>>>> Using associative constructs instead of relying on
>>>>>>>>> ordered
>>>>>>>>>       constructs
>>>>>>>>>       >>>>>> would
>>>>>>>>>       >>>>>>> be a stronger approach. Consider a signature that
>>>>>>>>> instead
>>>>>>>>>       looks like
>>>>>>>>>       >>>>>> this:
>>>>>>>>>       >>>>>>> Map<String, KStream<K,V>>
>>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>>       Predicate<?
>>>>>>>>>       >>>>>>> super K,? super V>>);
>>>>>>>>>       >>>>>>>
>>>>>>>>>       >>>>>>> Branches are given names in a map, and as a result,
>>>>>>>>> the API
>>>>>>>>>       returns a
>>>>>>>>>       >>>>>>> mapping of names to streams. The ordering of the
>>>>>>>> conditionals is
>>>>>>>>>       >>>>>> maintained
>>>>>>>>>       >>>>>>> because it’s a sorted map. Insert order determines
>>>>>>>>> the order
>>>>>>>> of
>>>>>>>>>       >>>>>> evaluation.
>>>>>>>>>       >>>>>>> This solves problem 1 because there are no more
>>>>>>>>> varargs. It
>>>>>>>>>       solves
>>>>>>>>>       >>>>>> problem
>>>>>>>>>       >>>>>>> 2 because you no longer lean on ordering to access the
>>>>>>>>>       branch you’re
>>>>>>>>>       >>>>>>> interested in. It solves problem 3 because you can
>>>>>>>>> introduce
>>>>>>>>>       another
>>>>>>>>>       >>>>>>> conditional by simply attaching another name to the
>>>>>>>>>       structure, rather
>>>>>>>>>       >>>>>> than
>>>>>>>>>       >>>>>>> messing with the existing indices.
>>>>>>>>>       >>>>>>>
>>>>>>>>>       >>>>>>> One of the drawbacks is that creating the map
>>>>>>>>> inline is
>>>>>>>>>       historically
>>>>>>>>>       >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>>>>       voluminously, but
>>>>>>>>>       >>>>>>> double brace initialization would clean up the
>>>>>>>>> aesthetics.
>>>>>>>>>       >>>>>>>
>>>>>>>>>       >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>>       <john@confluent.io <ma...@confluent.io>>
>>>>>>>>>       >>>>> wrote:
>>>>>>>>>       >>>>>>>> Hi Ivan,
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>> Thanks for the update.
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>> FWIW, I agree with Matthias that the current "start
>>>>>>>> branching"
>>>>>>>>>       >>>>> operator
>>>>>>>>>       >>>>>>> is
>>>>>>>>>       >>>>>>>> confusing when named the same way as the actual
>>>>>>>>> branches.
>>>>>>>>>       "Split"
>>>>>>>>>       >>>>> seems
>>>>>>>>>       >>>>>>>> like a good name. Alternatively, we can do without
>>>>>>>>> a "start
>>>>>>>>>       >>>>> branching"
>>>>>>>>>       >>>>>>>> operator at all, and just do:
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>> stream
>>>>>>>>>       >>>>>>>>      .branch(Predicate)
>>>>>>>>>       >>>>>>>>      .branch(Predicate)
>>>>>>>>>       >>>>>>>>      .defaultBranch();
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>> Tentatively, I think that this branching operation
>>>>>>>>> should be
>>>>>>>>>       >>>>> terminal.
>>>>>>>>>       >>>>>>> That
>>>>>>>>>       >>>>>>>> way, we don't create ambiguity about how to use
>>>>>>>>> it. That
>>>>>>>>>       is, `branch`
>>>>>>>>>       >>>>>>>> should return `KBranchedStream`, while
>>>>>>>>> `defaultBranch` is
>>>>>>>>>       `void`, to
>>>>>>>>>       >>>>>>>> enforce that it comes last, and that there is only
>>>>>>>>> one
>>>>>>>>>       definition of
>>>>>>>>>       >>>>>> the
>>>>>>>>>       >>>>>>>> default branch. Potentially, we should log a
>>>>>>>>> warning if
>>>>>>>>>       there's no
>>>>>>>>>       >>>>>>> default,
>>>>>>>>>       >>>>>>>> and additionally log a warning (or throw an
>>>>>>>>> exception) if a
>>>>>>>>>       record
>>>>>>>>>       >>>>>> falls
>>>>>>>>>       >>>>>>>> though with no default.
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>> Thoughts?
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>> Thanks,
>>>>>>>>>       >>>>>>>> -John
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>>>>       >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>>>>       >>>>>>>> wrote:
>>>>>>>>>       >>>>>>>>
>>>>>>>>>       >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>> this is to make the name similar to String#split
>>>>>>>>>       >>>>>>>>>>> that also returns an array, right?
>>>>>>>>>       >>>>>>>>> The intend was to avoid name duplication. The
>>>>>>>>> return type
>>>>>>>>>       should
>>>>>>>>>       >>>>>> _not_
>>>>>>>>>       >>>>>>>>> be an array.
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> The current proposal is
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> stream.branch()
>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>       >>>>>>>>>      .defaultBranch();
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>>>>       `branch()` does
>>>>>>>>>       >>>>> not
>>>>>>>>>       >>>>>>>>> take any parameters and has different semantics
>>>>>>>>> than the
>>>>>>>> later
>>>>>>>>>       >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>>> snippet above,
>>>>>>>> it's
>>>>>>>>>       >>>>> hidden
>>>>>>>>>       >>>>>>>>> that the first call is `KStream#branch()` while
>>>>>>>>> the others
>>>>>>>> are
>>>>>>>>>       >>>>>>>>> `KBranchedStream#branch()` what makes reading the
>>>>>>>>> code
>>>>>>>> harder.
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> Because I suggested to rename `addBranch()` ->
>>>>>>>>> `branch()`,
>>>>>>>>>       I though
>>>>>>>>>       >>>>>> it
>>>>>>>>>       >>>>>>>>> might be better to also rename `KStream#branch()`
>>>>>>>>> to avoid
>>>>>>>> the
>>>>>>>>>       >>>>> naming
>>>>>>>>>       >>>>>>>>> overlap that seems to be confusing. The following
>>>>>>>>> reads
>>>>>>>> much
>>>>>>>>>       >>>>> cleaner
>>>>>>>>>       >>>>>> to
>>>>>>>>>       >>>>>>>> me:
>>>>>>>>>       >>>>>>>>> stream.split()
>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>       >>>>>>>>>      .branch(Predicate)
>>>>>>>>>       >>>>>>>>>      .defaultBranch();
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> Maybe there is a better alternative to `split()`
>>>>>>>>> though to
>>>>>>>>>       avoid
>>>>>>>>>       >>>>> the
>>>>>>>>>       >>>>>>>>> naming overlap.
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>>> unfortunately
>>>>>>>> we
>>>>>>>>>       >>>>> cannot
>>>>>>>>>       >>>>>>> have
>>>>>>>>>       >>>>>>>>> a method with such name :-)
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> Bummer. Didn't consider this. Maybe we can still
>>>>>>>>> come up
>>>>>>>>>       with a
>>>>>>>>>       >>>>> short
>>>>>>>>>       >>>>>>>> name?
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> Can you add the interface `KBranchedStream` to
>>>>>>>>> the KIP
>>>>>>>>>       with all
>>>>>>>>>       >>>>> it's
>>>>>>>>>       >>>>>>>>> methods? It will be part of public API and should be
>>>>>>>>>       contained in
>>>>>>>>>       >>>>> the
>>>>>>>>>       >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>>> return type of
>>>>>>>>>       >>>>>>>>> `defaultBranch()` is.
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>>       `KBranchedStream#get(int
>>>>>>>>>       >>>>>>> index)
>>>>>>>>>       >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>>       branched-KStreams. Would
>>>>>>>>>       >>>>>> be
>>>>>>>>>       >>>>>>>>> nice to get your feedback about it. It seems you
>>>>>>>>> suggest
>>>>>>>>>       that users
>>>>>>>>>       >>>>>>>>> would need to write custom utility code
>>>>>>>>> otherwise, to
>>>>>>>>>       access them.
>>>>>>>>>       >>>>> We
>>>>>>>>>       >>>>>>>>> should discuss the pros and cons of both
>>>>>>>>> approaches. It
>>>>>>>> feels
>>>>>>>>>       >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>>> built-in support
>>>>>>>>>       to get
>>>>>>>>>       >>>>> the
>>>>>>>>>       >>>>>>>>> branched-KStreams directly.
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>> -Matthias
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>       >>>>>>>>>> Hi all!
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> I have updated the KIP-418 according to the new
>>>>>>>>> vision.
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>       >>>>>>>>>> I can see your point: this is to make the name
>>>>>>>>> similar to
>>>>>>>>>       >>>>>>> String#split
>>>>>>>>>       >>>>>>>>>> that also returns an array, right? But is it
>>>>>>>>> worth the
>>>>>>>>>       loss of
>>>>>>>>>       >>>>>>>> backwards
>>>>>>>>>       >>>>>>>>>> compatibility? We can have overloaded branch()
>>>>>>>>> as well
>>>>>>>>>       without
>>>>>>>>>       >>>>>>>> affecting
>>>>>>>>>       >>>>>>>>>> the existing code. Maybe the old array-based
>>>>>>>>> `branch`
>>>>>>>> method
>>>>>>>>>       >>>>> should
>>>>>>>>>       >>>>>>> be
>>>>>>>>>       >>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>>       >>>>> BranchingKStream#branch(),
>>>>>>>>>       >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>> BranchingKStream#default()
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> Totally agree with 'addBranch->branch' rename.
>>>>>>>>> 'default'
>>>>>>>> is,
>>>>>>>>>       >>>>>>> however, a
>>>>>>>>>       >>>>>>>>>> reserved word, so unfortunately we cannot have a
>>>>>>>>> method
>>>>>>>>>       with such
>>>>>>>>>       >>>>>>> name
>>>>>>>>>       >>>>>>>>> :-)
>>>>>>>>>       >>>>>>>>>>> defaultBranch() does take an `Predicate` as
>>>>>>>>> argument,
>>>>>>>> but I
>>>>>>>>>       >>>>> think
>>>>>>>>>       >>>>>>> that
>>>>>>>>>       >>>>>>>>>> is not required?
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> Absolutely! I think that was just copy-paste
>>>>>>>>> error or
>>>>>>>>>       something.
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> Dear colleagues,
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> please revise the new version of the KIP and
>>>>>>>>> Paul's PR
>>>>>>>>>       >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> Ivan
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>
>>>>>>>>>       >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>       >>>>>>>>>>> Thanks for driving the discussion of this KIP.
>>>>>>>>> It seems
>>>>>>>> that
>>>>>>>>>       >>>>>>> everybody
>>>>>>>>>       >>>>>>>>>>> agrees that the current branch() method using
>>>>>>>>> arrays is
>>>>>>>> not
>>>>>>>>>       >>>>>> optimal.
>>>>>>>>>       >>>>>>>>>>> I had a quick look into the PR and I like the
>>>>>>>>> overall
>>>>>>>>>       proposal.
>>>>>>>>>       >>>>>>> There
>>>>>>>>>       >>>>>>>>>>> are some minor things we need to consider. I would
>>>>>>>>>       recommend the
>>>>>>>>>       >>>>>>>>>>> following renaming:
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>       >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>>> BranchingKStream#branch()
>>>>>>>>>       >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>>       BranchingKStream#default()
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> It's just a suggestion to get slightly shorter
>>>>>>>>> method
>>>>>>>> names.
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>>>>       `Predicate` as
>>>>>>>>>       >>>>>>>> argument,
>>>>>>>>>       >>>>>>>>>>> but I think that is not required?
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>>> recently
>>>>>>>>>       accepted and
>>>>>>>>>       >>>>>> is
>>>>>>>>>       >>>>>>>>>>> currently implemented:
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>
>>>>>>>>>
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> Ie, we should add overloads that accepted a
>>>>>>>>> `Named`
>>>>>>>>>       parameter.
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> For the issue that the created `KStream` object
>>>>>>>>> are in
>>>>>>>>>       different
>>>>>>>>>       >>>>>>>> scopes:
>>>>>>>>>       >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>>>>>>>>>       index)` method
>>>>>>>>>       >>>>>>> that
>>>>>>>>>       >>>>>>>>>>> returns the corresponding "branched" result
>>>>>>>>> `KStream`
>>>>>>>>>       object?
>>>>>>>>>       >>>>>> Maybe,
>>>>>>>>>       >>>>>>>> the
>>>>>>>>>       >>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>>>>>>       >>>>>> `Consumer<KStream>`
>>>>>>>>>       >>>>>>>> but
>>>>>>>>>       >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could
>>>>>>>>> return
>>>>>>>>>       whatever
>>>>>>>>>       >>>>>> the
>>>>>>>>>       >>>>>>>>>>> `Function` returns?
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> Finally, I would also suggest to update the KIP
>>>>>>>>> with the
>>>>>>>>>       current
>>>>>>>>>       >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>> -Matthias
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>       >>>>>>>>>>>> Ivan,
>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>>> think it
>>>>>>>>>       makes sense
>>>>>>>>>       >>>>>> for
>>>>>>>>>       >>>>>>>> you
>>>>>>>>>       >>>>>>>>> to
>>>>>>>>>       >>>>>>>>>>>> revise the KIP and continue the discussion.
>>>>>>>>> Obviously
>>>>>>>>>       we'll
>>>>>>>>>       >>>>> need
>>>>>>>>>       >>>>>>>> some
>>>>>>>>>       >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>>> binding votes on
>>>>>>>>>       >>>>> whether
>>>>>>>>>       >>>>>>> the
>>>>>>>>>       >>>>>>>>> KIP
>>>>>>>>>       >>>>>>>>>>>> could be adopted.  It would be great to hear
>>>>>>>>> if they
>>>>>>>>>       think this
>>>>>>>>>       >>>>>> is
>>>>>>>>>       >>>>>>> a
>>>>>>>>>       >>>>>>>>> good
>>>>>>>>>       >>>>>>>>>>>> idea overall.  I'm not sure if that happens
>>>>>>>>> just by
>>>>>>>>>       starting a
>>>>>>>>>       >>>>>>> vote,
>>>>>>>>>       >>>>>>>>> or if
>>>>>>>>>       >>>>>>>>>>>> there is generally some indication of interest
>>>>>>>> beforehand.
>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>> That being said, I'll continue the discussion
>>>>>>>>> a bit:
>>>>>>>>>       assuming
>>>>>>>>>       >>>>> we
>>>>>>>>>       >>>>>> do
>>>>>>>>>       >>>>>>>>> move
>>>>>>>>>       >>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>>>>>>       >>>>>> KBranchedStream",
>>>>>>>>>       >>>>>>> do
>>>>>>>>>       >>>>>>>>> we
>>>>>>>>>       >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>>> KStream[]"?  I
>>>>>>>> would
>>>>>>>>>       >>>>> favor
>>>>>>>>>       >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>>> exclusive APIs
>>>>>>>> that
>>>>>>>>>       >>>>>>> accomplish
>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>>> they're fairly
>>>>>>>>>       similar
>>>>>>>>>       >>>>>>>>> anyway.  We
>>>>>>>>>       >>>>>>>>>>>> just need to be sure we're not making something
>>>>>>>>>       >>>>>>> impossible/difficult
>>>>>>>>>       >>>>>>>>> that
>>>>>>>>>       >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>>> structure would
>>>>>>>> work,
>>>>>>>>>       >>>>> it's
>>>>>>>>>       >>>>>>>> just a
>>>>>>>>>       >>>>>>>>>>>> little sloppy overall in terms of naming and
>>>>>>>>> clarity. In
>>>>>>>>>       >>>>>>> particular,
>>>>>>>>>       >>>>>>>>>>>> passing in the "predicates" and "children"
>>>>>>>>> lists which
>>>>>>>> get
>>>>>>>>>       >>>>>> modified
>>>>>>>>>       >>>>>>>> in
>>>>>>>>>       >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>>       KStreamLazyBranch is
>>>>>>>>>       >>>>> a
>>>>>>>>>       >>>>>>> bit
>>>>>>>>>       >>>>>>>>>>>> complicated to follow.
>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>> Thanks,
>>>>>>>>>       >>>>>>>>>>>> Paul
>>>>>>>>>       >>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>       >>>>>>>>> wrote:
>>>>>>>>>       >>>>>>>>>>>>> Hi Paul!
>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>>>>       convinced: your
>>>>>>>>>       >>>>>>>> proposal
>>>>>>>>>       >>>>>>>>>>>>> looks better and should work. We just have to
>>>>>>>>> document
>>>>>>>> the
>>>>>>>>>       >>>>>> crucial
>>>>>>>>>       >>>>>>>>> fact
>>>>>>>>>       >>>>>>>>>>>>> that KStream consumers are invoked as they're
>>>>>>>>> added.
>>>>>>>>>       And then
>>>>>>>>>       >>>>>> it's
>>>>>>>>>       >>>>>>>> all
>>>>>>>>>       >>>>>>>>>>>>> going to be very nice.
>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>> What shall we do now? I should re-write the
>>>>>>>>> KIP and
>>>>>>>>>       resume the
>>>>>>>>>       >>>>>>>>>>>>> discussion here, right?
>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>> Why are you telling that your PR 'should not
>>>>>>>>> be even a
>>>>>>>>>       >>>>> starting
>>>>>>>>>       >>>>>>>> point
>>>>>>>>>       >>>>>>>>> if
>>>>>>>>>       >>>>>>>>>>>>> we go in this direction'? To me it looks like
>>>>>>>>> a good
>>>>>>>>>       starting
>>>>>>>>>       >>>>>>> point.
>>>>>>>>>       >>>>>>>>> But
>>>>>>>>>       >>>>>>>>>>>>> as a novice in this project I might miss some
>>>>>>>>> important
>>>>>>>>>       >>>>> details.
>>>>>>>>>       >>>>>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>> Ivan
>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>       >>>>>>>>>>>>>> Ivan,
>>>>>>>>>       >>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>>>>>>       >>>>> stream.branch()
>>>>>>>>>       >>>>>>>>> solution
>>>>>>>>>       >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>>> consumers will be
>>>>>>>>>       >>>>> invoked
>>>>>>>>>       >>>>>> as
>>>>>>>>>       >>>>>>>>> they’re
>>>>>>>>>       >>>>>>>>>>>>> added, not during streamsBuilder.build(). So
>>>>>>>>> the user
>>>>>>>>>       still
>>>>>>>>>       >>>>>> ought
>>>>>>>>>       >>>>>>> to
>>>>>>>>>       >>>>>>>>> be
>>>>>>>>>       >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward
>>>>>>>>> and
>>>>>>>>>       depend on
>>>>>>>>>       >>>>> the
>>>>>>>>>       >>>>>>>>> branched
>>>>>>>>>       >>>>>>>>>>>>> streams having been set.
>>>>>>>>>       >>>>>>>>>>>>>> The issue I mean to point out is that it is
>>>>>>>>> hard to
>>>>>>>>>       access
>>>>>>>>>       >>>>> the
>>>>>>>>>       >>>>>>>>> branched
>>>>>>>>>       >>>>>>>>>>>>> streams in the same scope as the original
>>>>>>>>> stream (that
>>>>>>>>>       is, not
>>>>>>>>>       >>>>>>>> inside
>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>> couponIssuer), which is a problem with both
>>>>>>>>> proposed
>>>>>>>>>       >>>>> solutions.
>>>>>>>>>       >>>>>> It
>>>>>>>>>       >>>>>>>>> can be
>>>>>>>>>       >>>>>>>>>>>>> worked around though.
>>>>>>>>>       >>>>>>>>>>>>>> [Also, great to hear additional interest in
>>>>>>>>> 401, I’m
>>>>>>>>>       excited
>>>>>>>>>       >>>>> to
>>>>>>>>>       >>>>>>>> hear
>>>>>>>>>       >>>>>>>>>>>>> your thoughts!]
>>>>>>>>>       >>>>>>>>>>>>>> Paul
>>>>>>>>>       >>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>>>>>>       >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>       >>>>>>>>> wrote:
>>>>>>>>>       >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> The idea to postpone the wiring of branches
>>>>>>>>> to the
>>>>>>>>>       >>>>>>>>>>>>> streamsBuilder.build() also looked great for
>>>>>>>>> me at
>>>>>>>> first
>>>>>>>>>       >>>>> glance,
>>>>>>>>>       >>>>>>> but
>>>>>>>>>       >>>>>>>>> ---
>>>>>>>>>       >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>>> available in the
>>>>>>>>>       same
>>>>>>>>>       >>>>>> scope
>>>>>>>>>       >>>>>>> as
>>>>>>>>>       >>>>>>>>> each
>>>>>>>>>       >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
>>>>>>>> together
>>>>>>>>>       >>>>> again
>>>>>>>>>       >>>>>> I
>>>>>>>>>       >>>>>>>>> don't see
>>>>>>>>>       >>>>>>>>>>>>> a way to do that.
>>>>>>>>>       >>>>>>>>>>>>>>> You just took the words right out of my
>>>>>>>>> mouth, I was
>>>>>>>>>       just
>>>>>>>>>       >>>>>> going
>>>>>>>>>       >>>>>>> to
>>>>>>>>>       >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>       >>>>>>>>>>>>>>> Consider the example from Bill's book, p.
>>>>>>>>> 101: say
>>>>>>>>>       we need
>>>>>>>>>       >>>>> to
>>>>>>>>>       >>>>>>>>> identify
>>>>>>>>>       >>>>>>>>>>>>> customers who have bought coffee and made a
>>>>>>>>> purchase
>>>>>>>>>       in the
>>>>>>>>>       >>>>>>>>> electronics
>>>>>>>>>       >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>       >>>>>>>>>>>>>>> This is the code I usually write under these
>>>>>>>>>       circumstances
>>>>>>>>>       >>>>>> using
>>>>>>>>>       >>>>>>>> my
>>>>>>>>>       >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>       >>>>>>>>>>>>>>> @Setter
>>>>>>>>>       >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>>       >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>       >>>>>>>>>>>>>>>       return
>>>>>>>>>       >>>>>>>>>
>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>       >>>>>>>>>>>>>>>       /*In the real world the code here can be
>>>>>>>>>       complex, so
>>>>>>>>>       >>>>>>>>> creation of
>>>>>>>>>       >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>>> justified, in
>>>>>>>>>       order to
>>>>>>>>>       >>>>>>>> separate
>>>>>>>>>       >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>       >>>>>>>>>>>>>>>  }
>>>>>>>>>       >>>>>>>>>>>>>>> }
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>>> CouponIssuer();
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>>       >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>>       >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>       >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to
>>>>>>>>> wire up
>>>>>>>>>       everything
>>>>>>>>>       >>>>>>>> later,
>>>>>>>>>       >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>       >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>>>>       initialize the
>>>>>>>>>       >>>>>>>>> CouponIssuer
>>>>>>>>>       >>>>>>>>>>>>> we need the terminal operation to be called
>>>>>>>>> before
>>>>>>>>>       >>>>>>>>> streamsBuilder.build()
>>>>>>>>>       >>>>>>>>>>>>> is called.
>>>>>>>>>       >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>>> KIP-401 is
>>>>>>>>>       essentially
>>>>>>>>>       >>>>>> the
>>>>>>>>>       >>>>>>>>> next
>>>>>>>>>       >>>>>>>>>>>>> KIP I was going to write here. I have some
>>>>>>>>> thoughts
>>>>>>>>>       based on
>>>>>>>>>       >>>>> my
>>>>>>>>>       >>>>>>>>> experience,
>>>>>>>>>       >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>>>>       >>>>>>>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> Ivan
>>>>>>>>>       >>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>       >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>       >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>>> concept of a
>>>>>>>>>       fluent
>>>>>>>>>       >>>>> API
>>>>>>>>>       >>>>>>>> based
>>>>>>>>>       >>>>>>>>>>>>> off of
>>>>>>>>>       >>>>>>>>>>>>>>>> KStream here
>>>>>>>>>       (https://github.com/apache/kafka/pull/6512),
>>>>>>>>>       >>>>>> and
>>>>>>>>>       >>>>>>> I
>>>>>>>>>       >>>>>>>>> think
>>>>>>>>>       >>>>>>>>>>>>> I
>>>>>>>>>       >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>       >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>>> earlier about
>>>>>>>>>       >>>>>>> compatibility
>>>>>>>>>       >>>>>>>>>>>>> issues,
>>>>>>>>>       >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was
>>>>>>>>> unaware
>>>>>>>>>       that Java
>>>>>>>>>       >>>>> is
>>>>>>>>>       >>>>>>>> smart
>>>>>>>>>       >>>>>>>>>>>>> enough to
>>>>>>>>>       >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>>>>>>>>>       returning one
>>>>>>>>>       >>>>>>> thing
>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>       >>>>>>>>>>>>> branch()
>>>>>>>>>       >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>>>>       >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't
>>>>>>>>> actually
>>>>>>>>>       need
>>>>>>>>>       >>>>> it.
>>>>>>>>>       >>>>>>> We
>>>>>>>>>       >>>>>>>>> can
>>>>>>>>>       >>>>>>>>>>>>> just
>>>>>>>>>       >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>>> KBranchedStream who
>>>>>>>>>       shares
>>>>>>>>>       >>>>>> its
>>>>>>>>>       >>>>>>>>> state
>>>>>>>>>       >>>>>>>>>>>>> with the
>>>>>>>>>       >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do
>>>>>>>>> the
>>>>>>>>>       branching.
>>>>>>>>>       >>>>>>> It's
>>>>>>>>>       >>>>>>>>> not
>>>>>>>>>       >>>>>>>>>>>>> terribly
>>>>>>>>>       >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>>>>>>>>>       demonstrates
>>>>>>>>>       >>>>>> its
>>>>>>>>>       >>>>>>>>>>>>> feasibility.
>>>>>>>>>       >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>>> request should
>>>>>>>> be
>>>>>>>>>       >>>>> final
>>>>>>>>>       >>>>>> or
>>>>>>>>>       >>>>>>>>> even a
>>>>>>>>>       >>>>>>>>>>>>>>>> starting point if we go in this direction,
>>>>>>>>> I just
>>>>>>>>>       wanted to
>>>>>>>>>       >>>>>> see
>>>>>>>>>       >>>>>>>> how
>>>>>>>>>       >>>>>>>>>>>>>>>> challenging it would be to get the API
>>>>>>>>> working.
>>>>>>>>>       >>>>>>>>>>>>>>>> I will say though, that I'm not sure the
>>>>>>>>> existing
>>>>>>>>>       solution
>>>>>>>>>       >>>>>>> could
>>>>>>>>>       >>>>>>>> be
>>>>>>>>>       >>>>>>>>>>>>>>>> deprecated in favor of this, which I had
>>>>>>>>> originally
>>>>>>>>>       >>>>> suggested
>>>>>>>>>       >>>>>>>> was a
>>>>>>>>>       >>>>>>>>>>>>>>>> possibility.  The reason is that the newly
>>>>>>>>> branched
>>>>>>>>>       streams
>>>>>>>>>       >>>>>> are
>>>>>>>>>       >>>>>>>> not
>>>>>>>>>       >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>>> other.  That
>>>>>>>>>       is, if we
>>>>>>>>>       >>>>>>> wanted
>>>>>>>>>       >>>>>>>>> to
>>>>>>>>>       >>>>>>>>>>>>> merge
>>>>>>>>>       >>>>>>>>>>>>>>>> them back together again I don't see a way
>>>>>>>>> to do
>>>>>>>>>       that.  The
>>>>>>>>>       >>>>>> KIP
>>>>>>>>>       >>>>>>>>>>>>> proposal
>>>>>>>>>       >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>>> means is that
>>>>>>>> for
>>>>>>>>>       >>>>>> either
>>>>>>>>>       >>>>>>>>>>>>> solution,
>>>>>>>>>       >>>>>>>>>>>>>>>> deprecating the existing branch(...) is
>>>>>>>>> not on the
>>>>>>>>>       table.
>>>>>>>>>       >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>       >>>>>>>>>>>>>>>> Paul
>>>>>>>>>       >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan
>>>>>>>>> Ponomarev <
>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>>>>       >>>>>>>>>>>>> wrote:
>>>>>>>>>       >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>>> discussed up to
>>>>>>>> this
>>>>>>>>>       >>>>>> point.
>>>>>>>>>       >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed
>>>>>>>>> that
>>>>>>>>>       branch API
>>>>>>>>>       >>>>>>> needs
>>>>>>>>>       >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>       >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...)....
>>>>>>>>> //onTopOf
>>>>>>>>>       returns
>>>>>>>>>       >>>>>> its
>>>>>>>>>       >>>>>>>>> argument
>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2)
>>>>>>>>> The code
>>>>>>>> won't
>>>>>>>>>       >>>>> make
>>>>>>>>>       >>>>>>>> sense
>>>>>>>>>       >>>>>>>>>>>>> until
>>>>>>>>>       >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>       instance
>>>>>>>>>       >>>>>>>>> contrasts the
>>>>>>>>>       >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> stream
>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>       >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>       >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>>> noDefault(). Both
>>>>>>>>>       >>>>>>>>> defaultBranch(..)
>>>>>>>>>       >>>>>>>>>>>>> and
>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams
>>>>>>>>> interface
>>>>>>>> is
>>>>>>>>>       >>>>>> defined.
>>>>>>>>>       >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>>>>>       >>>>>>>> (defaultBranch(ks->)
>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>       >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very
>>>>>>>>> easy to
>>>>>>>>>       miss the
>>>>>>>>>       >>>>>> fact
>>>>>>>>>       >>>>>>>>> that one
>>>>>>>>>       >>>>>>>>>>>>>>>>> of the terminal methods should be called.
>>>>>>>>> If these
>>>>>>>>>       methods
>>>>>>>>>       >>>>>> are
>>>>>>>>>       >>>>>>>> not
>>>>>>>>>       >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>>> runtime.
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can
>>>>>>>>> we do
>>>>>>>> better?
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>       >>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>       >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> I see your point when you are talking
>>>>>>>>> about
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>>>>>>>>>       implemented the
>>>>>>>>>       >>>>>>> easy
>>>>>>>>>       >>>>>>>>> way.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
>>>>>>>> assumes
>>>>>>>>>       >>>>> nothing
>>>>>>>>>       >>>>>>>> will
>>>>>>>>>       >>>>>>>>>>>>> reach
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>> occurs.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be
>>>>>>>>> the only
>>>>>>>> option
>>>>>>>>>       >>>>>> besides
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios
>>>>>>>>> when we
>>>>>>>>>       want to
>>>>>>>>>       >>>>>> just
>>>>>>>>>       >>>>>>>>> silently
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>>>>       predicate. 2)
>>>>>>>>>       >>>>>>> Throwing
>>>>>>>>>       >>>>>>>>> an
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> exception in the middle of data flow
>>>>>>>>> processing
>>>>>>>>>       looks
>>>>>>>>>       >>>>>> like a
>>>>>>>>>       >>>>>>>> bad
>>>>>>>>>       >>>>>>>>>>>>> idea.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would
>>>>>>>>> prefer to
>>>>>>>>>       emit a
>>>>>>>>>       >>>>>>>> special
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is
>>>>>>>>> exactly
>>>>>>>> where
>>>>>>>>>       >>>>>>> `default`
>>>>>>>>>       >>>>>>>>> can
>>>>>>>>>       >>>>>>>>>>>>> be
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>       >>>>> InternalTopologyBuilder
>>>>>>>>>       >>>>>>> to
>>>>>>>>>       >>>>>>>>> track
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>> and raise
>>>>>>>>>       a clear
>>>>>>>>>       >>>>>>> error
>>>>>>>>>       >>>>>>>>>>>>> before it
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the
>>>>>>>>> program is
>>>>>>>>>       >>>>> compiled
>>>>>>>>>       >>>>>>> and
>>>>>>>>>       >>>>>>>>> run?
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>>>>>>>>>       compile if
>>>>>>>>>       >>>>> used
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>>>>>>>>>       method chain
>>>>>>>>>       >>>>>>>> starting
>>>>>>>>>       >>>>>>>>>>>>> from
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost
>>>>>>>>> difference
>>>>>>>>>       between
>>>>>>>>>       >>>>>>>> runtime
>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure
>>>>>>>>> uncovers
>>>>>>>>>       >>>>> instantly
>>>>>>>>>       >>>>>> on
>>>>>>>>>       >>>>>>>>> unit
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> tests, it costs more for the project
>>>>>>>>> than a
>>>>>>>>>       compilation
>>>>>>>>>       >>>>>>>> failure.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>>> operation being
>>>>>>>>>       required.
>>>>>>>>>       >>>>>>> But
>>>>>>>>>       >>>>>>>> is
>>>>>>>>>       >>>>>>>>>>>>> that
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't
>>>>>>>>> want a
>>>>>>>>>       >>>>>> defaultBranch
>>>>>>>>>       >>>>>>>>> they
>>>>>>>>>       >>>>>>>>>>>>> can
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>>> (noDefaultBranch()?)
>>>>>>>>>       just as
>>>>>>>>>       >>>>>>>>> easily.  In
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a
>>>>>>>>> nicer API
>>>>>>>> - a
>>>>>>>>>       >>>>> user
>>>>>>>>>       >>>>>>>> could
>>>>>>>>>       >>>>>>>>>>>>> specify
>>>>>>>>>       >>>>>>>>>>>>>>>>> a
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing
>>>>>>>>> will reach
>>>>>>>> the
>>>>>>>>>       >>>>>> default
>>>>>>>>>       >>>>>>>>> branch,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>>> occurs.
>>>>>>>> That
>>>>>>>>>       >>>>> seems
>>>>>>>>>       >>>>>>> like
>>>>>>>>>       >>>>>>>>> an
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> improvement over the current branch()
>>>>>>>>> API,
>>>>>>>>>       which allows
>>>>>>>>>       >>>>>> for
>>>>>>>>>       >>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>> more
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
>>>>>>>> dropped.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>>> certainly has
>>>>>>>>>       to be
>>>>>>>>>       >>>>>> well
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>>       >>>>> InternalTopologyBuilder
>>>>>>>>>       >>>>>>> to
>>>>>>>>>       >>>>>>>>> track
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>>> and raise
>>>>>>>>>       a clear
>>>>>>>>>       >>>>>>> error
>>>>>>>>>       >>>>>>>>>>>>> before it
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that
>>>>>>>>> there is
>>>>>>>> a
>>>>>>>>>       >>>>> "build
>>>>>>>>>       >>>>>>>> step"
>>>>>>>>>       >>>>>>>>>>>>> where
>>>>>>>>>       >>>>>>>>>>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>>>>       >>>>>> StreamsBuilder.build()
>>>>>>>>>       >>>>>>> is
>>>>>>>>>       >>>>>>>>>>>>> called.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>>> argument, I
>>>>>>>> agree
>>>>>>>>>       >>>>> that
>>>>>>>>>       >>>>>>> it's
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> allow users to do other operations on
>>>>>>>>> the input
>>>>>>>>>       stream.
>>>>>>>>>       >>>>>>> With
>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>>>>>> fluent
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same
>>>>>>>>> way all
>>>>>>>> other
>>>>>>>>>       >>>>>>> operations
>>>>>>>>>       >>>>>>>>> do -
>>>>>>>>>       >>>>>>>>>>>>> if
>>>>>>>>>       >>>>>>>>>>>>>>>>> you
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
>>>>>>>> multiple
>>>>>>>>>       >>>>> times,
>>>>>>>>>       >>>>>>> you
>>>>>>>>>       >>>>>>>>> just
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call
>>>>>>>>> as many
>>>>>>>>>       operations
>>>>>>>>>       >>>>>> on
>>>>>>>>>       >>>>>>> it
>>>>>>>>>       >>>>>>>>> as
>>>>>>>>>       >>>>>>>>>>>>> you
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan
>>>>>>>>> Ponomarev <
>>>>>>>>>       >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we
>>>>>>>>> do not
>>>>>>>>>       always need
>>>>>>>>>       >>>>>> the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>>>>> operation we
>>>>>>>>>       >>>>> don't
>>>>>>>>>       >>>>>>>> know
>>>>>>>>>       >>>>>>>>>>>>> when to
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its
>>>>>>>>> argument,
>>>>>>>>>       so we
>>>>>>>>>       >>>>> can
>>>>>>>>>       >>>>>> do
>>>>>>>>>       >>>>>>>>>>>>> something
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> more with the original branch after
>>>>>>>>> branching.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
>>>>>>>> special
>>>>>>>>>       >>>>> object
>>>>>>>>>       >>>>>>>>>>>>> construction
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream
>>>>>>>>> methods.
>>>>>>>> But
>>>>>>>>>       >>>>> here
>>>>>>>>>       >>>>>> we
>>>>>>>>>       >>>>>>>>> have a
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to
>>>>>>>>> split the
>>>>>>>>>       flow,
>>>>>>>>>       >>>>> so
>>>>>>>>>       >>>>>> I
>>>>>>>>>       >>>>>>>>> think
>>>>>>>>>       >>>>>>>>>>>>> this
>>>>>>>>>       >>>>>>>>>>>>>>>>> is
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve
>>>>>>>>> this
>>>>>>>>>       API, but I
>>>>>>>>>       >>>>>> find
>>>>>>>>>       >>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>>>>>>>>>       contrasts the
>>>>>>>>>       >>>>>>> fluency
>>>>>>>>>       >>>>>>>>> of
>>>>>>>>>       >>>>>>>>>>>>> other
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd
>>>>>>>>> like to
>>>>>>>>>       just call
>>>>>>>>>       >>>>> a
>>>>>>>>>       >>>>>>>>> method on
>>>>>>>>>       >>>>>>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if
>>>>>>>>> the branch
>>>>>>>>>       cases
>>>>>>>>>       >>>>> are
>>>>>>>>>       >>>>>>>>> defined
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate,
>>>>>>>>> handleCase)
>>>>>>>>>       is very
>>>>>>>>>       >>>>>> nice
>>>>>>>>>       >>>>>>>>> and the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped
>>>>>>>>> around
>>>>>>>>>       how we
>>>>>>>>>       >>>>>>> specify
>>>>>>>>>       >>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>> source
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>>>>> this::handle1)
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>>>>> this::handle2)
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>>> KBranchedStreams or
>>>>>>>>>       >>>>>>>> KStreamBrancher
>>>>>>>>>       >>>>>>>>> or
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>>>>       terminated by
>>>>>>>>>       >>>>>>>>>>>>> defaultBranch()
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>>>>       incompatible with
>>>>>>>>>       >>>>> the
>>>>>>>>>       >>>>>>>>> current
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to
>>>>>>>>> have a
>>>>>>>>>       different
>>>>>>>>>       >>>>>> name,
>>>>>>>>>       >>>>>>>> but
>>>>>>>>>       >>>>>>>>> that
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we
>>>>>>>>> could call it
>>>>>>>>>       >>>>>> something
>>>>>>>>>       >>>>>>>> like
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the
>>>>>>>>> old API.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of
>>>>>>>>> your
>>>>>>>>>       KIP?  It
>>>>>>>>>       >>>>>> seems
>>>>>>>>>       >>>>>>>>> like it
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>>> branching
>>>>>>>>>       while also
>>>>>>>>>       >>>>>>>> allowing
>>>>>>>>>       >>>>>>>>> you
>>>>>>>>>       >>>>>>>>>>>>> to
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>>>>       KBranchedStreams
>>>>>>>>>       >>>>>> if
>>>>>>>>>       >>>>>>>>> desired.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan
>>>>>>>>> Ponomarev
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>>       ks){
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>>>>>>>>>       String> ks){
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String,
>>>>>>>>> String>()
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>>       this::handleFirstCase)
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>>       this::handleSecondCase)
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>>> KafkaStreamsBrancher
>>>>>>>>>       >>>>> takes a
>>>>>>>>>       >>>>>>>>> Consumer
>>>>>>>>>       >>>>>>>>>>>>> as a
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing,
>>>>>>>>> and the
>>>>>>>>>       example in
>>>>>>>>>       >>>>>> the
>>>>>>>>>       >>>>>>>> KIP
>>>>>>>>>       >>>>>>>>>>>>> shows
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a
>>>>>>>>> terminal node
>>>>>>>>>       >>>>>>>>> (KafkaStreams#to()
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but
>>>>>>>>> how would
>>>>>>>> we
>>>>>>>>>       >>>>> handle
>>>>>>>>>       >>>>>>> the
>>>>>>>>>       >>>>>>>>> case
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>>> wants to
>>>>>>>> continue
>>>>>>>>>       >>>>>>>> processing
>>>>>>>>>       >>>>>>>>> and
>>>>>>>>>       >>>>>>>>>>>>> not
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on
>>>>>>>>> the
>>>>>>>> branched
>>>>>>>>>       >>>>>> stream
>>>>>>>>>       >>>>>>>>>>>>> immediately?
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic
>>>>>>>>> as is if
>>>>>>>>>       we had
>>>>>>>>>       >>>>>>>> something
>>>>>>>>>       >>>>>>>>> like
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM
>>>>>>>>> Bill Bejeck
>>>>>>>> <
>>>>>>>>>       >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>>> discussion for
>>>>>>>> KIP-
>>>>>>>>>       >>>>> 418.
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
>>>>>>>> KIP-418.
>>>>>>>>>       >>>>> Please
>>>>>>>>>       >>>>>>>> take
>>>>>>>>>       >>>>>>>>> a
>>>>>>>>>       >>>>>>>>>>>>> look
>>>>>>>>>       >>>>>>>>>>>>>>>>> at
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>>> appreciate any
>>>>>>>>>       feedback :)
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>       >>>>>
>>>>>>>>>
>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>       >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>>       >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>       >>>>>>>>>
>>>>>>>>>       >
>>>>>>>>>
>>>>>>>>
>>>>>
>>>>
>>>
>>
> 



Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <mj...@apache.org>.
Ivan,

no worries about getting side tracked. Glad to have you back!

The DSL improved further in the meantime and we already have a `Named`
config object to name operators. It seems reasonable to me to build on this.

Furthermore, John did a writeup about "DSL design principles" that we
want to follow:
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+DSL+Grammar
-- might be worth to checkout.


-Matthias


On 4/17/20 4:30 PM, Ivan Ponomarev wrote:
> Hi everyone!
> 
> Let me revive the discussion of this KIP.
> 
> I'm very sorry for stopping my participation in the discussion in June
> 2019. My project work was very intensive then and it didn't leave me
> spare time. But I think I must finish this, because we invested
> substantial effort into this discussion and I'm not feel entitled to
> propose other things before this one is finalized.
> 
> During these months I proceeded with writing and reviewing Kafka
> Streams-related code. Every time I needed branching, Spring-Kafka's
> KafkaStreamBrancher class of my invention (the original idea for this
> KIP) worked for me -- that's another reason why I gave up pushing the
> KIP forward. When I was coming across the problem with the scope of
> branches, I worked around it this way:
> 
> AtomicReference<KStream<...>> result = new AtomicReference<>();
> new KafkaStreamBrancher<....>()
>     .branch(....)
>     .defaultBranch(result::set)
>     .onTopOf(someStream);
> result.get()...
> 
> 
> And yes, of course I don't feel very happy with this approach.
> 
> I think that Matthias came up with a bright solution in his post from
> May, 24th 2019. Let me quote it:
> 
> KStream#split() -> KBranchedStream
> // branch is not easily accessible in current scope
> KBranchedStream#branch(Predicate, Consumer<KStream>)
>   -> KBranchedStream
> // assign a name to the branch and
> // return the sub-stream to the current scope later
> //
> // can be simple as `#branch(p, s->s, "name")`
> // or also complex as `#branch(p, s->s.filter(...), "name")`
> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>   -> KBranchedStream
> // default branch is not easily accessible
> // return map of all named sub-stream into current scope
> KBranchedStream#default(Cosumer<KStream>)
>   -> Map<String,KStream>
> // assign custom name to default-branch
> // return map of all named sub-stream into current scope
> KBranchedStream#default(Function<KStream,KStream>, String)
>   -> Map<String,KStream>
> // assign a default name for default
> // return map of all named sub-stream into current scope
> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>   -> Map<String,KStream>
> // return map of all names sub-stream into current scope
> KBranchedStream#noDefaultBranch()
>   -> Map<String,KStream>
> 
> I believe this would satisfy everyone. Optional names seems to be a good
> idea: when you don't need to have the branches in the same scope, you
> just don't use names and you don't risk making your code brittle. Or,
> you might want to add names just for debugging purposes. Or, finally,
> you might use the returned Map to have the named branches in the
> original scope.
> 
> There also was an input from John Roesler on June 4th, 2019, who
> suggested using Named class. I can't comment on this. The idea seems
> reasonable, but in this matter I'd rather trust people who are more
> familiar with Streams API design principles than me.
> 
> Regards,
> 
> Ivan
> 
> 
> 
> 08.10.2019 1:38, Matthias J. Sax пишет:
>> I am moving this KIP into "inactive status". Feel free to resume the KIP
>> at any point.
>>
>> If anybody else is interested in picking up this KIP, feel free to do so.
>>
>>
>>
>> -Matthias
>>
>> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>>> Ivan,
>>>
>>> did you see my last reply? What do you think about my proposal to mix
>>> both approaches and try to get best-of-both worlds?
>>>
>>>
>>> -Matthias
>>>
>>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>>> Thanks for the input John!
>>>>
>>>>> under your suggestion, it seems that the name is required
>>>>
>>>> If you want to get the `KStream` as part of the `Map` back using a
>>>> `Function`, yes. If you follow the "embedded chaining" pattern using a
>>>> `Consumer`, no.
>>>>
>>>> Allowing for a default name via `split()` can of course be done.
>>>> Similarly, using `Named` instead of `String` is possible.
>>>>
>>>> I wanted to sketch out a high level proposal to merge both patterns
>>>> only. Your suggestions to align the new API with the existing API make
>>>> totally sense.
>>>>
>>>>
>>>>
>>>> One follow up question: Would `Named` be optional or required in
>>>> `split()` and `branch()`? It's unclear from your example.
>>>>
>>>> If both are mandatory, what do we gain by it? The returned `Map` only
>>>> contains the corresponding branches, so why should we prefix all of
>>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>>> `split()`, the same question raises?
>>>>
>>>> Requiring `Named` in `split()` seems only to make sense, if `Named` is
>>>> optional in `branch()` and we generate `-X` suffix using a counter for
>>>> different branch name. However, this might lead to the problem of
>>>> changing names if branches are added/removed. Also, how would the names
>>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>>> returned in the `Map`).
>>>>
>>>> If `Named` is optional for both, it could happen that a user misses to
>>>> specify a name for a branch what would lead to runtime issues.
>>>>
>>>>
>>>> Hence, I am actually in favor to not allow a default name but keep
>>>> `split()` without parameter and make `Named` in `branch()` required
>>>> if a
>>>> `Function` is used. This makes it explicit to the user that
>>>> specifying a
>>>> name is required if a `Function` is used.
>>>>
>>>>
>>>>
>>>> About
>>>>
>>>>> KBranchedStream#branch(BranchConfig)
>>>>
>>>> I don't think that the branching predicate is a configuration and hence
>>>> would not include it in a configuration object.
>>>>
>>>>>      withChain(...);
>>>>
>>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>>>> seem to be a configuration. We can also not prevent a user to call
>>>> `withName()` in combination of `withChain()` what does not make sense
>>>> IMHO. We could of course throw an RTE but not have a compile time check
>>>> seems less appealing. Also, it could happen that neither `withChain()`
>>>> not `withName()` is called and the branch is missing in the returned
>>>> `Map` what lead to runtime issues, too.
>>>>
>>>> Hence, I don't think that we should add `BranchConfig`. A config object
>>>> is helpful if each configuration can be set independently of all
>>>> others,
>>>> but this seems not to be the case here. If we add new configuration
>>>> later, we can also just move forward by deprecating the methods that
>>>> accept `Named` and add new methods that accepted `BranchConfig` (that
>>>> would of course implement `Named`).
>>>>
>>>>
>>>> Thoughts?
>>>>
>>>>
>>>> @Ivan, what do you think about the general idea to blend the two main
>>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>>
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>>
>>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>>>> everyone. Returning the map from the terminal operations also solves
>>>>> the problem of merging/joining the branched streams, if we want to add
>>>>> support for the compliment later on.
>>>>>
>>>>> Under your suggestion, it seems that the name is required. Otherwise,
>>>>> we wouldn't have keys for the map to return. I this this is actually
>>>>> not too bad, since experience has taught us that, although names for
>>>>> operations are not required to define stream processing logic, it does
>>>>> significantly improve the operational experience when you can map the
>>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>>> wouldn't (have to) reference the name to chain extra processing onto
>>>>> the branch (thanks to the second argument), you can avoid the
>>>>> "unchecked name" problem that Ivan pointed out.
>>>>>
>>>>> In the current implementation of Branch, you can name the branch
>>>>> operator itself, and then all the branches get index-suffixed names
>>>>> built from the branch operator name. I guess under this proposal, we
>>>>> could naturally append the branch name to the branching operator name,
>>>>> like this:
>>>>>
>>>>>     stream.split(Named.withName("mysplit")) //creates node "mysplit"
>>>>>                .branch(..., ..., "abranch") // creates node
>>>>> "mysplit-abranch"
>>>>>                .defaultBranch(...) // creates node "mysplit-default"
>>>>>
>>>>> It does make me wonder about the DSL syntax itself, though.
>>>>>
>>>>> We don't have a defined grammar, so there's plenty of room to debate
>>>>> the "best" syntax in the context of each operation, but in general,
>>>>> the KStream DSL operators follow this pattern:
>>>>>
>>>>>      operator(function, config_object?) OR operator(config_object)
>>>>>
>>>>> where config_object is often just Named in the "function" variant.
>>>>> Even when the config_object isn't a Named, but some other config
>>>>> class, that config class _always_ implements NamedOperation.
>>>>>
>>>>> Here, we're introducing a totally different pattern:
>>>>>
>>>>>    operator(function, function, string)
>>>>>
>>>>> where the string is the name.
>>>>> My first question is whether the name should instead be specified with
>>>>> the NamedOperation interface.
>>>>>
>>>>> My second question is whether we should just roll all these arguments
>>>>> up into a config object like:
>>>>>
>>>>>     KBranchedStream#branch(BranchConfig)
>>>>>
>>>>>     interface BranchConfig extends NamedOperation {
>>>>>      withPredicate(...);
>>>>>      withChain(...);
>>>>>      withName(...);
>>>>>    }
>>>>>
>>>>> Although I guess we'd like to call BranchConfig something more like
>>>>> "Branched", even if I don't particularly like that pattern.
>>>>>
>>>>> This makes the source code a little noisier, but it also makes us more
>>>>> future-proof, as we can deal with a wide range of alternatives purely
>>>>> in the config interface, and never have to deal with adding overloads
>>>>> to the KBranchedStream if/when we decide we want the name to be
>>>>> optional, or the KStream->KStream to be optional.
>>>>>
>>>>> WDYT?
>>>>>
>>>>> Thanks,
>>>>> -John
>>>>>
>>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>>> <mi...@confluent.io> wrote:
>>>>>>
>>>>>> Matthias: I think that's pretty reasonable from my point of view.
>>>>>> Good
>>>>>> suggestion.
>>>>>>
>>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax
>>>>>> <ma...@confluent.io>
>>>>>> wrote:
>>>>>>
>>>>>>> Interesting discussion.
>>>>>>>
>>>>>>> I am wondering, if we cannot unify the advantage of both approaches:
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> KStream#split() -> KBranchedStream
>>>>>>>
>>>>>>> // branch is not easily accessible in current scope
>>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>>    -> KBranchedStream
>>>>>>>
>>>>>>> // assign a name to the branch and
>>>>>>> // return the sub-stream to the current scope later
>>>>>>> //
>>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>>    -> KBranchedStream
>>>>>>>
>>>>>>> // default branch is not easily accessible
>>>>>>> // return map of all named sub-stream into current scope
>>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>>    -> Map<String,KStream>
>>>>>>>
>>>>>>> // assign custom name to default-branch
>>>>>>> // return map of all named sub-stream into current scope
>>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>>    -> Map<String,KStream>
>>>>>>>
>>>>>>> // assign a default name for default
>>>>>>> // return map of all named sub-stream into current scope
>>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>>    -> Map<String,KStream>
>>>>>>>
>>>>>>> // return map of all names sub-stream into current scope
>>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>>    -> Map<String,KStream>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Hence, for each sub-stream, the user can pick to add a name and
>>>>>>> return
>>>>>>> the branch "result" to the calling scope or not. The
>>>>>>> implementation can
>>>>>>> also check at runtime that all returned names are unique. The
>>>>>>> returned
>>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>>
>>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>>> Ivan,
>>>>>>>>
>>>>>>>> That's a very good point about the "start" operator in the
>>>>>>>> dynamic case.
>>>>>>>> I had no problem with "split()"; I was just questioning the
>>>>>>>> necessity.
>>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>>>> "split()" start operator. Thanks!
>>>>>>>>
>>>>>>>> Separately, I'm interested to see where the present discussion
>>>>>>>> leads.
>>>>>>>> I've written enough Javascript code in my life to be suspicious of
>>>>>>>> nested closures. You have a good point about using method
>>>>>>>> references (or
>>>>>>>> indeed function literals also work). It should be validating
>>>>>>>> that this
>>>>>>>> was also the JS community's first approach to flattening the
>>>>>>>> logic when
>>>>>>>> their nested closure situation got out of hand. Unfortunately, it's
>>>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>>>> readability (but in different ways for different reasons). In other
>>>>>>>> words, I agree that function references is *the* first-order
>>>>>>>> solution if
>>>>>>>> the nested code does indeed become a problem.
>>>>>>>>
>>>>>>>> However, the history of JS also tells us that function
>>>>>>>> references aren't
>>>>>>>> the end of the story either, and you can see that by observing that
>>>>>>>> there have been two follow-on eras, as they continue trying to
>>>>>>>> cope with
>>>>>>>> the consequences of living in such a callback-heavy language.
>>>>>>>> First, you
>>>>>>>> have Futures/Promises, which essentially let you convert nested
>>>>>>>> code to
>>>>>>>> method-chained code (Observables/FP is a popular variation on
>>>>>>>> this).
>>>>>>>> Most lately, you have async/await, which is an effort to apply
>>>>>>>> language
>>>>>>>> (not just API) syntax to the problem, and offer the "flattest"
>>>>>>>> possible
>>>>>>>> programming style to solve the problem (because you get back to
>>>>>>>> just one
>>>>>>>> code block per functional unit).
>>>>>>>>
>>>>>>>> Stream-processing is a different domain, and Java+KStreams is
>>>>>>>> nowhere
>>>>>>>> near as callback heavy as JS, so I don't think we have to take
>>>>>>>> the JS
>>>>>>>> story for granted, but then again, I think we can derive some
>>>>>>>> valuable
>>>>>>>> lessons by looking sideways to adjacent domains. I'm just
>>>>>>>> bringing this
>>>>>>>> up to inspire further/deeper discussion. At the same time, just
>>>>>>>> like JS,
>>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>>
>>>>>>>> Separately again, I'm interested in the post-branch merge (and
>>>>>>>> I'd also
>>>>>>>> add join) problem that Paul brought up. We can clearly punt on
>>>>>>>> it, by
>>>>>>>> terminating the nested branches with sink operators. But is
>>>>>>>> there a DSL
>>>>>>>> way to do it?
>>>>>>>>
>>>>>>>> Thanks again for your driving this,
>>>>>>>> -John
>>>>>>>>
>>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>>
>>>>>>>>      Ivan, I’ll definitely forfeit my point on the clumsiness of
>>>>>>>> the
>>>>>>>>      branch(predicate, consumer) solution, I don’t see any real
>>>>>>>> drawbacks
>>>>>>>>      for the dynamic case.
>>>>>>>>
>>>>>>>>      IMO the one trade off to consider at this point is the scope
>>>>>>>>      question. I don’t know if I totally agree that “we rarely
>>>>>>>> need them
>>>>>>>>      in the same scope” since merging the branches back together
>>>>>>>> later
>>>>>>>>      seems like a perfectly plausible use case that can be a lot
>>>>>>>> nicer
>>>>>>>>      when the branched streams are in the same scope. That being
>>>>>>>> said,
>>>>>>>>      for the reasons Ivan listed, I think it is overall the better
>>>>>>>>      solution - working around the scope thing is easy enough if
>>>>>>>> you need
>>>>>>>>      to.
>>>>>>>>
>>>>>>>>      > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>>      <ip...@mail.ru.invalid> wrote:
>>>>>>>>      >
>>>>>>>>      > Hello everyone, thank you all for joining the discussion!
>>>>>>>>      >
>>>>>>>>      > Well, I don't think the idea of named branches, be it a
>>>>>>>>      LinkedHashMap (no other Map will do, because order of
>>>>>>>> definition
>>>>>>>>      matters) or `branch` method  taking name and Consumer has more
>>>>>>>>      advantages than drawbacks.
>>>>>>>>      >
>>>>>>>>      > In my opinion, the only real positive outcome from Michael's
>>>>>>>>      proposal is that all the returned branches are in the same
>>>>>>>> scope.
>>>>>>>>      But 1) we rarely need them in the same scope 2) there is a
>>>>>>>>      workaround for the scope problem, described in the KIP.
>>>>>>>>      >
>>>>>>>>      > 'Inlining the complex logic' is not a problem, because we
>>>>>>>> can use
>>>>>>>>      method references instead of lambdas. In real world
>>>>>>>> scenarios you
>>>>>>>>      tend to split the complex logic to methods anyway, so the
>>>>>>>> code is
>>>>>>>>      going to be clean.
>>>>>>>>      >
>>>>>>>>      > The drawbacks are strong. The cohesion between predicates
>>>>>>>> and
>>>>>>>>      handlers is lost. We have to define predicates in one
>>>>>>>> place, and
>>>>>>>>      handlers in another. This opens the door for bugs:
>>>>>>>>      >
>>>>>>>>      > - what if we forget to define a handler for a name? or a
>>>>>>>> name for
>>>>>>>>      a handler?
>>>>>>>>      > - what if we misspell a name?
>>>>>>>>      > - what if we copy-paste and duplicate a name?
>>>>>>>>      >
>>>>>>>>      > What Michael propose would have been totally OK if we had
>>>>>>>> been
>>>>>>>>      writing the API in Lua, Ruby or Python. In those languages the
>>>>>>>>      "dynamic naming" approach would have looked most concise and
>>>>>>>>      beautiful. But in Java we expect all the problems related to
>>>>>>>>      identifiers to be eliminated in compile time.
>>>>>>>>      >
>>>>>>>>      > Do we have to invent duck-typing for the Java API?
>>>>>>>>      >
>>>>>>>>      > And if we do, what advantage are we supposed to get
>>>>>>>> besides having
>>>>>>>>      all the branches in the same scope? Michael, maybe I'm
>>>>>>>> missing your
>>>>>>>>      point?
>>>>>>>>      >
>>>>>>>>      > ---
>>>>>>>>      >
>>>>>>>>      > Earlier in this discussion John Roesler also proposed to do
>>>>>>>>      without "start branching" operator, and later Paul
>>>>>>>> mentioned that in
>>>>>>>>      the case when we have to add a dynamic number of branches, the
>>>>>>>>      current KIP is 'clumsier' compared to Michael's 'Map'
>>>>>>>> solution. Let
>>>>>>>>      me address both comments here.
>>>>>>>>      >
>>>>>>>>      > 1) "Start branching" operator (I think that *split* is a
>>>>>>>> good name
>>>>>>>>      for it indeed) is critical when we need to do a dynamic
>>>>>>>> branching,
>>>>>>>>      see example below.
>>>>>>>>      >
>>>>>>>>      > 2) No, dynamic branching in current KIP is not clumsy at
>>>>>>>> all.
>>>>>>>>      Imagine a real-world scenario when you need one branch per
>>>>>>>> enum
>>>>>>>>      value (say, RecordType). You can have something like this:
>>>>>>>>      >
>>>>>>>>      > /*John:if we had to start with stream.branch(...) here,
>>>>>>>> it would
>>>>>>>>      have been much messier.*/
>>>>>>>>      > KBranchedStream branched = stream.split();
>>>>>>>>      >
>>>>>>>>      > /*Not clumsy at all :-)*/
>>>>>>>>      > for (RecordType recordType : RecordType.values())
>>>>>>>>      >             branched = branched.branch((k, v) ->
>>>>>>>> v.getRecType() ==
>>>>>>>>      recordType,
>>>>>>>>      >                     recordType::processRecords);
>>>>>>>>      >
>>>>>>>>      > Regards,
>>>>>>>>      >
>>>>>>>>      > Ivan
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>>      >> I also agree with Michael's observation about the core
>>>>>>>> problem of
>>>>>>>>      >> current `branch()` implementation.
>>>>>>>>      >>
>>>>>>>>      >> However, I also don't like to pass in a clumsy Map
>>>>>>>> object. My
>>>>>>>>      thinking
>>>>>>>>      >> was more aligned with Paul's proposal to just add a name
>>>>>>>> to each
>>>>>>>>      >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>>>      >>
>>>>>>>>      >> It makes the code easier to read, and also make the
>>>>>>>> order of
>>>>>>>>      >> `Predicates` (that is essential) easier to grasp.
>>>>>>>>      >>
>>>>>>>>      >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>      >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>      >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>      >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>>      >> An open question is the case for which no
>>>>>>>> defaultBranch() should
>>>>>>> be
>>>>>>>>      >> specified. Atm, `split()` and `branch()` would return
>>>>>>>>      `BranchedKStream`
>>>>>>>>      >> and the call to `defaultBranch()` that returns the `Map` is
>>>>>>> mandatory
>>>>>>>>      >> (what is not the case atm). Or is this actually not a real
>>>>>>> problem,
>>>>>>>>      >> because users can just ignore the branch returned by
>>>>>>>>      `defaultBranch()`
>>>>>>>>      >> in the result `Map` ?
>>>>>>>>      >>
>>>>>>>>      >>
>>>>>>>>      >> About "inlining": So far, it seems to be a matter of
>>>>>>>> personal
>>>>>>>>      >> preference. I can see arguments for both, but no "killer
>>>>>>>>      argument" yet
>>>>>>>>      >> that clearly make the case for one or the other.
>>>>>>>>      >>
>>>>>>>>      >>
>>>>>>>>      >> -Matthias
>>>>>>>>      >>
>>>>>>>>      >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>>      >>> Perhaps inlining is the wrong terminology. It doesn’t
>>>>>>>> require
>>>>>>>>      that a lambda with the full downstream topology be defined
>>>>>>>> inline -
>>>>>>>>      it can be a method reference as with Ivan’s original
>>>>>>>> suggestion.
>>>>>>>>      The advantage of putting the predicate and its downstream
>>>>>>>> logic
>>>>>>>>      (Consumer) together in branch() is that they are required
>>>>>>>> to be near
>>>>>>>>      to each other.
>>>>>>>>      >>>
>>>>>>>>      >>> Ultimately the downstream code has to live somewhere,
>>>>>>>> and deep
>>>>>>>>      branch trees will be hard to read regardless.
>>>>>>>>      >>>
>>>>>>>>      >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>>      <michael.drogalis@confluent.io
>>>>>>>>      <ma...@confluent.io>> wrote:
>>>>>>>>      >>>>
>>>>>>>>      >>>> I'm less enthusiastic about inlining the branch logic
>>>>>>>> with its
>>>>>>>>      downstream
>>>>>>>>      >>>> functionality. Programs that have deep branch trees will
>>>>>>>>      quickly become
>>>>>>>>      >>>> harder to read as a single unit.
>>>>>>>>      >>>>
>>>>>>>>      >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>>      <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Also +1 on the issues/goals as Michael outlined them,
>>>>>>>> I think
>>>>>>>>      that sets a
>>>>>>>>      >>>>> great framework for the discussion.
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Regarding the SortedMap solution, my understanding is
>>>>>>>> that the
>>>>>>>>      current
>>>>>>>>      >>>>> proposal in the KIP is what is in my PR which
>>>>>>>> (pending naming
>>>>>>>>      decisions) is
>>>>>>>>      >>>>> roughly this:
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> stream.split()
>>>>>>>>      >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>      >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>>      >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Obviously some ordering is necessary, since branching
>>>>>>>> as a
>>>>>>>>      construct
>>>>>>>>      >>>>> doesn't work without it, but this solution seems like it
>>>>>>>>      provides as much
>>>>>>>>      >>>>> associativity as the SortedMap solution, because each
>>>>>>>> branch()
>>>>>>>>      call
>>>>>>>>      >>>>> directly associates the "conditional" with the "code
>>>>>>>> block."
>>>>>>>>      The value it
>>>>>>>>      >>>>> provides over the KIP solution is the accessing of
>>>>>>>> streams in
>>>>>>>>      the same
>>>>>>>>      >>>>> scope.
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> The KIP solution is less "dynamic" than the SortedMap
>>>>>>>> solution
>>>>>>>>      in the sense
>>>>>>>>      >>>>> that it is slightly clumsier to add a dynamic number of
>>>>>>>>      branches, but it is
>>>>>>>>      >>>>> certainly possible.  It seems to me like the API
>>>>>>>> should favor
>>>>>>>>      the "static"
>>>>>>>>      >>>>> case anyway, and should make it simple and readable to
>>>>>>>>      fluently declare and
>>>>>>>>      >>>>> access your branches in-line.  It also makes it
>>>>>>>> impossible to
>>>>>>>>      ignore a
>>>>>>>>      >>>>> branch, and it is possible to build an (almost)
>>>>>>>> identical
>>>>>>>>      SortedMap
>>>>>>>>      >>>>> solution on top of it.
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> I could also see a middle ground where instead of a raw
>>>>>>>>      SortedMap being
>>>>>>>>      >>>>> taken in, branch() takes a name and not a Consumer. 
>>>>>>>> Something
>>>>>>>>      like this:
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>>      >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>>      >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>>      >>>>>    .defaultBranch("defaultBranch",
>>>>>>>> Consumer<KStream<K, V>>);
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Pros for that solution:
>>>>>>>>      >>>>> - accessing branched KStreams in same scope
>>>>>>>>      >>>>> - no double brace initialization, hopefully slightly
>>>>>>>> more
>>>>>>>>      readable than
>>>>>>>>      >>>>> SortedMap
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Cons
>>>>>>>>      >>>>> - downstream branch logic cannot be specified inline
>>>>>>>> which
>>>>>>>>      makes it harder
>>>>>>>>      >>>>> to read top to bottom (like existing API and
>>>>>>>> SortedMap, but
>>>>>>>>      unlike the KIP)
>>>>>>>>      >>>>> - you can forget to "handle" one of the branched
>>>>>>>> streams (like
>>>>>>>>      existing
>>>>>>>>      >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> (KBranchedStreams could even work *both* ways but
>>>>>>>> perhaps
>>>>>>>>      that's overdoing
>>>>>>>>      >>>>> it).
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Overall I'm curious how important it is to be able to
>>>>>>>> easily
>>>>>>>>      access the
>>>>>>>>      >>>>> branched KStream in the same scope as the original. 
>>>>>>>> It's
>>>>>>>>      possible that it
>>>>>>>>      >>>>> doesn't need to be handled directly by the API, but
>>>>>>>> instead
>>>>>>>>      left up to the
>>>>>>>>      >>>>> user.  I'm sort of in the middle on it.
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> Paul
>>>>>>>>      >>>>>
>>>>>>>>      >>>>>
>>>>>>>>      >>>>>
>>>>>>>>      >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>>>      <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>>      >>>>> wrote:
>>>>>>>>      >>>>>
>>>>>>>>      >>>>>> I'd like to +1 what Michael said about the issues
>>>>>>>> with the
>>>>>>>>      existing
>>>>>>>>      >>>>> branch
>>>>>>>>      >>>>>> method, I agree with what he's outlined and I think
>>>>>>>> we should
>>>>>>>>      proceed by
>>>>>>>>      >>>>>> trying to alleviate these problems. Specifically it
>>>>>>>> seems
>>>>>>>>      important to be
>>>>>>>>      >>>>>> able to cleanly access the individual branches (eg
>>>>>>>> by mapping
>>>>>>>>      >>>>>> name->stream), which I thought was the original
>>>>>>>> intention of
>>>>>>>>      this KIP.
>>>>>>>>      >>>>>>
>>>>>>>>      >>>>>> That said, I don't think we should so easily give in
>>>>>>>> to the
>>>>>>>>      double brace
>>>>>>>>      >>>>>> anti-pattern or force ours users into it if at all
>>>>>>>> possible to
>>>>>>>>      >>>>> avoid...just
>>>>>>>>      >>>>>> my two cents.
>>>>>>>>      >>>>>>
>>>>>>>>      >>>>>> Cheers,
>>>>>>>>      >>>>>> Sophie
>>>>>>>>      >>>>>>
>>>>>>>>      >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>>>      >>>>>> michael.drogalis@confluent.io
>>>>>>>>      <ma...@confluent.io>> wrote:
>>>>>>>>      >>>>>>
>>>>>>>>      >>>>>>> I’d like to propose a different way of thinking
>>>>>>>> about this.
>>>>>>>>      To me,
>>>>>>>>      >>>>> there
>>>>>>>>      >>>>>>> are three problems with the existing branch signature:
>>>>>>>>      >>>>>>>
>>>>>>>>      >>>>>>> 1. If you use it the way most people do, Java
>>>>>>>> raises unsafe
>>>>>>> type
>>>>>>>>      >>>>>> warnings.
>>>>>>>>      >>>>>>> 2. The way in which you use the stream branches is
>>>>>>>>      positionally coupled
>>>>>>>>      >>>>>> to
>>>>>>>>      >>>>>>> the ordering of the conditionals.
>>>>>>>>      >>>>>>> 3. It is brittle to extend existing branch calls with
>>>>>>>>      additional code
>>>>>>>>      >>>>>>> paths.
>>>>>>>>      >>>>>>>
>>>>>>>>      >>>>>>> Using associative constructs instead of relying on
>>>>>>>> ordered
>>>>>>>>      constructs
>>>>>>>>      >>>>>> would
>>>>>>>>      >>>>>>> be a stronger approach. Consider a signature that
>>>>>>>> instead
>>>>>>>>      looks like
>>>>>>>>      >>>>>> this:
>>>>>>>>      >>>>>>> Map<String, KStream<K,V>>
>>>>>>>> KStream#branch(SortedMap<String,
>>>>>>>>      Predicate<?
>>>>>>>>      >>>>>>> super K,? super V>>);
>>>>>>>>      >>>>>>>
>>>>>>>>      >>>>>>> Branches are given names in a map, and as a result,
>>>>>>>> the API
>>>>>>>>      returns a
>>>>>>>>      >>>>>>> mapping of names to streams. The ordering of the
>>>>>>> conditionals is
>>>>>>>>      >>>>>> maintained
>>>>>>>>      >>>>>>> because it’s a sorted map. Insert order determines
>>>>>>>> the order
>>>>>>> of
>>>>>>>>      >>>>>> evaluation.
>>>>>>>>      >>>>>>> This solves problem 1 because there are no more
>>>>>>>> varargs. It
>>>>>>>>      solves
>>>>>>>>      >>>>>> problem
>>>>>>>>      >>>>>>> 2 because you no longer lean on ordering to access the
>>>>>>>>      branch you’re
>>>>>>>>      >>>>>>> interested in. It solves problem 3 because you can
>>>>>>>> introduce
>>>>>>>>      another
>>>>>>>>      >>>>>>> conditional by simply attaching another name to the
>>>>>>>>      structure, rather
>>>>>>>>      >>>>>> than
>>>>>>>>      >>>>>>> messing with the existing indices.
>>>>>>>>      >>>>>>>
>>>>>>>>      >>>>>>> One of the drawbacks is that creating the map
>>>>>>>> inline is
>>>>>>>>      historically
>>>>>>>>      >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>>>      voluminously, but
>>>>>>>>      >>>>>>> double brace initialization would clean up the
>>>>>>>> aesthetics.
>>>>>>>>      >>>>>>>
>>>>>>>>      >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>>      <john@confluent.io <ma...@confluent.io>>
>>>>>>>>      >>>>> wrote:
>>>>>>>>      >>>>>>>> Hi Ivan,
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>> Thanks for the update.
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>> FWIW, I agree with Matthias that the current "start
>>>>>>> branching"
>>>>>>>>      >>>>> operator
>>>>>>>>      >>>>>>> is
>>>>>>>>      >>>>>>>> confusing when named the same way as the actual
>>>>>>>> branches.
>>>>>>>>      "Split"
>>>>>>>>      >>>>> seems
>>>>>>>>      >>>>>>>> like a good name. Alternatively, we can do without
>>>>>>>> a "start
>>>>>>>>      >>>>> branching"
>>>>>>>>      >>>>>>>> operator at all, and just do:
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>> stream
>>>>>>>>      >>>>>>>>      .branch(Predicate)
>>>>>>>>      >>>>>>>>      .branch(Predicate)
>>>>>>>>      >>>>>>>>      .defaultBranch();
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>> Tentatively, I think that this branching operation
>>>>>>>> should be
>>>>>>>>      >>>>> terminal.
>>>>>>>>      >>>>>>> That
>>>>>>>>      >>>>>>>> way, we don't create ambiguity about how to use
>>>>>>>> it. That
>>>>>>>>      is, `branch`
>>>>>>>>      >>>>>>>> should return `KBranchedStream`, while
>>>>>>>> `defaultBranch` is
>>>>>>>>      `void`, to
>>>>>>>>      >>>>>>>> enforce that it comes last, and that there is only
>>>>>>>> one
>>>>>>>>      definition of
>>>>>>>>      >>>>>> the
>>>>>>>>      >>>>>>>> default branch. Potentially, we should log a
>>>>>>>> warning if
>>>>>>>>      there's no
>>>>>>>>      >>>>>>> default,
>>>>>>>>      >>>>>>>> and additionally log a warning (or throw an
>>>>>>>> exception) if a
>>>>>>>>      record
>>>>>>>>      >>>>>> falls
>>>>>>>>      >>>>>>>> though with no default.
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>> Thoughts?
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>> Thanks,
>>>>>>>>      >>>>>>>> -John
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>>>      >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>>>      >>>>>>>> wrote:
>>>>>>>>      >>>>>>>>
>>>>>>>>      >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>> this is to make the name similar to String#split
>>>>>>>>      >>>>>>>>>>> that also returns an array, right?
>>>>>>>>      >>>>>>>>> The intend was to avoid name duplication. The
>>>>>>>> return type
>>>>>>>>      should
>>>>>>>>      >>>>>> _not_
>>>>>>>>      >>>>>>>>> be an array.
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> The current proposal is
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> stream.branch()
>>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>>      >>>>>>>>>      .defaultBranch();
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>>>      `branch()` does
>>>>>>>>      >>>>> not
>>>>>>>>      >>>>>>>>> take any parameters and has different semantics
>>>>>>>> than the
>>>>>>> later
>>>>>>>>      >>>>>>>>> `branch()` calls. Note, that from the code
>>>>>>>> snippet above,
>>>>>>> it's
>>>>>>>>      >>>>> hidden
>>>>>>>>      >>>>>>>>> that the first call is `KStream#branch()` while
>>>>>>>> the others
>>>>>>> are
>>>>>>>>      >>>>>>>>> `KBranchedStream#branch()` what makes reading the
>>>>>>>> code
>>>>>>> harder.
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> Because I suggested to rename `addBranch()` ->
>>>>>>>> `branch()`,
>>>>>>>>      I though
>>>>>>>>      >>>>>> it
>>>>>>>>      >>>>>>>>> might be better to also rename `KStream#branch()`
>>>>>>>> to avoid
>>>>>>> the
>>>>>>>>      >>>>> naming
>>>>>>>>      >>>>>>>>> overlap that seems to be confusing. The following
>>>>>>>> reads
>>>>>>> much
>>>>>>>>      >>>>> cleaner
>>>>>>>>      >>>>>> to
>>>>>>>>      >>>>>>>> me:
>>>>>>>>      >>>>>>>>> stream.split()
>>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>>      >>>>>>>>>      .defaultBranch();
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> Maybe there is a better alternative to `split()`
>>>>>>>> though to
>>>>>>>>      avoid
>>>>>>>>      >>>>> the
>>>>>>>>      >>>>>>>>> naming overlap.
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>> 'default' is, however, a reserved word, so
>>>>>>>> unfortunately
>>>>>>> we
>>>>>>>>      >>>>> cannot
>>>>>>>>      >>>>>>> have
>>>>>>>>      >>>>>>>>> a method with such name :-)
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> Bummer. Didn't consider this. Maybe we can still
>>>>>>>> come up
>>>>>>>>      with a
>>>>>>>>      >>>>> short
>>>>>>>>      >>>>>>>> name?
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> Can you add the interface `KBranchedStream` to
>>>>>>>> the KIP
>>>>>>>>      with all
>>>>>>>>      >>>>> it's
>>>>>>>>      >>>>>>>>> methods? It will be part of public API and should be
>>>>>>>>      contained in
>>>>>>>>      >>>>> the
>>>>>>>>      >>>>>>>>> KIP. For example, it's unclear atm, what the
>>>>>>>> return type of
>>>>>>>>      >>>>>>>>> `defaultBranch()` is.
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> You did not comment on the idea to add a
>>>>>>>>      `KBranchedStream#get(int
>>>>>>>>      >>>>>>> index)
>>>>>>>>      >>>>>>>>> -> KStream` method to get the individually
>>>>>>>>      branched-KStreams. Would
>>>>>>>>      >>>>>> be
>>>>>>>>      >>>>>>>>> nice to get your feedback about it. It seems you
>>>>>>>> suggest
>>>>>>>>      that users
>>>>>>>>      >>>>>>>>> would need to write custom utility code
>>>>>>>> otherwise, to
>>>>>>>>      access them.
>>>>>>>>      >>>>> We
>>>>>>>>      >>>>>>>>> should discuss the pros and cons of both
>>>>>>>> approaches. It
>>>>>>> feels
>>>>>>>>      >>>>>>>>> "incomplete" to me atm, if the API has no
>>>>>>>> built-in support
>>>>>>>>      to get
>>>>>>>>      >>>>> the
>>>>>>>>      >>>>>>>>> branched-KStreams directly.
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>> -Matthias
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>      >>>>>>>>>> Hi all!
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> I have updated the KIP-418 according to the new
>>>>>>>> vision.
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>      >>>>>>>>>> I can see your point: this is to make the name
>>>>>>>> similar to
>>>>>>>>      >>>>>>> String#split
>>>>>>>>      >>>>>>>>>> that also returns an array, right? But is it
>>>>>>>> worth the
>>>>>>>>      loss of
>>>>>>>>      >>>>>>>> backwards
>>>>>>>>      >>>>>>>>>> compatibility? We can have overloaded branch()
>>>>>>>> as well
>>>>>>>>      without
>>>>>>>>      >>>>>>>> affecting
>>>>>>>>      >>>>>>>>>> the existing code. Maybe the old array-based
>>>>>>>> `branch`
>>>>>>> method
>>>>>>>>      >>>>> should
>>>>>>>>      >>>>>>> be
>>>>>>>>      >>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>>      >>>>> BranchingKStream#branch(),
>>>>>>>>      >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>> BranchingKStream#default()
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> Totally agree with 'addBranch->branch' rename.
>>>>>>>> 'default'
>>>>>>> is,
>>>>>>>>      >>>>>>> however, a
>>>>>>>>      >>>>>>>>>> reserved word, so unfortunately we cannot have a
>>>>>>>> method
>>>>>>>>      with such
>>>>>>>>      >>>>>>> name
>>>>>>>>      >>>>>>>>> :-)
>>>>>>>>      >>>>>>>>>>> defaultBranch() does take an `Predicate` as
>>>>>>>> argument,
>>>>>>> but I
>>>>>>>>      >>>>> think
>>>>>>>>      >>>>>>> that
>>>>>>>>      >>>>>>>>>> is not required?
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> Absolutely! I think that was just copy-paste
>>>>>>>> error or
>>>>>>>>      something.
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> Dear colleagues,
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> please revise the new version of the KIP and
>>>>>>>> Paul's PR
>>>>>>>>      >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> Any new suggestions/objections?
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> Ivan
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>>
>>>>>>>>      >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>      >>>>>>>>>>> Thanks for driving the discussion of this KIP.
>>>>>>>> It seems
>>>>>>> that
>>>>>>>>      >>>>>>> everybody
>>>>>>>>      >>>>>>>>>>> agrees that the current branch() method using
>>>>>>>> arrays is
>>>>>>> not
>>>>>>>>      >>>>>> optimal.
>>>>>>>>      >>>>>>>>>>> I had a quick look into the PR and I like the
>>>>>>>> overall
>>>>>>>>      proposal.
>>>>>>>>      >>>>>>> There
>>>>>>>>      >>>>>>>>>>> are some minor things we need to consider. I would
>>>>>>>>      recommend the
>>>>>>>>      >>>>>>>>>>> following renaming:
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>      >>>>>>>>>>> KBranchedStream#addBranch() ->
>>>>>>>> BranchingKStream#branch()
>>>>>>>>      >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>>      BranchingKStream#default()
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> It's just a suggestion to get slightly shorter
>>>>>>>> method
>>>>>>> names.
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>>>      `Predicate` as
>>>>>>>>      >>>>>>>> argument,
>>>>>>>>      >>>>>>>>>>> but I think that is not required?
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> Also, we should consider KIP-307, that was
>>>>>>>> recently
>>>>>>>>      accepted and
>>>>>>>>      >>>>>> is
>>>>>>>>      >>>>>>>>>>> currently implemented:
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>
>>>>>>>>
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>
>>>>>>>>      >>>>>>>>>>> Ie, we should add overloads that accepted a
>>>>>>>> `Named`
>>>>>>>>      parameter.
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> For the issue that the created `KStream` object
>>>>>>>> are in
>>>>>>>>      different
>>>>>>>>      >>>>>>>> scopes:
>>>>>>>>      >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>>>>>>>>      index)` method
>>>>>>>>      >>>>>>> that
>>>>>>>>      >>>>>>>>>>> returns the corresponding "branched" result
>>>>>>>> `KStream`
>>>>>>>>      object?
>>>>>>>>      >>>>>> Maybe,
>>>>>>>>      >>>>>>>> the
>>>>>>>>      >>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>>>>>      >>>>>> `Consumer<KStream>`
>>>>>>>>      >>>>>>>> but
>>>>>>>>      >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could
>>>>>>>> return
>>>>>>>>      whatever
>>>>>>>>      >>>>>> the
>>>>>>>>      >>>>>>>>>>> `Function` returns?
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> Finally, I would also suggest to update the KIP
>>>>>>>> with the
>>>>>>>>      current
>>>>>>>>      >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>> -Matthias
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>      >>>>>>>>>>>> Ivan,
>>>>>>>>      >>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>> I'm a bit of a novice here as well, but I
>>>>>>>> think it
>>>>>>>>      makes sense
>>>>>>>>      >>>>>> for
>>>>>>>>      >>>>>>>> you
>>>>>>>>      >>>>>>>>> to
>>>>>>>>      >>>>>>>>>>>> revise the KIP and continue the discussion. 
>>>>>>>> Obviously
>>>>>>>>      we'll
>>>>>>>>      >>>>> need
>>>>>>>>      >>>>>>>> some
>>>>>>>>      >>>>>>>>>>>> buy-in from committers that have actual
>>>>>>>> binding votes on
>>>>>>>>      >>>>> whether
>>>>>>>>      >>>>>>> the
>>>>>>>>      >>>>>>>>> KIP
>>>>>>>>      >>>>>>>>>>>> could be adopted.  It would be great to hear
>>>>>>>> if they
>>>>>>>>      think this
>>>>>>>>      >>>>>> is
>>>>>>>>      >>>>>>> a
>>>>>>>>      >>>>>>>>> good
>>>>>>>>      >>>>>>>>>>>> idea overall.  I'm not sure if that happens
>>>>>>>> just by
>>>>>>>>      starting a
>>>>>>>>      >>>>>>> vote,
>>>>>>>>      >>>>>>>>> or if
>>>>>>>>      >>>>>>>>>>>> there is generally some indication of interest
>>>>>>> beforehand.
>>>>>>>>      >>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>> That being said, I'll continue the discussion
>>>>>>>> a bit:
>>>>>>>>      assuming
>>>>>>>>      >>>>> we
>>>>>>>>      >>>>>> do
>>>>>>>>      >>>>>>>>> move
>>>>>>>>      >>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>>>>>      >>>>>> KBranchedStream",
>>>>>>>>      >>>>>>> do
>>>>>>>>      >>>>>>>>> we
>>>>>>>>      >>>>>>>>>>>> deprecate "stream.branch(...) returns
>>>>>>>> KStream[]"?  I
>>>>>>> would
>>>>>>>>      >>>>> favor
>>>>>>>>      >>>>>>>>>>>> deprecating, since having two mutually
>>>>>>>> exclusive APIs
>>>>>>> that
>>>>>>>>      >>>>>>> accomplish
>>>>>>>>      >>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>> same thing is confusing, especially when
>>>>>>>> they're fairly
>>>>>>>>      similar
>>>>>>>>      >>>>>>>>> anyway.  We
>>>>>>>>      >>>>>>>>>>>> just need to be sure we're not making something
>>>>>>>>      >>>>>>> impossible/difficult
>>>>>>>>      >>>>>>>>> that
>>>>>>>>      >>>>>>>>>>>> is currently possible/easy.
>>>>>>>>      >>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>> Regarding my PR - I think the general
>>>>>>>> structure would
>>>>>>> work,
>>>>>>>>      >>>>> it's
>>>>>>>>      >>>>>>>> just a
>>>>>>>>      >>>>>>>>>>>> little sloppy overall in terms of naming and
>>>>>>>> clarity. In
>>>>>>>>      >>>>>>> particular,
>>>>>>>>      >>>>>>>>>>>> passing in the "predicates" and "children"
>>>>>>>> lists which
>>>>>>> get
>>>>>>>>      >>>>>> modified
>>>>>>>>      >>>>>>>> in
>>>>>>>>      >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>>      KStreamLazyBranch is
>>>>>>>>      >>>>> a
>>>>>>>>      >>>>>>> bit
>>>>>>>>      >>>>>>>>>>>> complicated to follow.
>>>>>>>>      >>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>> Thanks,
>>>>>>>>      >>>>>>>>>>>> Paul
>>>>>>>>      >>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>>>>>      >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>      >>>>>>>>> wrote:
>>>>>>>>      >>>>>>>>>>>>> Hi Paul!
>>>>>>>>      >>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>>>      convinced: your
>>>>>>>>      >>>>>>>> proposal
>>>>>>>>      >>>>>>>>>>>>> looks better and should work. We just have to
>>>>>>>> document
>>>>>>> the
>>>>>>>>      >>>>>> crucial
>>>>>>>>      >>>>>>>>> fact
>>>>>>>>      >>>>>>>>>>>>> that KStream consumers are invoked as they're
>>>>>>>> added.
>>>>>>>>      And then
>>>>>>>>      >>>>>> it's
>>>>>>>>      >>>>>>>> all
>>>>>>>>      >>>>>>>>>>>>> going to be very nice.
>>>>>>>>      >>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>> What shall we do now? I should re-write the
>>>>>>>> KIP and
>>>>>>>>      resume the
>>>>>>>>      >>>>>>>>>>>>> discussion here, right?
>>>>>>>>      >>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>> Why are you telling that your PR 'should not
>>>>>>>> be even a
>>>>>>>>      >>>>> starting
>>>>>>>>      >>>>>>>> point
>>>>>>>>      >>>>>>>>> if
>>>>>>>>      >>>>>>>>>>>>> we go in this direction'? To me it looks like
>>>>>>>> a good
>>>>>>>>      starting
>>>>>>>>      >>>>>>> point.
>>>>>>>>      >>>>>>>>> But
>>>>>>>>      >>>>>>>>>>>>> as a novice in this project I might miss some
>>>>>>>> important
>>>>>>>>      >>>>> details.
>>>>>>>>      >>>>>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>> Ivan
>>>>>>>>      >>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>      >>>>>>>>>>>>>> Ivan,
>>>>>>>>      >>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>>>>>      >>>>> stream.branch()
>>>>>>>>      >>>>>>>>> solution
>>>>>>>>      >>>>>>>>>>>>> supports this. The couponIssuer::set*
>>>>>>>> consumers will be
>>>>>>>>      >>>>> invoked
>>>>>>>>      >>>>>> as
>>>>>>>>      >>>>>>>>> they’re
>>>>>>>>      >>>>>>>>>>>>> added, not during streamsBuilder.build(). So
>>>>>>>> the user
>>>>>>>>      still
>>>>>>>>      >>>>>> ought
>>>>>>>>      >>>>>>> to
>>>>>>>>      >>>>>>>>> be
>>>>>>>>      >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward
>>>>>>>> and
>>>>>>>>      depend on
>>>>>>>>      >>>>> the
>>>>>>>>      >>>>>>>>> branched
>>>>>>>>      >>>>>>>>>>>>> streams having been set.
>>>>>>>>      >>>>>>>>>>>>>> The issue I mean to point out is that it is
>>>>>>>> hard to
>>>>>>>>      access
>>>>>>>>      >>>>> the
>>>>>>>>      >>>>>>>>> branched
>>>>>>>>      >>>>>>>>>>>>> streams in the same scope as the original
>>>>>>>> stream (that
>>>>>>>>      is, not
>>>>>>>>      >>>>>>>> inside
>>>>>>>>      >>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>> couponIssuer), which is a problem with both
>>>>>>>> proposed
>>>>>>>>      >>>>> solutions.
>>>>>>>>      >>>>>> It
>>>>>>>>      >>>>>>>>> can be
>>>>>>>>      >>>>>>>>>>>>> worked around though.
>>>>>>>>      >>>>>>>>>>>>>> [Also, great to hear additional interest in
>>>>>>>> 401, I’m
>>>>>>>>      excited
>>>>>>>>      >>>>> to
>>>>>>>>      >>>>>>>> hear
>>>>>>>>      >>>>>>>>>>>>> your thoughts!]
>>>>>>>>      >>>>>>>>>>>>>> Paul
>>>>>>>>      >>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>>>>>      >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>      >>>>>>>>> wrote:
>>>>>>>>      >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> The idea to postpone the wiring of branches
>>>>>>>> to the
>>>>>>>>      >>>>>>>>>>>>> streamsBuilder.build() also looked great for
>>>>>>>> me at
>>>>>>> first
>>>>>>>>      >>>>> glance,
>>>>>>>>      >>>>>>> but
>>>>>>>>      >>>>>>>>> ---
>>>>>>>>      >>>>>>>>>>>>>>>> the newly branched streams are not
>>>>>>>> available in the
>>>>>>>>      same
>>>>>>>>      >>>>>> scope
>>>>>>>>      >>>>>>> as
>>>>>>>>      >>>>>>>>> each
>>>>>>>>      >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
>>>>>>> together
>>>>>>>>      >>>>> again
>>>>>>>>      >>>>>> I
>>>>>>>>      >>>>>>>>> don't see
>>>>>>>>      >>>>>>>>>>>>> a way to do that.
>>>>>>>>      >>>>>>>>>>>>>>> You just took the words right out of my
>>>>>>>> mouth, I was
>>>>>>>>      just
>>>>>>>>      >>>>>> going
>>>>>>>>      >>>>>>> to
>>>>>>>>      >>>>>>>>>>>>> write in details about this issue.
>>>>>>>>      >>>>>>>>>>>>>>> Consider the example from Bill's book, p.
>>>>>>>> 101: say
>>>>>>>>      we need
>>>>>>>>      >>>>> to
>>>>>>>>      >>>>>>>>> identify
>>>>>>>>      >>>>>>>>>>>>> customers who have bought coffee and made a
>>>>>>>> purchase
>>>>>>>>      in the
>>>>>>>>      >>>>>>>>> electronics
>>>>>>>>      >>>>>>>>>>>>> store to give them coupons.
>>>>>>>>      >>>>>>>>>>>>>>> This is the code I usually write under these
>>>>>>>>      circumstances
>>>>>>>>      >>>>>> using
>>>>>>>>      >>>>>>>> my
>>>>>>>>      >>>>>>>>>>>>> 'brancher' class:
>>>>>>>>      >>>>>>>>>>>>>>> @Setter
>>>>>>>>      >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>      >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>      >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>      >>>>>>>>>>>>>>>       return
>>>>>>>>      >>>>>>>>>
>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>      >>>>>>>>>>>>>>>       /*In the real world the code here can be
>>>>>>>>      complex, so
>>>>>>>>      >>>>>>>>> creation of
>>>>>>>>      >>>>>>>>>>>>> a separate CouponIssuer class is fully
>>>>>>>> justified, in
>>>>>>>>      order to
>>>>>>>>      >>>>>>>> separate
>>>>>>>>      >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>      >>>>>>>>>>>>>>>  }
>>>>>>>>      >>>>>>>>>>>>>>> }
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new
>>>>>>>> CouponIssuer();
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>      >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>>      >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>>      >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>      >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to
>>>>>>>> wire up
>>>>>>>>      everything
>>>>>>>>      >>>>>>>> later,
>>>>>>>>      >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>      >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>>>      initialize the
>>>>>>>>      >>>>>>>>> CouponIssuer
>>>>>>>>      >>>>>>>>>>>>> we need the terminal operation to be called
>>>>>>>> before
>>>>>>>>      >>>>>>>>> streamsBuilder.build()
>>>>>>>>      >>>>>>>>>>>>> is called.
>>>>>>>>      >>>>>>>>>>>>>>> [BTW Paul, I just found out that your
>>>>>>>> KIP-401 is
>>>>>>>>      essentially
>>>>>>>>      >>>>>> the
>>>>>>>>      >>>>>>>>> next
>>>>>>>>      >>>>>>>>>>>>> KIP I was going to write here. I have some
>>>>>>>> thoughts
>>>>>>>>      based on
>>>>>>>>      >>>>> my
>>>>>>>>      >>>>>>>>> experience,
>>>>>>>>      >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>>>      >>>>>>>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> Ivan
>>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>      >>>>>>>>>>>>>>>> Ivan,
>>>>>>>>      >>>>>>>>>>>>>>>> I tried to make a very rough proof of
>>>>>>>> concept of a
>>>>>>>>      fluent
>>>>>>>>      >>>>> API
>>>>>>>>      >>>>>>>> based
>>>>>>>>      >>>>>>>>>>>>> off of
>>>>>>>>      >>>>>>>>>>>>>>>> KStream here
>>>>>>>>      (https://github.com/apache/kafka/pull/6512),
>>>>>>>>      >>>>>> and
>>>>>>>>      >>>>>>> I
>>>>>>>>      >>>>>>>>> think
>>>>>>>>      >>>>>>>>>>>>> I
>>>>>>>>      >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>      >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect
>>>>>>>> earlier about
>>>>>>>>      >>>>>>> compatibility
>>>>>>>>      >>>>>>>>>>>>> issues,
>>>>>>>>      >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was
>>>>>>>> unaware
>>>>>>>>      that Java
>>>>>>>>      >>>>> is
>>>>>>>>      >>>>>>>> smart
>>>>>>>>      >>>>>>>>>>>>> enough to
>>>>>>>>      >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>>>>>>>>      returning one
>>>>>>>>      >>>>>>> thing
>>>>>>>>      >>>>>>>>> and
>>>>>>>>      >>>>>>>>>>>>> branch()
>>>>>>>>      >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>>>      >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't
>>>>>>>> actually
>>>>>>>>      need
>>>>>>>>      >>>>> it.
>>>>>>>>      >>>>>>> We
>>>>>>>>      >>>>>>>>> can
>>>>>>>>      >>>>>>>>>>>>> just
>>>>>>>>      >>>>>>>>>>>>>>>>    build up the branches in the
>>>>>>>> KBranchedStream who
>>>>>>>>      shares
>>>>>>>>      >>>>>> its
>>>>>>>>      >>>>>>>>> state
>>>>>>>>      >>>>>>>>>>>>> with the
>>>>>>>>      >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do
>>>>>>>> the
>>>>>>>>      branching.
>>>>>>>>      >>>>>>> It's
>>>>>>>>      >>>>>>>>> not
>>>>>>>>      >>>>>>>>>>>>> terribly
>>>>>>>>      >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>>>>>>>>      demonstrates
>>>>>>>>      >>>>>> its
>>>>>>>>      >>>>>>>>>>>>> feasibility.
>>>>>>>>      >>>>>>>>>>>>>>>> To be clear, I don't think that pull
>>>>>>>> request should
>>>>>>> be
>>>>>>>>      >>>>> final
>>>>>>>>      >>>>>> or
>>>>>>>>      >>>>>>>>> even a
>>>>>>>>      >>>>>>>>>>>>>>>> starting point if we go in this direction,
>>>>>>>> I just
>>>>>>>>      wanted to
>>>>>>>>      >>>>>> see
>>>>>>>>      >>>>>>>> how
>>>>>>>>      >>>>>>>>>>>>>>>> challenging it would be to get the API
>>>>>>>> working.
>>>>>>>>      >>>>>>>>>>>>>>>> I will say though, that I'm not sure the
>>>>>>>> existing
>>>>>>>>      solution
>>>>>>>>      >>>>>>> could
>>>>>>>>      >>>>>>>> be
>>>>>>>>      >>>>>>>>>>>>>>>> deprecated in favor of this, which I had
>>>>>>>> originally
>>>>>>>>      >>>>> suggested
>>>>>>>>      >>>>>>>> was a
>>>>>>>>      >>>>>>>>>>>>>>>> possibility.  The reason is that the newly
>>>>>>>> branched
>>>>>>>>      streams
>>>>>>>>      >>>>>> are
>>>>>>>>      >>>>>>>> not
>>>>>>>>      >>>>>>>>>>>>>>>> available in the same scope as each
>>>>>>>> other.  That
>>>>>>>>      is, if we
>>>>>>>>      >>>>>>> wanted
>>>>>>>>      >>>>>>>>> to
>>>>>>>>      >>>>>>>>>>>>> merge
>>>>>>>>      >>>>>>>>>>>>>>>> them back together again I don't see a way
>>>>>>>> to do
>>>>>>>>      that.  The
>>>>>>>>      >>>>>> KIP
>>>>>>>>      >>>>>>>>>>>>> proposal
>>>>>>>>      >>>>>>>>>>>>>>>> has the same issue, though - all this
>>>>>>>> means is that
>>>>>>> for
>>>>>>>>      >>>>>> either
>>>>>>>>      >>>>>>>>>>>>> solution,
>>>>>>>>      >>>>>>>>>>>>>>>> deprecating the existing branch(...) is
>>>>>>>> not on the
>>>>>>>>      table.
>>>>>>>>      >>>>>>>>>>>>>>>> Thanks,
>>>>>>>>      >>>>>>>>>>>>>>>> Paul
>>>>>>>>      >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan
>>>>>>>> Ponomarev <
>>>>>>>>      >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>>>      >>>>>>>>>>>>> wrote:
>>>>>>>>      >>>>>>>>>>>>>>>>> OK, let me summarize what we have
>>>>>>>> discussed up to
>>>>>>> this
>>>>>>>>      >>>>>> point.
>>>>>>>>      >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed
>>>>>>>> that
>>>>>>>>      branch API
>>>>>>>>      >>>>>>> needs
>>>>>>>>      >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>      >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>      >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...)....
>>>>>>>> //onTopOf
>>>>>>>>      returns
>>>>>>>>      >>>>>> its
>>>>>>>>      >>>>>>>>> argument
>>>>>>>>      >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2)
>>>>>>>> The code
>>>>>>> won't
>>>>>>>>      >>>>> make
>>>>>>>>      >>>>>>>> sense
>>>>>>>>      >>>>>>>>>>>>> until
>>>>>>>>      >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> CONS: The need to create a
>>>>>>>> KafkaStreamsBrancher
>>>>>>>>      instance
>>>>>>>>      >>>>>>>>> contrasts the
>>>>>>>>      >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> stream
>>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>      >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or
>>>>>>>> noDefault(). Both
>>>>>>>>      >>>>>>>>> defaultBranch(..)
>>>>>>>>      >>>>>>>>>>>>> and
>>>>>>>>      >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams
>>>>>>>> interface
>>>>>>> is
>>>>>>>>      >>>>>> defined.
>>>>>>>>      >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>>>>      >>>>>>>> (defaultBranch(ks->)
>>>>>>>>      >>>>>>>>> and
>>>>>>>>      >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very
>>>>>>>> easy to
>>>>>>>>      miss the
>>>>>>>>      >>>>>> fact
>>>>>>>>      >>>>>>>>> that one
>>>>>>>>      >>>>>>>>>>>>>>>>> of the terminal methods should be called.
>>>>>>>> If these
>>>>>>>>      methods
>>>>>>>>      >>>>>> are
>>>>>>>>      >>>>>>>> not
>>>>>>>>      >>>>>>>>>>>>>>>>> called, we can throw an exception in
>>>>>>>> runtime.
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can
>>>>>>>> we do
>>>>>>> better?
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> Ivan
>>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>      >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>      >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>> I see your point when you are talking
>>>>>>>> about
>>>>>>>>      >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>>>>>>>>      implemented the
>>>>>>>>      >>>>>>> easy
>>>>>>>>      >>>>>>>>> way.
>>>>>>>>      >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
>>>>>>> assumes
>>>>>>>>      >>>>> nothing
>>>>>>>>      >>>>>>>> will
>>>>>>>>      >>>>>>>>>>>>> reach
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>      >>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>> occurs.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be
>>>>>>>> the only
>>>>>>> option
>>>>>>>>      >>>>>> besides
>>>>>>>>      >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios
>>>>>>>> when we
>>>>>>>>      want to
>>>>>>>>      >>>>>> just
>>>>>>>>      >>>>>>>>> silently
>>>>>>>>      >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>>>      predicate. 2)
>>>>>>>>      >>>>>>> Throwing
>>>>>>>>      >>>>>>>>> an
>>>>>>>>      >>>>>>>>>>>>>>>>>>> exception in the middle of data flow
>>>>>>>> processing
>>>>>>>>      looks
>>>>>>>>      >>>>>> like a
>>>>>>>>      >>>>>>>> bad
>>>>>>>>      >>>>>>>>>>>>> idea.
>>>>>>>>      >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would
>>>>>>>> prefer to
>>>>>>>>      emit a
>>>>>>>>      >>>>>>>> special
>>>>>>>>      >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is
>>>>>>>> exactly
>>>>>>> where
>>>>>>>>      >>>>>>> `default`
>>>>>>>>      >>>>>>>>> can
>>>>>>>>      >>>>>>>>>>>>> be
>>>>>>>>      >>>>>>>>>>>>>>>>>>> used.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>      >>>>> InternalTopologyBuilder
>>>>>>>>      >>>>>>> to
>>>>>>>>      >>>>>>>>> track
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>      >>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>> and raise
>>>>>>>>      a clear
>>>>>>>>      >>>>>>> error
>>>>>>>>      >>>>>>>>>>>>> before it
>>>>>>>>      >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the
>>>>>>>> program is
>>>>>>>>      >>>>> compiled
>>>>>>>>      >>>>>>> and
>>>>>>>>      >>>>>>>>> run?
>>>>>>>>      >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>>>>>>>>      compile if
>>>>>>>>      >>>>> used
>>>>>>>>      >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>>>>>>>>      method chain
>>>>>>>>      >>>>>>>> starting
>>>>>>>>      >>>>>>>>>>>>> from
>>>>>>>>      >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost
>>>>>>>> difference
>>>>>>>>      between
>>>>>>>>      >>>>>>>> runtime
>>>>>>>>      >>>>>>>>> and
>>>>>>>>      >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure
>>>>>>>> uncovers
>>>>>>>>      >>>>> instantly
>>>>>>>>      >>>>>> on
>>>>>>>>      >>>>>>>>> unit
>>>>>>>>      >>>>>>>>>>>>>>>>>>> tests, it costs more for the project
>>>>>>>> than a
>>>>>>>>      compilation
>>>>>>>>      >>>>>>>> failure.
>>>>>>>>      >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> Good point about the terminal
>>>>>>>> operation being
>>>>>>>>      required.
>>>>>>>>      >>>>>>> But
>>>>>>>>      >>>>>>>> is
>>>>>>>>      >>>>>>>>>>>>> that
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> really
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't
>>>>>>>> want a
>>>>>>>>      >>>>>> defaultBranch
>>>>>>>>      >>>>>>>>> they
>>>>>>>>      >>>>>>>>>>>>> can
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> call
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> some other terminal method
>>>>>>>> (noDefaultBranch()?)
>>>>>>>>      just as
>>>>>>>>      >>>>>>>>> easily.  In
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a
>>>>>>>> nicer API
>>>>>>> - a
>>>>>>>>      >>>>> user
>>>>>>>>      >>>>>>>> could
>>>>>>>>      >>>>>>>>>>>>> specify
>>>>>>>>      >>>>>>>>>>>>>>>>> a
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing
>>>>>>>> will reach
>>>>>>> the
>>>>>>>>      >>>>>> default
>>>>>>>>      >>>>>>>>> branch,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case
>>>>>>>> occurs.
>>>>>>> That
>>>>>>>>      >>>>> seems
>>>>>>>>      >>>>>>> like
>>>>>>>>      >>>>>>>>> an
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> improvement over the current branch()
>>>>>>>> API,
>>>>>>>>      which allows
>>>>>>>>      >>>>>> for
>>>>>>>>      >>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>> more
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
>>>>>>> dropped.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> The need for a terminal operation
>>>>>>>> certainly has
>>>>>>>>      to be
>>>>>>>>      >>>>>> well
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>>      >>>>> InternalTopologyBuilder
>>>>>>>>      >>>>>>> to
>>>>>>>>      >>>>>>>>> track
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated
>>>>>>>> and raise
>>>>>>>>      a clear
>>>>>>>>      >>>>>>> error
>>>>>>>>      >>>>>>>>>>>>> before it
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that
>>>>>>>> there is
>>>>>>> a
>>>>>>>>      >>>>> "build
>>>>>>>>      >>>>>>>> step"
>>>>>>>>      >>>>>>>>>>>>> where
>>>>>>>>      >>>>>>>>>>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>>>      >>>>>> StreamsBuilder.build()
>>>>>>>>      >>>>>>> is
>>>>>>>>      >>>>>>>>>>>>> called.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its
>>>>>>>> argument, I
>>>>>>> agree
>>>>>>>>      >>>>> that
>>>>>>>>      >>>>>>> it's
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> allow users to do other operations on
>>>>>>>> the input
>>>>>>>>      stream.
>>>>>>>>      >>>>>>> With
>>>>>>>>      >>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>>>>>> fluent
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same
>>>>>>>> way all
>>>>>>> other
>>>>>>>>      >>>>>>> operations
>>>>>>>>      >>>>>>>>> do -
>>>>>>>>      >>>>>>>>>>>>> if
>>>>>>>>      >>>>>>>>>>>>>>>>> you
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
>>>>>>> multiple
>>>>>>>>      >>>>> times,
>>>>>>>>      >>>>>>> you
>>>>>>>>      >>>>>>>>> just
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call
>>>>>>>> as many
>>>>>>>>      operations
>>>>>>>>      >>>>>> on
>>>>>>>>      >>>>>>> it
>>>>>>>>      >>>>>>>>> as
>>>>>>>>      >>>>>>>>>>>>> you
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan
>>>>>>>> Ponomarev <
>>>>>>>>      >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we
>>>>>>>> do not
>>>>>>>>      always need
>>>>>>>>      >>>>>> the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>>>> operation we
>>>>>>>>      >>>>> don't
>>>>>>>>      >>>>>>>> know
>>>>>>>>      >>>>>>>>>>>>> when to
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its
>>>>>>>> argument,
>>>>>>>>      so we
>>>>>>>>      >>>>> can
>>>>>>>>      >>>>>> do
>>>>>>>>      >>>>>>>>>>>>> something
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> more with the original branch after
>>>>>>>> branching.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
>>>>>>> special
>>>>>>>>      >>>>> object
>>>>>>>>      >>>>>>>>>>>>> construction
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream
>>>>>>>> methods.
>>>>>>> But
>>>>>>>>      >>>>> here
>>>>>>>>      >>>>>> we
>>>>>>>>      >>>>>>>>> have a
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to
>>>>>>>> split the
>>>>>>>>      flow,
>>>>>>>>      >>>>> so
>>>>>>>>      >>>>>> I
>>>>>>>>      >>>>>>>>> think
>>>>>>>>      >>>>>>>>>>>>> this
>>>>>>>>      >>>>>>>>>>>>>>>>> is
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve
>>>>>>>> this
>>>>>>>>      API, but I
>>>>>>>>      >>>>>> find
>>>>>>>>      >>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>>>>>>>>      contrasts the
>>>>>>>>      >>>>>>> fluency
>>>>>>>>      >>>>>>>>> of
>>>>>>>>      >>>>>>>>>>>>> other
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd
>>>>>>>> like to
>>>>>>>>      just call
>>>>>>>>      >>>>> a
>>>>>>>>      >>>>>>>>> method on
>>>>>>>>      >>>>>>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if
>>>>>>>> the branch
>>>>>>>>      cases
>>>>>>>>      >>>>> are
>>>>>>>>      >>>>>>>>> defined
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate,
>>>>>>>> handleCase)
>>>>>>>>      is very
>>>>>>>>      >>>>>> nice
>>>>>>>>      >>>>>>>>> and the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped
>>>>>>>> around
>>>>>>>>      how we
>>>>>>>>      >>>>>>> specify
>>>>>>>>      >>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>> source
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>>>> this::handle1)
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>>>> this::handle2)
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>          
>>>>>>>> .defaultBranch(this::handleDefault);
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a
>>>>>>>> KBranchedStreams or
>>>>>>>>      >>>>>>>> KStreamBrancher
>>>>>>>>      >>>>>>>>> or
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>>>      terminated by
>>>>>>>>      >>>>>>>>>>>>> defaultBranch()
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>>>      incompatible with
>>>>>>>>      >>>>> the
>>>>>>>>      >>>>>>>>> current
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to
>>>>>>>> have a
>>>>>>>>      different
>>>>>>>>      >>>>>> name,
>>>>>>>>      >>>>>>>> but
>>>>>>>>      >>>>>>>>> that
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we
>>>>>>>> could call it
>>>>>>>>      >>>>>> something
>>>>>>>>      >>>>>>>> like
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the
>>>>>>>> old API.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of
>>>>>>>> your
>>>>>>>>      KIP?  It
>>>>>>>>      >>>>>> seems
>>>>>>>>      >>>>>>>>> like it
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line
>>>>>>>> branching
>>>>>>>>      while also
>>>>>>>>      >>>>>>>> allowing
>>>>>>>>      >>>>>>>>> you
>>>>>>>>      >>>>>>>>>>>>> to
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>>>      KBranchedStreams
>>>>>>>>      >>>>>> if
>>>>>>>>      >>>>>>>>> desired.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan
>>>>>>>> Ponomarev
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> void
>>>>>>>> handleFirstCase(KStream<String, String>
>>>>>>>>      ks){
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>          
>>>>>>>> ks.filter(....).mapValues(...)
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>>>>>>>>      String> ks){
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>          
>>>>>>>> ks.selectKey(...).groupByKey()...
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String,
>>>>>>>> String>()
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>>      this::handleFirstCase)
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>>      this::handleSecondCase)
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>>> KafkaStreamsBrancher
>>>>>>>>      >>>>> takes a
>>>>>>>>      >>>>>>>>> Consumer
>>>>>>>>      >>>>>>>>>>>>> as a
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing,
>>>>>>>> and the
>>>>>>>>      example in
>>>>>>>>      >>>>>> the
>>>>>>>>      >>>>>>>> KIP
>>>>>>>>      >>>>>>>>>>>>> shows
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a
>>>>>>>> terminal node
>>>>>>>>      >>>>>>>>> (KafkaStreams#to()
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but
>>>>>>>> how would
>>>>>>> we
>>>>>>>>      >>>>> handle
>>>>>>>>      >>>>>>> the
>>>>>>>>      >>>>>>>>> case
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but
>>>>>>>> wants to
>>>>>>> continue
>>>>>>>>      >>>>>>>> processing
>>>>>>>>      >>>>>>>>> and
>>>>>>>>      >>>>>>>>>>>>> not
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on
>>>>>>>> the
>>>>>>> branched
>>>>>>>>      >>>>>> stream
>>>>>>>>      >>>>>>>>>>>>> immediately?
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic
>>>>>>>> as is if
>>>>>>>>      we had
>>>>>>>>      >>>>>>>> something
>>>>>>>>      >>>>>>>>> like
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM
>>>>>>>> Bill Bejeck
>>>>>>> <
>>>>>>>>      >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the
>>>>>>>> discussion for
>>>>>>> KIP-
>>>>>>>>      >>>>> 418.
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
>>>>>>> KIP-418.
>>>>>>>>      >>>>> Please
>>>>>>>>      >>>>>>>> take
>>>>>>>>      >>>>>>>>> a
>>>>>>>>      >>>>>>>>>>>>> look
>>>>>>>>      >>>>>>>>>>>>>>>>> at
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would
>>>>>>>> appreciate any
>>>>>>>>      feedback :)
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>      >>>>>
>>>>>>>>
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>      >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>>      >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>      >>>>>>>>>
>>>>>>>>      >
>>>>>>>>
>>>>>>>
>>>>
>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi everyone!

Let me revive the discussion of this KIP.

I'm very sorry for stopping my participation in the discussion in June 
2019. My project work was very intensive then and it didn't leave me 
spare time. But I think I must finish this, because we invested 
substantial effort into this discussion and I'm not feel entitled to 
propose other things before this one is finalized.

During these months I proceeded with writing and reviewing Kafka 
Streams-related code. Every time I needed branching, Spring-Kafka's 
KafkaStreamBrancher class of my invention (the original idea for this 
KIP) worked for me -- that's another reason why I gave up pushing the 
KIP forward. When I was coming across the problem with the scope of 
branches, I worked around it this way:

AtomicReference<KStream<...>> result = new AtomicReference<>();
new KafkaStreamBrancher<....>()
     .branch(....)
     .defaultBranch(result::set)
     .onTopOf(someStream);
result.get()...


And yes, of course I don't feel very happy with this approach.

I think that Matthias came up with a bright solution in his post from 
May, 24th 2019. Let me quote it:

KStream#split() -> KBranchedStream
// branch is not easily accessible in current scope
KBranchedStream#branch(Predicate, Consumer<KStream>)
   -> KBranchedStream
// assign a name to the branch and
// return the sub-stream to the current scope later
//
// can be simple as `#branch(p, s->s, "name")`
// or also complex as `#branch(p, s->s.filter(...), "name")`
KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
   -> KBranchedStream
// default branch is not easily accessible
// return map of all named sub-stream into current scope
KBranchedStream#default(Cosumer<KStream>)
   -> Map<String,KStream>
// assign custom name to default-branch
// return map of all named sub-stream into current scope
KBranchedStream#default(Function<KStream,KStream>, String)
   -> Map<String,KStream>
// assign a default name for default
// return map of all named sub-stream into current scope
KBranchedStream#defaultBranch(Function<KStream,KStream>)
   -> Map<String,KStream>
// return map of all names sub-stream into current scope
KBranchedStream#noDefaultBranch()
   -> Map<String,KStream>

I believe this would satisfy everyone. Optional names seems to be a good 
idea: when you don't need to have the branches in the same scope, you 
just don't use names and you don't risk making your code brittle. Or, 
you might want to add names just for debugging purposes. Or, finally, 
you might use the returned Map to have the named branches in the 
original scope.

There also was an input from John Roesler on June 4th, 2019, who 
suggested using Named class. I can't comment on this. The idea seems 
reasonable, but in this matter I'd rather trust people who are more 
familiar with Streams API design principles than me.

Regards,

Ivan



08.10.2019 1:38, Matthias J. Sax пишет:
> I am moving this KIP into "inactive status". Feel free to resume the KIP
> at any point.
> 
> If anybody else is interested in picking up this KIP, feel free to do so.
> 
> 
> 
> -Matthias
> 
> On 7/11/19 4:00 PM, Matthias J. Sax wrote:
>> Ivan,
>>
>> did you see my last reply? What do you think about my proposal to mix
>> both approaches and try to get best-of-both worlds?
>>
>>
>> -Matthias
>>
>> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>>> Thanks for the input John!
>>>
>>>> under your suggestion, it seems that the name is required
>>>
>>> If you want to get the `KStream` as part of the `Map` back using a
>>> `Function`, yes. If you follow the "embedded chaining" pattern using a
>>> `Consumer`, no.
>>>
>>> Allowing for a default name via `split()` can of course be done.
>>> Similarly, using `Named` instead of `String` is possible.
>>>
>>> I wanted to sketch out a high level proposal to merge both patterns
>>> only. Your suggestions to align the new API with the existing API make
>>> totally sense.
>>>
>>>
>>>
>>> One follow up question: Would `Named` be optional or required in
>>> `split()` and `branch()`? It's unclear from your example.
>>>
>>> If both are mandatory, what do we gain by it? The returned `Map` only
>>> contains the corresponding branches, so why should we prefix all of
>>> them? If only `Named` is mandatory in `branch()`, but optional in
>>> `split()`, the same question raises?
>>>
>>> Requiring `Named` in `split()` seems only to make sense, if `Named` is
>>> optional in `branch()` and we generate `-X` suffix using a counter for
>>> different branch name. However, this might lead to the problem of
>>> changing names if branches are added/removed. Also, how would the names
>>> be generated if `Consumer` is mixed in (ie, not all branches are
>>> returned in the `Map`).
>>>
>>> If `Named` is optional for both, it could happen that a user misses to
>>> specify a name for a branch what would lead to runtime issues.
>>>
>>>
>>> Hence, I am actually in favor to not allow a default name but keep
>>> `split()` without parameter and make `Named` in `branch()` required if a
>>> `Function` is used. This makes it explicit to the user that specifying a
>>> name is required if a `Function` is used.
>>>
>>>
>>>
>>> About
>>>
>>>> KBranchedStream#branch(BranchConfig)
>>>
>>> I don't think that the branching predicate is a configuration and hence
>>> would not include it in a configuration object.
>>>
>>>>      withChain(...);
>>>
>>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>>> seem to be a configuration. We can also not prevent a user to call
>>> `withName()` in combination of `withChain()` what does not make sense
>>> IMHO. We could of course throw an RTE but not have a compile time check
>>> seems less appealing. Also, it could happen that neither `withChain()`
>>> not `withName()` is called and the branch is missing in the returned
>>> `Map` what lead to runtime issues, too.
>>>
>>> Hence, I don't think that we should add `BranchConfig`. A config object
>>> is helpful if each configuration can be set independently of all others,
>>> but this seems not to be the case here. If we add new configuration
>>> later, we can also just move forward by deprecating the methods that
>>> accept `Named` and add new methods that accepted `BranchConfig` (that
>>> would of course implement `Named`).
>>>
>>>
>>> Thoughts?
>>>
>>>
>>> @Ivan, what do you think about the general idea to blend the two main
>>> approaches of returning a `Map` plus an "embedded chaining"?
>>>
>>>
>>>
>>> -Matthias
>>>
>>>
>>>
>>> On 6/4/19 10:33 AM, John Roesler wrote:
>>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>>> everyone. Returning the map from the terminal operations also solves
>>>> the problem of merging/joining the branched streams, if we want to add
>>>> support for the compliment later on.
>>>>
>>>> Under your suggestion, it seems that the name is required. Otherwise,
>>>> we wouldn't have keys for the map to return. I this this is actually
>>>> not too bad, since experience has taught us that, although names for
>>>> operations are not required to define stream processing logic, it does
>>>> significantly improve the operational experience when you can map the
>>>> topology, logs, metrics, etc. back to the source code. Since you
>>>> wouldn't (have to) reference the name to chain extra processing onto
>>>> the branch (thanks to the second argument), you can avoid the
>>>> "unchecked name" problem that Ivan pointed out.
>>>>
>>>> In the current implementation of Branch, you can name the branch
>>>> operator itself, and then all the branches get index-suffixed names
>>>> built from the branch operator name. I guess under this proposal, we
>>>> could naturally append the branch name to the branching operator name,
>>>> like this:
>>>>
>>>>     stream.split(Named.withName("mysplit")) //creates node "mysplit"
>>>>                .branch(..., ..., "abranch") // creates node "mysplit-abranch"
>>>>                .defaultBranch(...) // creates node "mysplit-default"
>>>>
>>>> It does make me wonder about the DSL syntax itself, though.
>>>>
>>>> We don't have a defined grammar, so there's plenty of room to debate
>>>> the "best" syntax in the context of each operation, but in general,
>>>> the KStream DSL operators follow this pattern:
>>>>
>>>>      operator(function, config_object?) OR operator(config_object)
>>>>
>>>> where config_object is often just Named in the "function" variant.
>>>> Even when the config_object isn't a Named, but some other config
>>>> class, that config class _always_ implements NamedOperation.
>>>>
>>>> Here, we're introducing a totally different pattern:
>>>>
>>>>    operator(function, function, string)
>>>>
>>>> where the string is the name.
>>>> My first question is whether the name should instead be specified with
>>>> the NamedOperation interface.
>>>>
>>>> My second question is whether we should just roll all these arguments
>>>> up into a config object like:
>>>>
>>>>     KBranchedStream#branch(BranchConfig)
>>>>
>>>>     interface BranchConfig extends NamedOperation {
>>>>      withPredicate(...);
>>>>      withChain(...);
>>>>      withName(...);
>>>>    }
>>>>
>>>> Although I guess we'd like to call BranchConfig something more like
>>>> "Branched", even if I don't particularly like that pattern.
>>>>
>>>> This makes the source code a little noisier, but it also makes us more
>>>> future-proof, as we can deal with a wide range of alternatives purely
>>>> in the config interface, and never have to deal with adding overloads
>>>> to the KBranchedStream if/when we decide we want the name to be
>>>> optional, or the KStream->KStream to be optional.
>>>>
>>>> WDYT?
>>>>
>>>> Thanks,
>>>> -John
>>>>
>>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>>> <mi...@confluent.io> wrote:
>>>>>
>>>>> Matthias: I think that's pretty reasonable from my point of view. Good
>>>>> suggestion.
>>>>>
>>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax <ma...@confluent.io>
>>>>> wrote:
>>>>>
>>>>>> Interesting discussion.
>>>>>>
>>>>>> I am wondering, if we cannot unify the advantage of both approaches:
>>>>>>
>>>>>>
>>>>>>
>>>>>> KStream#split() -> KBranchedStream
>>>>>>
>>>>>> // branch is not easily accessible in current scope
>>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>>    -> KBranchedStream
>>>>>>
>>>>>> // assign a name to the branch and
>>>>>> // return the sub-stream to the current scope later
>>>>>> //
>>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>>    -> KBranchedStream
>>>>>>
>>>>>> // default branch is not easily accessible
>>>>>> // return map of all named sub-stream into current scope
>>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>>    -> Map<String,KStream>
>>>>>>
>>>>>> // assign custom name to default-branch
>>>>>> // return map of all named sub-stream into current scope
>>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>>    -> Map<String,KStream>
>>>>>>
>>>>>> // assign a default name for default
>>>>>> // return map of all named sub-stream into current scope
>>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>>    -> Map<String,KStream>
>>>>>>
>>>>>> // return map of all names sub-stream into current scope
>>>>>> KBranchedStream#noDefaultBranch()
>>>>>>    -> Map<String,KStream>
>>>>>>
>>>>>>
>>>>>>
>>>>>> Hence, for each sub-stream, the user can pick to add a name and return
>>>>>> the branch "result" to the calling scope or not. The implementation can
>>>>>> also check at runtime that all returned names are unique. The returned
>>>>>> Map can be empty and it's also optional to use the Map.
>>>>>>
>>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>>> Ivan,
>>>>>>>
>>>>>>> That's a very good point about the "start" operator in the dynamic case.
>>>>>>> I had no problem with "split()"; I was just questioning the necessity.
>>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>>> "split()" start operator. Thanks!
>>>>>>>
>>>>>>> Separately, I'm interested to see where the present discussion leads.
>>>>>>> I've written enough Javascript code in my life to be suspicious of
>>>>>>> nested closures. You have a good point about using method references (or
>>>>>>> indeed function literals also work). It should be validating that this
>>>>>>> was also the JS community's first approach to flattening the logic when
>>>>>>> their nested closure situation got out of hand. Unfortunately, it's
>>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>>> readability (but in different ways for different reasons). In other
>>>>>>> words, I agree that function references is *the* first-order solution if
>>>>>>> the nested code does indeed become a problem.
>>>>>>>
>>>>>>> However, the history of JS also tells us that function references aren't
>>>>>>> the end of the story either, and you can see that by observing that
>>>>>>> there have been two follow-on eras, as they continue trying to cope with
>>>>>>> the consequences of living in such a callback-heavy language. First, you
>>>>>>> have Futures/Promises, which essentially let you convert nested code to
>>>>>>> method-chained code (Observables/FP is a popular variation on this).
>>>>>>> Most lately, you have async/await, which is an effort to apply language
>>>>>>> (not just API) syntax to the problem, and offer the "flattest" possible
>>>>>>> programming style to solve the problem (because you get back to just one
>>>>>>> code block per functional unit).
>>>>>>>
>>>>>>> Stream-processing is a different domain, and Java+KStreams is nowhere
>>>>>>> near as callback heavy as JS, so I don't think we have to take the JS
>>>>>>> story for granted, but then again, I think we can derive some valuable
>>>>>>> lessons by looking sideways to adjacent domains. I'm just bringing this
>>>>>>> up to inspire further/deeper discussion. At the same time, just like JS,
>>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>>
>>>>>>> Separately again, I'm interested in the post-branch merge (and I'd also
>>>>>>> add join) problem that Paul brought up. We can clearly punt on it, by
>>>>>>> terminating the nested branches with sink operators. But is there a DSL
>>>>>>> way to do it?
>>>>>>>
>>>>>>> Thanks again for your driving this,
>>>>>>> -John
>>>>>>>
>>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>>> <ma...@gmail.com>> wrote:
>>>>>>>
>>>>>>>      Ivan, I’ll definitely forfeit my point on the clumsiness of the
>>>>>>>      branch(predicate, consumer) solution, I don’t see any real drawbacks
>>>>>>>      for the dynamic case.
>>>>>>>
>>>>>>>      IMO the one trade off to consider at this point is the scope
>>>>>>>      question. I don’t know if I totally agree that “we rarely need them
>>>>>>>      in the same scope” since merging the branches back together later
>>>>>>>      seems like a perfectly plausible use case that can be a lot nicer
>>>>>>>      when the branched streams are in the same scope. That being said,
>>>>>>>      for the reasons Ivan listed, I think it is overall the better
>>>>>>>      solution - working around the scope thing is easy enough if you need
>>>>>>>      to.
>>>>>>>
>>>>>>>      > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>>      <ip...@mail.ru.invalid> wrote:
>>>>>>>      >
>>>>>>>      > Hello everyone, thank you all for joining the discussion!
>>>>>>>      >
>>>>>>>      > Well, I don't think the idea of named branches, be it a
>>>>>>>      LinkedHashMap (no other Map will do, because order of definition
>>>>>>>      matters) or `branch` method  taking name and Consumer has more
>>>>>>>      advantages than drawbacks.
>>>>>>>      >
>>>>>>>      > In my opinion, the only real positive outcome from Michael's
>>>>>>>      proposal is that all the returned branches are in the same scope.
>>>>>>>      But 1) we rarely need them in the same scope 2) there is a
>>>>>>>      workaround for the scope problem, described in the KIP.
>>>>>>>      >
>>>>>>>      > 'Inlining the complex logic' is not a problem, because we can use
>>>>>>>      method references instead of lambdas. In real world scenarios you
>>>>>>>      tend to split the complex logic to methods anyway, so the code is
>>>>>>>      going to be clean.
>>>>>>>      >
>>>>>>>      > The drawbacks are strong. The cohesion between predicates and
>>>>>>>      handlers is lost. We have to define predicates in one place, and
>>>>>>>      handlers in another. This opens the door for bugs:
>>>>>>>      >
>>>>>>>      > - what if we forget to define a handler for a name? or a name for
>>>>>>>      a handler?
>>>>>>>      > - what if we misspell a name?
>>>>>>>      > - what if we copy-paste and duplicate a name?
>>>>>>>      >
>>>>>>>      > What Michael propose would have been totally OK if we had been
>>>>>>>      writing the API in Lua, Ruby or Python. In those languages the
>>>>>>>      "dynamic naming" approach would have looked most concise and
>>>>>>>      beautiful. But in Java we expect all the problems related to
>>>>>>>      identifiers to be eliminated in compile time.
>>>>>>>      >
>>>>>>>      > Do we have to invent duck-typing for the Java API?
>>>>>>>      >
>>>>>>>      > And if we do, what advantage are we supposed to get besides having
>>>>>>>      all the branches in the same scope? Michael, maybe I'm missing your
>>>>>>>      point?
>>>>>>>      >
>>>>>>>      > ---
>>>>>>>      >
>>>>>>>      > Earlier in this discussion John Roesler also proposed to do
>>>>>>>      without "start branching" operator, and later Paul mentioned that in
>>>>>>>      the case when we have to add a dynamic number of branches, the
>>>>>>>      current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
>>>>>>>      me address both comments here.
>>>>>>>      >
>>>>>>>      > 1) "Start branching" operator (I think that *split* is a good name
>>>>>>>      for it indeed) is critical when we need to do a dynamic branching,
>>>>>>>      see example below.
>>>>>>>      >
>>>>>>>      > 2) No, dynamic branching in current KIP is not clumsy at all.
>>>>>>>      Imagine a real-world scenario when you need one branch per enum
>>>>>>>      value (say, RecordType). You can have something like this:
>>>>>>>      >
>>>>>>>      > /*John:if we had to start with stream.branch(...) here, it would
>>>>>>>      have been much messier.*/
>>>>>>>      > KBranchedStream branched = stream.split();
>>>>>>>      >
>>>>>>>      > /*Not clumsy at all :-)*/
>>>>>>>      > for (RecordType recordType : RecordType.values())
>>>>>>>      >             branched = branched.branch((k, v) -> v.getRecType() ==
>>>>>>>      recordType,
>>>>>>>      >                     recordType::processRecords);
>>>>>>>      >
>>>>>>>      > Regards,
>>>>>>>      >
>>>>>>>      > Ivan
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>>      >> I also agree with Michael's observation about the core problem of
>>>>>>>      >> current `branch()` implementation.
>>>>>>>      >>
>>>>>>>      >> However, I also don't like to pass in a clumsy Map object. My
>>>>>>>      thinking
>>>>>>>      >> was more aligned with Paul's proposal to just add a name to each
>>>>>>>      >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>>      >>
>>>>>>>      >> It makes the code easier to read, and also make the order of
>>>>>>>      >> `Predicates` (that is essential) easier to grasp.
>>>>>>>      >>
>>>>>>>      >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>      >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>      >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>      >>>>>>    .defaultBranch("defaultBranch");
>>>>>>>      >> An open question is the case for which no defaultBranch() should
>>>>>> be
>>>>>>>      >> specified. Atm, `split()` and `branch()` would return
>>>>>>>      `BranchedKStream`
>>>>>>>      >> and the call to `defaultBranch()` that returns the `Map` is
>>>>>> mandatory
>>>>>>>      >> (what is not the case atm). Or is this actually not a real
>>>>>> problem,
>>>>>>>      >> because users can just ignore the branch returned by
>>>>>>>      `defaultBranch()`
>>>>>>>      >> in the result `Map` ?
>>>>>>>      >>
>>>>>>>      >>
>>>>>>>      >> About "inlining": So far, it seems to be a matter of personal
>>>>>>>      >> preference. I can see arguments for both, but no "killer
>>>>>>>      argument" yet
>>>>>>>      >> that clearly make the case for one or the other.
>>>>>>>      >>
>>>>>>>      >>
>>>>>>>      >> -Matthias
>>>>>>>      >>
>>>>>>>      >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>>      >>> Perhaps inlining is the wrong terminology. It doesn’t require
>>>>>>>      that a lambda with the full downstream topology be defined inline -
>>>>>>>      it can be a method reference as with Ivan’s original suggestion.
>>>>>>>      The advantage of putting the predicate and its downstream logic
>>>>>>>      (Consumer) together in branch() is that they are required to be near
>>>>>>>      to each other.
>>>>>>>      >>>
>>>>>>>      >>> Ultimately the downstream code has to live somewhere, and deep
>>>>>>>      branch trees will be hard to read regardless.
>>>>>>>      >>>
>>>>>>>      >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>>      <michael.drogalis@confluent.io
>>>>>>>      <ma...@confluent.io>> wrote:
>>>>>>>      >>>>
>>>>>>>      >>>> I'm less enthusiastic about inlining the branch logic with its
>>>>>>>      downstream
>>>>>>>      >>>> functionality. Programs that have deep branch trees will
>>>>>>>      quickly become
>>>>>>>      >>>> harder to read as a single unit.
>>>>>>>      >>>>
>>>>>>>      >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>>      <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>>      >>>>>
>>>>>>>      >>>>> Also +1 on the issues/goals as Michael outlined them, I think
>>>>>>>      that sets a
>>>>>>>      >>>>> great framework for the discussion.
>>>>>>>      >>>>>
>>>>>>>      >>>>> Regarding the SortedMap solution, my understanding is that the
>>>>>>>      current
>>>>>>>      >>>>> proposal in the KIP is what is in my PR which (pending naming
>>>>>>>      decisions) is
>>>>>>>      >>>>> roughly this:
>>>>>>>      >>>>>
>>>>>>>      >>>>> stream.split()
>>>>>>>      >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>      >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>>      >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>>      >>>>>
>>>>>>>      >>>>> Obviously some ordering is necessary, since branching as a
>>>>>>>      construct
>>>>>>>      >>>>> doesn't work without it, but this solution seems like it
>>>>>>>      provides as much
>>>>>>>      >>>>> associativity as the SortedMap solution, because each branch()
>>>>>>>      call
>>>>>>>      >>>>> directly associates the "conditional" with the "code block."
>>>>>>>      The value it
>>>>>>>      >>>>> provides over the KIP solution is the accessing of streams in
>>>>>>>      the same
>>>>>>>      >>>>> scope.
>>>>>>>      >>>>>
>>>>>>>      >>>>> The KIP solution is less "dynamic" than the SortedMap solution
>>>>>>>      in the sense
>>>>>>>      >>>>> that it is slightly clumsier to add a dynamic number of
>>>>>>>      branches, but it is
>>>>>>>      >>>>> certainly possible.  It seems to me like the API should favor
>>>>>>>      the "static"
>>>>>>>      >>>>> case anyway, and should make it simple and readable to
>>>>>>>      fluently declare and
>>>>>>>      >>>>> access your branches in-line.  It also makes it impossible to
>>>>>>>      ignore a
>>>>>>>      >>>>> branch, and it is possible to build an (almost) identical
>>>>>>>      SortedMap
>>>>>>>      >>>>> solution on top of it.
>>>>>>>      >>>>>
>>>>>>>      >>>>> I could also see a middle ground where instead of a raw
>>>>>>>      SortedMap being
>>>>>>>      >>>>> taken in, branch() takes a name and not a Consumer.  Something
>>>>>>>      like this:
>>>>>>>      >>>>>
>>>>>>>      >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>>      >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>>      >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>>      >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>>>>>      >>>>>
>>>>>>>      >>>>> Pros for that solution:
>>>>>>>      >>>>> - accessing branched KStreams in same scope
>>>>>>>      >>>>> - no double brace initialization, hopefully slightly more
>>>>>>>      readable than
>>>>>>>      >>>>> SortedMap
>>>>>>>      >>>>>
>>>>>>>      >>>>> Cons
>>>>>>>      >>>>> - downstream branch logic cannot be specified inline which
>>>>>>>      makes it harder
>>>>>>>      >>>>> to read top to bottom (like existing API and SortedMap, but
>>>>>>>      unlike the KIP)
>>>>>>>      >>>>> - you can forget to "handle" one of the branched streams (like
>>>>>>>      existing
>>>>>>>      >>>>> API and SortedMap, but unlike the KIP)
>>>>>>>      >>>>>
>>>>>>>      >>>>> (KBranchedStreams could even work *both* ways but perhaps
>>>>>>>      that's overdoing
>>>>>>>      >>>>> it).
>>>>>>>      >>>>>
>>>>>>>      >>>>> Overall I'm curious how important it is to be able to easily
>>>>>>>      access the
>>>>>>>      >>>>> branched KStream in the same scope as the original.  It's
>>>>>>>      possible that it
>>>>>>>      >>>>> doesn't need to be handled directly by the API, but instead
>>>>>>>      left up to the
>>>>>>>      >>>>> user.  I'm sort of in the middle on it.
>>>>>>>      >>>>>
>>>>>>>      >>>>> Paul
>>>>>>>      >>>>>
>>>>>>>      >>>>>
>>>>>>>      >>>>>
>>>>>>>      >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>>      <sophie@confluent.io <ma...@confluent.io>>
>>>>>>>      >>>>> wrote:
>>>>>>>      >>>>>
>>>>>>>      >>>>>> I'd like to +1 what Michael said about the issues with the
>>>>>>>      existing
>>>>>>>      >>>>> branch
>>>>>>>      >>>>>> method, I agree with what he's outlined and I think we should
>>>>>>>      proceed by
>>>>>>>      >>>>>> trying to alleviate these problems. Specifically it seems
>>>>>>>      important to be
>>>>>>>      >>>>>> able to cleanly access the individual branches (eg by mapping
>>>>>>>      >>>>>> name->stream), which I thought was the original intention of
>>>>>>>      this KIP.
>>>>>>>      >>>>>>
>>>>>>>      >>>>>> That said, I don't think we should so easily give in to the
>>>>>>>      double brace
>>>>>>>      >>>>>> anti-pattern or force ours users into it if at all possible to
>>>>>>>      >>>>> avoid...just
>>>>>>>      >>>>>> my two cents.
>>>>>>>      >>>>>>
>>>>>>>      >>>>>> Cheers,
>>>>>>>      >>>>>> Sophie
>>>>>>>      >>>>>>
>>>>>>>      >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>>      >>>>>> michael.drogalis@confluent.io
>>>>>>>      <ma...@confluent.io>> wrote:
>>>>>>>      >>>>>>
>>>>>>>      >>>>>>> I’d like to propose a different way of thinking about this.
>>>>>>>      To me,
>>>>>>>      >>>>> there
>>>>>>>      >>>>>>> are three problems with the existing branch signature:
>>>>>>>      >>>>>>>
>>>>>>>      >>>>>>> 1. If you use it the way most people do, Java raises unsafe
>>>>>> type
>>>>>>>      >>>>>> warnings.
>>>>>>>      >>>>>>> 2. The way in which you use the stream branches is
>>>>>>>      positionally coupled
>>>>>>>      >>>>>> to
>>>>>>>      >>>>>>> the ordering of the conditionals.
>>>>>>>      >>>>>>> 3. It is brittle to extend existing branch calls with
>>>>>>>      additional code
>>>>>>>      >>>>>>> paths.
>>>>>>>      >>>>>>>
>>>>>>>      >>>>>>> Using associative constructs instead of relying on ordered
>>>>>>>      constructs
>>>>>>>      >>>>>> would
>>>>>>>      >>>>>>> be a stronger approach. Consider a signature that instead
>>>>>>>      looks like
>>>>>>>      >>>>>> this:
>>>>>>>      >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
>>>>>>>      Predicate<?
>>>>>>>      >>>>>>> super K,? super V>>);
>>>>>>>      >>>>>>>
>>>>>>>      >>>>>>> Branches are given names in a map, and as a result, the API
>>>>>>>      returns a
>>>>>>>      >>>>>>> mapping of names to streams. The ordering of the
>>>>>> conditionals is
>>>>>>>      >>>>>> maintained
>>>>>>>      >>>>>>> because it’s a sorted map. Insert order determines the order
>>>>>> of
>>>>>>>      >>>>>> evaluation.
>>>>>>>      >>>>>>> This solves problem 1 because there are no more varargs. It
>>>>>>>      solves
>>>>>>>      >>>>>> problem
>>>>>>>      >>>>>>> 2 because you no longer lean on ordering to access the
>>>>>>>      branch you’re
>>>>>>>      >>>>>>> interested in. It solves problem 3 because you can introduce
>>>>>>>      another
>>>>>>>      >>>>>>> conditional by simply attaching another name to the
>>>>>>>      structure, rather
>>>>>>>      >>>>>> than
>>>>>>>      >>>>>>> messing with the existing indices.
>>>>>>>      >>>>>>>
>>>>>>>      >>>>>>> One of the drawbacks is that creating the map inline is
>>>>>>>      historically
>>>>>>>      >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>>      voluminously, but
>>>>>>>      >>>>>>> double brace initialization would clean up the aesthetics.
>>>>>>>      >>>>>>>
>>>>>>>      >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>>      <john@confluent.io <ma...@confluent.io>>
>>>>>>>      >>>>> wrote:
>>>>>>>      >>>>>>>> Hi Ivan,
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>> Thanks for the update.
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>> FWIW, I agree with Matthias that the current "start
>>>>>> branching"
>>>>>>>      >>>>> operator
>>>>>>>      >>>>>>> is
>>>>>>>      >>>>>>>> confusing when named the same way as the actual branches.
>>>>>>>      "Split"
>>>>>>>      >>>>> seems
>>>>>>>      >>>>>>>> like a good name. Alternatively, we can do without a "start
>>>>>>>      >>>>> branching"
>>>>>>>      >>>>>>>> operator at all, and just do:
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>> stream
>>>>>>>      >>>>>>>>      .branch(Predicate)
>>>>>>>      >>>>>>>>      .branch(Predicate)
>>>>>>>      >>>>>>>>      .defaultBranch();
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>> Tentatively, I think that this branching operation should be
>>>>>>>      >>>>> terminal.
>>>>>>>      >>>>>>> That
>>>>>>>      >>>>>>>> way, we don't create ambiguity about how to use it. That
>>>>>>>      is, `branch`
>>>>>>>      >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
>>>>>>>      `void`, to
>>>>>>>      >>>>>>>> enforce that it comes last, and that there is only one
>>>>>>>      definition of
>>>>>>>      >>>>>> the
>>>>>>>      >>>>>>>> default branch. Potentially, we should log a warning if
>>>>>>>      there's no
>>>>>>>      >>>>>>> default,
>>>>>>>      >>>>>>>> and additionally log a warning (or throw an exception) if a
>>>>>>>      record
>>>>>>>      >>>>>> falls
>>>>>>>      >>>>>>>> though with no default.
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>> Thoughts?
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>> Thanks,
>>>>>>>      >>>>>>>> -John
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>>      >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>>      >>>>>>>> wrote:
>>>>>>>      >>>>>>>>
>>>>>>>      >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>> this is to make the name similar to String#split
>>>>>>>      >>>>>>>>>>> that also returns an array, right?
>>>>>>>      >>>>>>>>> The intend was to avoid name duplication. The return type
>>>>>>>      should
>>>>>>>      >>>>>> _not_
>>>>>>>      >>>>>>>>> be an array.
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> The current proposal is
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> stream.branch()
>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>      >>>>>>>>>      .defaultBranch();
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>>      `branch()` does
>>>>>>>      >>>>> not
>>>>>>>      >>>>>>>>> take any parameters and has different semantics than the
>>>>>> later
>>>>>>>      >>>>>>>>> `branch()` calls. Note, that from the code snippet above,
>>>>>> it's
>>>>>>>      >>>>> hidden
>>>>>>>      >>>>>>>>> that the first call is `KStream#branch()` while the others
>>>>>> are
>>>>>>>      >>>>>>>>> `KBranchedStream#branch()` what makes reading the code
>>>>>> harder.
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
>>>>>>>      I though
>>>>>>>      >>>>>> it
>>>>>>>      >>>>>>>>> might be better to also rename `KStream#branch()` to avoid
>>>>>> the
>>>>>>>      >>>>> naming
>>>>>>>      >>>>>>>>> overlap that seems to be confusing. The following reads
>>>>>> much
>>>>>>>      >>>>> cleaner
>>>>>>>      >>>>>> to
>>>>>>>      >>>>>>>> me:
>>>>>>>      >>>>>>>>> stream.split()
>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>      >>>>>>>>>      .branch(Predicate)
>>>>>>>      >>>>>>>>>      .defaultBranch();
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> Maybe there is a better alternative to `split()` though to
>>>>>>>      avoid
>>>>>>>      >>>>> the
>>>>>>>      >>>>>>>>> naming overlap.
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately
>>>>>> we
>>>>>>>      >>>>> cannot
>>>>>>>      >>>>>>> have
>>>>>>>      >>>>>>>>> a method with such name :-)
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
>>>>>>>      with a
>>>>>>>      >>>>> short
>>>>>>>      >>>>>>>> name?
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
>>>>>>>      with all
>>>>>>>      >>>>> it's
>>>>>>>      >>>>>>>>> methods? It will be part of public API and should be
>>>>>>>      contained in
>>>>>>>      >>>>> the
>>>>>>>      >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>>>      >>>>>>>>> `defaultBranch()` is.
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> You did not comment on the idea to add a
>>>>>>>      `KBranchedStream#get(int
>>>>>>>      >>>>>>> index)
>>>>>>>      >>>>>>>>> -> KStream` method to get the individually
>>>>>>>      branched-KStreams. Would
>>>>>>>      >>>>>> be
>>>>>>>      >>>>>>>>> nice to get your feedback about it. It seems you suggest
>>>>>>>      that users
>>>>>>>      >>>>>>>>> would need to write custom utility code otherwise, to
>>>>>>>      access them.
>>>>>>>      >>>>> We
>>>>>>>      >>>>>>>>> should discuss the pros and cons of both approaches. It
>>>>>> feels
>>>>>>>      >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
>>>>>>>      to get
>>>>>>>      >>>>> the
>>>>>>>      >>>>>>>>> branched-KStreams directly.
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>> -Matthias
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>
>>>>>>>      >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>      >>>>>>>>>> Hi all!
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>      >>>>>>>>>> I can see your point: this is to make the name similar to
>>>>>>>      >>>>>>> String#split
>>>>>>>      >>>>>>>>>> that also returns an array, right? But is it worth the
>>>>>>>      loss of
>>>>>>>      >>>>>>>> backwards
>>>>>>>      >>>>>>>>>> compatibility? We can have overloaded branch() as well
>>>>>>>      without
>>>>>>>      >>>>>>>> affecting
>>>>>>>      >>>>>>>>>> the existing code. Maybe the old array-based `branch`
>>>>>> method
>>>>>>>      >>>>> should
>>>>>>>      >>>>>>> be
>>>>>>>      >>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>>      >>>>> BranchingKStream#branch(),
>>>>>>>      >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>> BranchingKStream#default()
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default'
>>>>>> is,
>>>>>>>      >>>>>>> however, a
>>>>>>>      >>>>>>>>>> reserved word, so unfortunately we cannot have a method
>>>>>>>      with such
>>>>>>>      >>>>>>> name
>>>>>>>      >>>>>>>>> :-)
>>>>>>>      >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument,
>>>>>> but I
>>>>>>>      >>>>> think
>>>>>>>      >>>>>>> that
>>>>>>>      >>>>>>>>>> is not required?
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> Absolutely! I think that was just copy-paste error or
>>>>>>>      something.
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> Dear colleagues,
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>>>      >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> Any new suggestions/objections?
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> Ivan
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>>
>>>>>>>      >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>      >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems
>>>>>> that
>>>>>>>      >>>>>>> everybody
>>>>>>>      >>>>>>>>>>> agrees that the current branch() method using arrays is
>>>>>> not
>>>>>>>      >>>>>> optimal.
>>>>>>>      >>>>>>>>>>> I had a quick look into the PR and I like the overall
>>>>>>>      proposal.
>>>>>>>      >>>>>>> There
>>>>>>>      >>>>>>>>>>> are some minor things we need to consider. I would
>>>>>>>      recommend the
>>>>>>>      >>>>>>>>>>> following renaming:
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>      >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>>>      >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>>      BranchingKStream#default()
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>> It's just a suggestion to get slightly shorter method
>>>>>> names.
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>>      `Predicate` as
>>>>>>>      >>>>>>>> argument,
>>>>>>>      >>>>>>>>>>> but I think that is not required?
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>> Also, we should consider KIP-307, that was recently
>>>>>>>      accepted and
>>>>>>>      >>>>>> is
>>>>>>>      >>>>>>>>>>> currently implemented:
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>
>>>>>>>
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>      >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
>>>>>>>      parameter.
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>> For the issue that the created `KStream` object are in
>>>>>>>      different
>>>>>>>      >>>>>>>> scopes:
>>>>>>>      >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>>>>>>>      index)` method
>>>>>>>      >>>>>>> that
>>>>>>>      >>>>>>>>>>> returns the corresponding "branched" result `KStream`
>>>>>>>      object?
>>>>>>>      >>>>>> Maybe,
>>>>>>>      >>>>>>>> the
>>>>>>>      >>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>>>>      >>>>>> `Consumer<KStream>`
>>>>>>>      >>>>>>>> but
>>>>>>>      >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
>>>>>>>      whatever
>>>>>>>      >>>>>> the
>>>>>>>      >>>>>>>>>>> `Function` returns?
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
>>>>>>>      current
>>>>>>>      >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>> -Matthias
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>      >>>>>>>>>>>> Ivan,
>>>>>>>      >>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
>>>>>>>      makes sense
>>>>>>>      >>>>>> for
>>>>>>>      >>>>>>>> you
>>>>>>>      >>>>>>>>> to
>>>>>>>      >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
>>>>>>>      we'll
>>>>>>>      >>>>> need
>>>>>>>      >>>>>>>> some
>>>>>>>      >>>>>>>>>>>> buy-in from committers that have actual binding votes on
>>>>>>>      >>>>> whether
>>>>>>>      >>>>>>> the
>>>>>>>      >>>>>>>>> KIP
>>>>>>>      >>>>>>>>>>>> could be adopted.  It would be great to hear if they
>>>>>>>      think this
>>>>>>>      >>>>>> is
>>>>>>>      >>>>>>> a
>>>>>>>      >>>>>>>>> good
>>>>>>>      >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
>>>>>>>      starting a
>>>>>>>      >>>>>>> vote,
>>>>>>>      >>>>>>>>> or if
>>>>>>>      >>>>>>>>>>>> there is generally some indication of interest
>>>>>> beforehand.
>>>>>>>      >>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
>>>>>>>      assuming
>>>>>>>      >>>>> we
>>>>>>>      >>>>>> do
>>>>>>>      >>>>>>>>> move
>>>>>>>      >>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>>>>      >>>>>> KBranchedStream",
>>>>>>>      >>>>>>> do
>>>>>>>      >>>>>>>>> we
>>>>>>>      >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I
>>>>>> would
>>>>>>>      >>>>> favor
>>>>>>>      >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs
>>>>>> that
>>>>>>>      >>>>>>> accomplish
>>>>>>>      >>>>>>>>> the
>>>>>>>      >>>>>>>>>>>> same thing is confusing, especially when they're fairly
>>>>>>>      similar
>>>>>>>      >>>>>>>>> anyway.  We
>>>>>>>      >>>>>>>>>>>> just need to be sure we're not making something
>>>>>>>      >>>>>>> impossible/difficult
>>>>>>>      >>>>>>>>> that
>>>>>>>      >>>>>>>>>>>> is currently possible/easy.
>>>>>>>      >>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>> Regarding my PR - I think the general structure would
>>>>>> work,
>>>>>>>      >>>>> it's
>>>>>>>      >>>>>>>> just a
>>>>>>>      >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>>>>      >>>>>>> particular,
>>>>>>>      >>>>>>>>>>>> passing in the "predicates" and "children" lists which
>>>>>> get
>>>>>>>      >>>>>> modified
>>>>>>>      >>>>>>>> in
>>>>>>>      >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>>      KStreamLazyBranch is
>>>>>>>      >>>>> a
>>>>>>>      >>>>>>> bit
>>>>>>>      >>>>>>>>>>>> complicated to follow.
>>>>>>>      >>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>> Thanks,
>>>>>>>      >>>>>>>>>>>> Paul
>>>>>>>      >>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>>>>      >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>      >>>>>>>>> wrote:
>>>>>>>      >>>>>>>>>>>>> Hi Paul!
>>>>>>>      >>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>>      convinced: your
>>>>>>>      >>>>>>>> proposal
>>>>>>>      >>>>>>>>>>>>> looks better and should work. We just have to document
>>>>>> the
>>>>>>>      >>>>>> crucial
>>>>>>>      >>>>>>>>> fact
>>>>>>>      >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
>>>>>>>      And then
>>>>>>>      >>>>>> it's
>>>>>>>      >>>>>>>> all
>>>>>>>      >>>>>>>>>>>>> going to be very nice.
>>>>>>>      >>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
>>>>>>>      resume the
>>>>>>>      >>>>>>>>>>>>> discussion here, right?
>>>>>>>      >>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>>>>>>      >>>>> starting
>>>>>>>      >>>>>>>> point
>>>>>>>      >>>>>>>>> if
>>>>>>>      >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
>>>>>>>      starting
>>>>>>>      >>>>>>> point.
>>>>>>>      >>>>>>>>> But
>>>>>>>      >>>>>>>>>>>>> as a novice in this project I might miss some important
>>>>>>>      >>>>> details.
>>>>>>>      >>>>>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>> Ivan
>>>>>>>      >>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>      >>>>>>>>>>>>>> Ivan,
>>>>>>>      >>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>>>>      >>>>> stream.branch()
>>>>>>>      >>>>>>>>> solution
>>>>>>>      >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>>>>>>      >>>>> invoked
>>>>>>>      >>>>>> as
>>>>>>>      >>>>>>>>> they’re
>>>>>>>      >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
>>>>>>>      still
>>>>>>>      >>>>>> ought
>>>>>>>      >>>>>>> to
>>>>>>>      >>>>>>>>> be
>>>>>>>      >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
>>>>>>>      depend on
>>>>>>>      >>>>> the
>>>>>>>      >>>>>>>>> branched
>>>>>>>      >>>>>>>>>>>>> streams having been set.
>>>>>>>      >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
>>>>>>>      access
>>>>>>>      >>>>> the
>>>>>>>      >>>>>>>>> branched
>>>>>>>      >>>>>>>>>>>>> streams in the same scope as the original stream (that
>>>>>>>      is, not
>>>>>>>      >>>>>>>> inside
>>>>>>>      >>>>>>>>> the
>>>>>>>      >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>>>>>>      >>>>> solutions.
>>>>>>>      >>>>>> It
>>>>>>>      >>>>>>>>> can be
>>>>>>>      >>>>>>>>>>>>> worked around though.
>>>>>>>      >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
>>>>>>>      excited
>>>>>>>      >>>>> to
>>>>>>>      >>>>>>>> hear
>>>>>>>      >>>>>>>>>>>>> your thoughts!]
>>>>>>>      >>>>>>>>>>>>>> Paul
>>>>>>>      >>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>>>>      >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>      >>>>>>>>> wrote:
>>>>>>>      >>>>>>>>>>>>>>> Hi Paul!
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>>>      >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at
>>>>>> first
>>>>>>>      >>>>> glance,
>>>>>>>      >>>>>>> but
>>>>>>>      >>>>>>>>> ---
>>>>>>>      >>>>>>>>>>>>>>>> the newly branched streams are not available in the
>>>>>>>      same
>>>>>>>      >>>>>> scope
>>>>>>>      >>>>>>> as
>>>>>>>      >>>>>>>>> each
>>>>>>>      >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
>>>>>> together
>>>>>>>      >>>>> again
>>>>>>>      >>>>>> I
>>>>>>>      >>>>>>>>> don't see
>>>>>>>      >>>>>>>>>>>>> a way to do that.
>>>>>>>      >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
>>>>>>>      just
>>>>>>>      >>>>>> going
>>>>>>>      >>>>>>> to
>>>>>>>      >>>>>>>>>>>>> write in details about this issue.
>>>>>>>      >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
>>>>>>>      we need
>>>>>>>      >>>>> to
>>>>>>>      >>>>>>>>> identify
>>>>>>>      >>>>>>>>>>>>> customers who have bought coffee and made a purchase
>>>>>>>      in the
>>>>>>>      >>>>>>>>> electronics
>>>>>>>      >>>>>>>>>>>>> store to give them coupons.
>>>>>>>      >>>>>>>>>>>>>>> This is the code I usually write under these
>>>>>>>      circumstances
>>>>>>>      >>>>>> using
>>>>>>>      >>>>>>>> my
>>>>>>>      >>>>>>>>>>>>> 'brancher' class:
>>>>>>>      >>>>>>>>>>>>>>> @Setter
>>>>>>>      >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>      >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>      >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>      >>>>>>>>>>>>>>>       return
>>>>>>>      >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>      >>>>>>>>>>>>>>>       /*In the real world the code here can be
>>>>>>>      complex, so
>>>>>>>      >>>>>>>>> creation of
>>>>>>>      >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
>>>>>>>      order to
>>>>>>>      >>>>>>>> separate
>>>>>>>      >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>      >>>>>>>>>>>>>>>  }
>>>>>>>      >>>>>>>>>>>>>>> }
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>      >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>>> couponIssuer::setCoffePurchases)
>>>>>>>      >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>>      >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>      >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
>>>>>>>      everything
>>>>>>>      >>>>>>>> later,
>>>>>>>      >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>      >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>>      initialize the
>>>>>>>      >>>>>>>>> CouponIssuer
>>>>>>>      >>>>>>>>>>>>> we need the terminal operation to be called before
>>>>>>>      >>>>>>>>> streamsBuilder.build()
>>>>>>>      >>>>>>>>>>>>> is called.
>>>>>>>      >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
>>>>>>>      essentially
>>>>>>>      >>>>>> the
>>>>>>>      >>>>>>>>> next
>>>>>>>      >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
>>>>>>>      based on
>>>>>>>      >>>>> my
>>>>>>>      >>>>>>>>> experience,
>>>>>>>      >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>>      >>>>>>>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> Ivan
>>>>>>>      >>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>      >>>>>>>>>>>>>>>> Ivan,
>>>>>>>      >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
>>>>>>>      fluent
>>>>>>>      >>>>> API
>>>>>>>      >>>>>>>> based
>>>>>>>      >>>>>>>>>>>>> off of
>>>>>>>      >>>>>>>>>>>>>>>> KStream here
>>>>>>>      (https://github.com/apache/kafka/pull/6512),
>>>>>>>      >>>>>> and
>>>>>>>      >>>>>>> I
>>>>>>>      >>>>>>>>> think
>>>>>>>      >>>>>>>>>>>>> I
>>>>>>>      >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>      >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>>>>>      >>>>>>> compatibility
>>>>>>>      >>>>>>>>>>>>> issues,
>>>>>>>      >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
>>>>>>>      that Java
>>>>>>>      >>>>> is
>>>>>>>      >>>>>>>> smart
>>>>>>>      >>>>>>>>>>>>> enough to
>>>>>>>      >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>>>>>>>      returning one
>>>>>>>      >>>>>>> thing
>>>>>>>      >>>>>>>>> and
>>>>>>>      >>>>>>>>>>>>> branch()
>>>>>>>      >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>>      >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
>>>>>>>      need
>>>>>>>      >>>>> it.
>>>>>>>      >>>>>>> We
>>>>>>>      >>>>>>>>> can
>>>>>>>      >>>>>>>>>>>>> just
>>>>>>>      >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
>>>>>>>      shares
>>>>>>>      >>>>>> its
>>>>>>>      >>>>>>>>> state
>>>>>>>      >>>>>>>>>>>>> with the
>>>>>>>      >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
>>>>>>>      branching.
>>>>>>>      >>>>>>> It's
>>>>>>>      >>>>>>>>> not
>>>>>>>      >>>>>>>>>>>>> terribly
>>>>>>>      >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>>>>>>>      demonstrates
>>>>>>>      >>>>>> its
>>>>>>>      >>>>>>>>>>>>> feasibility.
>>>>>>>      >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should
>>>>>> be
>>>>>>>      >>>>> final
>>>>>>>      >>>>>> or
>>>>>>>      >>>>>>>>> even a
>>>>>>>      >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
>>>>>>>      wanted to
>>>>>>>      >>>>>> see
>>>>>>>      >>>>>>>> how
>>>>>>>      >>>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>>>      >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
>>>>>>>      solution
>>>>>>>      >>>>>>> could
>>>>>>>      >>>>>>>> be
>>>>>>>      >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>>>>>>      >>>>> suggested
>>>>>>>      >>>>>>>> was a
>>>>>>>      >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
>>>>>>>      streams
>>>>>>>      >>>>>> are
>>>>>>>      >>>>>>>> not
>>>>>>>      >>>>>>>>>>>>>>>> available in the same scope as each other.  That
>>>>>>>      is, if we
>>>>>>>      >>>>>>> wanted
>>>>>>>      >>>>>>>>> to
>>>>>>>      >>>>>>>>>>>>> merge
>>>>>>>      >>>>>>>>>>>>>>>> them back together again I don't see a way to do
>>>>>>>      that.  The
>>>>>>>      >>>>>> KIP
>>>>>>>      >>>>>>>>>>>>> proposal
>>>>>>>      >>>>>>>>>>>>>>>> has the same issue, though - all this means is that
>>>>>> for
>>>>>>>      >>>>>> either
>>>>>>>      >>>>>>>>>>>>> solution,
>>>>>>>      >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
>>>>>>>      table.
>>>>>>>      >>>>>>>>>>>>>>>> Thanks,
>>>>>>>      >>>>>>>>>>>>>>>> Paul
>>>>>>>      >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>>>      >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>>      >>>>>>>>>>>>> wrote:
>>>>>>>      >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to
>>>>>> this
>>>>>>>      >>>>>> point.
>>>>>>>      >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
>>>>>>>      branch API
>>>>>>>      >>>>>>> needs
>>>>>>>      >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>      >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>      >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
>>>>>>>      returns
>>>>>>>      >>>>>> its
>>>>>>>      >>>>>>>>> argument
>>>>>>>      >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code
>>>>>> won't
>>>>>>>      >>>>> make
>>>>>>>      >>>>>>>> sense
>>>>>>>      >>>>>>>>>>>>> until
>>>>>>>      >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
>>>>>>>      instance
>>>>>>>      >>>>>>>>> contrasts the
>>>>>>>      >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> stream
>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>      >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>      >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>>>>      >>>>>>>>> defaultBranch(..)
>>>>>>>      >>>>>>>>>>>>> and
>>>>>>>      >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface
>>>>>> is
>>>>>>>      >>>>>> defined.
>>>>>>>      >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>>>      >>>>>>>> (defaultBranch(ks->)
>>>>>>>      >>>>>>>>> and
>>>>>>>      >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
>>>>>>>      miss the
>>>>>>>      >>>>>> fact
>>>>>>>      >>>>>>>>> that one
>>>>>>>      >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
>>>>>>>      methods
>>>>>>>      >>>>>> are
>>>>>>>      >>>>>>>> not
>>>>>>>      >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do
>>>>>> better?
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> Ivan
>>>>>>>      >>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>      >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>      >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>>>      >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>>>>>>>      implemented the
>>>>>>>      >>>>>>> easy
>>>>>>>      >>>>>>>>> way.
>>>>>>>      >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
>>>>>> assumes
>>>>>>>      >>>>> nothing
>>>>>>>      >>>>>>>> will
>>>>>>>      >>>>>>>>>>>>> reach
>>>>>>>      >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>      >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only
>>>>>> option
>>>>>>>      >>>>>> besides
>>>>>>>      >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
>>>>>>>      want to
>>>>>>>      >>>>>> just
>>>>>>>      >>>>>>>>> silently
>>>>>>>      >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>>      predicate. 2)
>>>>>>>      >>>>>>> Throwing
>>>>>>>      >>>>>>>>> an
>>>>>>>      >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
>>>>>>>      looks
>>>>>>>      >>>>>> like a
>>>>>>>      >>>>>>>> bad
>>>>>>>      >>>>>>>>>>>>> idea.
>>>>>>>      >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
>>>>>>>      emit a
>>>>>>>      >>>>>>>> special
>>>>>>>      >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly
>>>>>> where
>>>>>>>      >>>>>>> `default`
>>>>>>>      >>>>>>>>> can
>>>>>>>      >>>>>>>>>>>>> be
>>>>>>>      >>>>>>>>>>>>>>>>>>> used.
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>      >>>>> InternalTopologyBuilder
>>>>>>>      >>>>>>> to
>>>>>>>      >>>>>>>>> track
>>>>>>>      >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>      >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>>>      a clear
>>>>>>>      >>>>>>> error
>>>>>>>      >>>>>>>>>>>>> before it
>>>>>>>      >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>>>>>>      >>>>> compiled
>>>>>>>      >>>>>>> and
>>>>>>>      >>>>>>>>> run?
>>>>>>>      >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>>>>>>>      compile if
>>>>>>>      >>>>> used
>>>>>>>      >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>>>>>>>      method chain
>>>>>>>      >>>>>>>> starting
>>>>>>>      >>>>>>>>>>>>> from
>>>>>>>      >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
>>>>>>>      between
>>>>>>>      >>>>>>>> runtime
>>>>>>>      >>>>>>>>> and
>>>>>>>      >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>>>>>>      >>>>> instantly
>>>>>>>      >>>>>> on
>>>>>>>      >>>>>>>>> unit
>>>>>>>      >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
>>>>>>>      compilation
>>>>>>>      >>>>>>>> failure.
>>>>>>>      >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>      >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
>>>>>>>      required.
>>>>>>>      >>>>>>> But
>>>>>>>      >>>>>>>> is
>>>>>>>      >>>>>>>>>>>>> that
>>>>>>>      >>>>>>>>>>>>>>>>>>>> really
>>>>>>>      >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>>>>>      >>>>>> defaultBranch
>>>>>>>      >>>>>>>>> they
>>>>>>>      >>>>>>>>>>>>> can
>>>>>>>      >>>>>>>>>>>>>>>>>>>> call
>>>>>>>      >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
>>>>>>>      just as
>>>>>>>      >>>>>>>>> easily.  In
>>>>>>>      >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>      >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API
>>>>>> - a
>>>>>>>      >>>>> user
>>>>>>>      >>>>>>>> could
>>>>>>>      >>>>>>>>>>>>> specify
>>>>>>>      >>>>>>>>>>>>>>>>> a
>>>>>>>      >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach
>>>>>> the
>>>>>>>      >>>>>> default
>>>>>>>      >>>>>>>>> branch,
>>>>>>>      >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>> That
>>>>>>>      >>>>> seems
>>>>>>>      >>>>>>> like
>>>>>>>      >>>>>>>>> an
>>>>>>>      >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
>>>>>>>      which allows
>>>>>>>      >>>>>> for
>>>>>>>      >>>>>>>> the
>>>>>>>      >>>>>>>>>>>>> more
>>>>>>>      >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>      >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
>>>>>> dropped.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
>>>>>>>      to be
>>>>>>>      >>>>>> well
>>>>>>>      >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>      >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>>      >>>>> InternalTopologyBuilder
>>>>>>>      >>>>>>> to
>>>>>>>      >>>>>>>>> track
>>>>>>>      >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>      >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>>>      a clear
>>>>>>>      >>>>>>> error
>>>>>>>      >>>>>>>>>>>>> before it
>>>>>>>      >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is
>>>>>> a
>>>>>>>      >>>>> "build
>>>>>>>      >>>>>>>> step"
>>>>>>>      >>>>>>>>>>>>> where
>>>>>>>      >>>>>>>>>>>>>>>>> the
>>>>>>>      >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>>      >>>>>> StreamsBuilder.build()
>>>>>>>      >>>>>>> is
>>>>>>>      >>>>>>>>>>>>> called.
>>>>>>>      >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I
>>>>>> agree
>>>>>>>      >>>>> that
>>>>>>>      >>>>>>> it's
>>>>>>>      >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>      >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
>>>>>>>      stream.
>>>>>>>      >>>>>>> With
>>>>>>>      >>>>>>>>> the
>>>>>>>      >>>>>>>>>>>>>>>>> fluent
>>>>>>>      >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all
>>>>>> other
>>>>>>>      >>>>>>> operations
>>>>>>>      >>>>>>>>> do -
>>>>>>>      >>>>>>>>>>>>> if
>>>>>>>      >>>>>>>>>>>>>>>>> you
>>>>>>>      >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
>>>>>> multiple
>>>>>>>      >>>>> times,
>>>>>>>      >>>>>>> you
>>>>>>>      >>>>>>>>> just
>>>>>>>      >>>>>>>>>>>>>>>>>>>> need the
>>>>>>>      >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
>>>>>>>      operations
>>>>>>>      >>>>>> on
>>>>>>>      >>>>>>> it
>>>>>>>      >>>>>>>>> as
>>>>>>>      >>>>>>>>>>>>> you
>>>>>>>      >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>      >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>>>      >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>>      >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
>>>>>>>      always need
>>>>>>>      >>>>>> the
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>>> operation we
>>>>>>>      >>>>> don't
>>>>>>>      >>>>>>>> know
>>>>>>>      >>>>>>>>>>>>> when to
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
>>>>>>>      so we
>>>>>>>      >>>>> can
>>>>>>>      >>>>>> do
>>>>>>>      >>>>>>>>>>>>> something
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
>>>>>> special
>>>>>>>      >>>>> object
>>>>>>>      >>>>>>>>>>>>> construction
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods.
>>>>>> But
>>>>>>>      >>>>> here
>>>>>>>      >>>>>> we
>>>>>>>      >>>>>>>>> have a
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
>>>>>>>      flow,
>>>>>>>      >>>>> so
>>>>>>>      >>>>>> I
>>>>>>>      >>>>>>>>> think
>>>>>>>      >>>>>>>>>>>>> this
>>>>>>>      >>>>>>>>>>>>>>>>> is
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
>>>>>>>      API, but I
>>>>>>>      >>>>>> find
>>>>>>>      >>>>>>>> the
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>>>>>>>      contrasts the
>>>>>>>      >>>>>>> fluency
>>>>>>>      >>>>>>>>> of
>>>>>>>      >>>>>>>>>>>>> other
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
>>>>>>>      just call
>>>>>>>      >>>>> a
>>>>>>>      >>>>>>>>> method on
>>>>>>>      >>>>>>>>>>>>> the
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
>>>>>>>      cases
>>>>>>>      >>>>> are
>>>>>>>      >>>>>>>>> defined
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
>>>>>>>      is very
>>>>>>>      >>>>>> nice
>>>>>>>      >>>>>>>>> and the
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> way
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
>>>>>>>      how we
>>>>>>>      >>>>>>> specify
>>>>>>>      >>>>>>>>> the
>>>>>>>      >>>>>>>>>>>>> source
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>>> this::handle1)
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>>> this::handle2)
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>>>>      >>>>>>>> KStreamBrancher
>>>>>>>      >>>>>>>>> or
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>>      terminated by
>>>>>>>      >>>>>>>>>>>>> defaultBranch()
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>>      incompatible with
>>>>>>>      >>>>> the
>>>>>>>      >>>>>>>>> current
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
>>>>>>>      different
>>>>>>>      >>>>>> name,
>>>>>>>      >>>>>>>> but
>>>>>>>      >>>>>>>>> that
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>>>>>      >>>>>> something
>>>>>>>      >>>>>>>> like
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> or
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
>>>>>>>      KIP?  It
>>>>>>>      >>>>>> seems
>>>>>>>      >>>>>>>>> like it
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
>>>>>>>      while also
>>>>>>>      >>>>>>>> allowing
>>>>>>>      >>>>>>>>> you
>>>>>>>      >>>>>>>>>>>>> to
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>>      KBranchedStreams
>>>>>>>      >>>>>> if
>>>>>>>      >>>>>>>>> desired.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
>>>>>>>      ks){
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>>>>>>>      String> ks){
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>>      this::handleFirstCase)
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>>      this::handleSecondCase)
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>>> KafkaStreamsBrancher
>>>>>>>      >>>>> takes a
>>>>>>>      >>>>>>>>> Consumer
>>>>>>>      >>>>>>>>>>>>> as a
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
>>>>>>>      example in
>>>>>>>      >>>>>> the
>>>>>>>      >>>>>>>> KIP
>>>>>>>      >>>>>>>>>>>>> shows
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>>>      >>>>>>>>> (KafkaStreams#to()
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would
>>>>>> we
>>>>>>>      >>>>> handle
>>>>>>>      >>>>>>> the
>>>>>>>      >>>>>>>>> case
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to
>>>>>> continue
>>>>>>>      >>>>>>>> processing
>>>>>>>      >>>>>>>>> and
>>>>>>>      >>>>>>>>>>>>> not
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the
>>>>>> branched
>>>>>>>      >>>>>> stream
>>>>>>>      >>>>>>>>>>>>> immediately?
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
>>>>>>>      we had
>>>>>>>      >>>>>>>> something
>>>>>>>      >>>>>>>>> like
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck
>>>>>> <
>>>>>>>      >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for
>>>>>> KIP-
>>>>>>>      >>>>> 418.
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
>>>>>> KIP-418.
>>>>>>>      >>>>> Please
>>>>>>>      >>>>>>>> take
>>>>>>>      >>>>>>>>> a
>>>>>>>      >>>>>>>>>>>>> look
>>>>>>>      >>>>>>>>>>>>>>>>> at
>>>>>>>      >>>>>>>>>>>>>>>>>>>>> the
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
>>>>>>>      feedback :)
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>      >>>>>
>>>>>>>
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>      >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>>      >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>      >>>>>>>>>
>>>>>>>      >
>>>>>>>
>>>>>>
>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I am moving this KIP into "inactive status". Feel free to resume the KIP
at any point.

If anybody else is interested in picking up this KIP, feel free to do so.



-Matthias

On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> Ivan,
> 
> did you see my last reply? What do you think about my proposal to mix
> both approaches and try to get best-of-both worlds?
> 
> 
> -Matthias
> 
> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>> Thanks for the input John!
>>
>>> under your suggestion, it seems that the name is required
>>
>> If you want to get the `KStream` as part of the `Map` back using a
>> `Function`, yes. If you follow the "embedded chaining" pattern using a
>> `Consumer`, no.
>>
>> Allowing for a default name via `split()` can of course be done.
>> Similarly, using `Named` instead of `String` is possible.
>>
>> I wanted to sketch out a high level proposal to merge both patterns
>> only. Your suggestions to align the new API with the existing API make
>> totally sense.
>>
>>
>>
>> One follow up question: Would `Named` be optional or required in
>> `split()` and `branch()`? It's unclear from your example.
>>
>> If both are mandatory, what do we gain by it? The returned `Map` only
>> contains the corresponding branches, so why should we prefix all of
>> them? If only `Named` is mandatory in `branch()`, but optional in
>> `split()`, the same question raises?
>>
>> Requiring `Named` in `split()` seems only to make sense, if `Named` is
>> optional in `branch()` and we generate `-X` suffix using a counter for
>> different branch name. However, this might lead to the problem of
>> changing names if branches are added/removed. Also, how would the names
>> be generated if `Consumer` is mixed in (ie, not all branches are
>> returned in the `Map`).
>>
>> If `Named` is optional for both, it could happen that a user misses to
>> specify a name for a branch what would lead to runtime issues.
>>
>>
>> Hence, I am actually in favor to not allow a default name but keep
>> `split()` without parameter and make `Named` in `branch()` required if a
>> `Function` is used. This makes it explicit to the user that specifying a
>> name is required if a `Function` is used.
>>
>>
>>
>> About
>>
>>> KBranchedStream#branch(BranchConfig)
>>
>> I don't think that the branching predicate is a configuration and hence
>> would not include it in a configuration object.
>>
>>>     withChain(...);
>>
>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>> seem to be a configuration. We can also not prevent a user to call
>> `withName()` in combination of `withChain()` what does not make sense
>> IMHO. We could of course throw an RTE but not have a compile time check
>> seems less appealing. Also, it could happen that neither `withChain()`
>> not `withName()` is called and the branch is missing in the returned
>> `Map` what lead to runtime issues, too.
>>
>> Hence, I don't think that we should add `BranchConfig`. A config object
>> is helpful if each configuration can be set independently of all others,
>> but this seems not to be the case here. If we add new configuration
>> later, we can also just move forward by deprecating the methods that
>> accept `Named` and add new methods that accepted `BranchConfig` (that
>> would of course implement `Named`).
>>
>>
>> Thoughts?
>>
>>
>> @Ivan, what do you think about the general idea to blend the two main
>> approaches of returning a `Map` plus an "embedded chaining"?
>>
>>
>>
>> -Matthias
>>
>>
>>
>> On 6/4/19 10:33 AM, John Roesler wrote:
>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>> everyone. Returning the map from the terminal operations also solves
>>> the problem of merging/joining the branched streams, if we want to add
>>> support for the compliment later on.
>>>
>>> Under your suggestion, it seems that the name is required. Otherwise,
>>> we wouldn't have keys for the map to return. I this this is actually
>>> not too bad, since experience has taught us that, although names for
>>> operations are not required to define stream processing logic, it does
>>> significantly improve the operational experience when you can map the
>>> topology, logs, metrics, etc. back to the source code. Since you
>>> wouldn't (have to) reference the name to chain extra processing onto
>>> the branch (thanks to the second argument), you can avoid the
>>> "unchecked name" problem that Ivan pointed out.
>>>
>>> In the current implementation of Branch, you can name the branch
>>> operator itself, and then all the branches get index-suffixed names
>>> built from the branch operator name. I guess under this proposal, we
>>> could naturally append the branch name to the branching operator name,
>>> like this:
>>>
>>>    stream.split(Named.withName("mysplit")) //creates node "mysplit"
>>>               .branch(..., ..., "abranch") // creates node "mysplit-abranch"
>>>               .defaultBranch(...) // creates node "mysplit-default"
>>>
>>> It does make me wonder about the DSL syntax itself, though.
>>>
>>> We don't have a defined grammar, so there's plenty of room to debate
>>> the "best" syntax in the context of each operation, but in general,
>>> the KStream DSL operators follow this pattern:
>>>
>>>     operator(function, config_object?) OR operator(config_object)
>>>
>>> where config_object is often just Named in the "function" variant.
>>> Even when the config_object isn't a Named, but some other config
>>> class, that config class _always_ implements NamedOperation.
>>>
>>> Here, we're introducing a totally different pattern:
>>>
>>>   operator(function, function, string)
>>>
>>> where the string is the name.
>>> My first question is whether the name should instead be specified with
>>> the NamedOperation interface.
>>>
>>> My second question is whether we should just roll all these arguments
>>> up into a config object like:
>>>
>>>    KBranchedStream#branch(BranchConfig)
>>>
>>>    interface BranchConfig extends NamedOperation {
>>>     withPredicate(...);
>>>     withChain(...);
>>>     withName(...);
>>>   }
>>>
>>> Although I guess we'd like to call BranchConfig something more like
>>> "Branched", even if I don't particularly like that pattern.
>>>
>>> This makes the source code a little noisier, but it also makes us more
>>> future-proof, as we can deal with a wide range of alternatives purely
>>> in the config interface, and never have to deal with adding overloads
>>> to the KBranchedStream if/when we decide we want the name to be
>>> optional, or the KStream->KStream to be optional.
>>>
>>> WDYT?
>>>
>>> Thanks,
>>> -John
>>>
>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>> <mi...@confluent.io> wrote:
>>>>
>>>> Matthias: I think that's pretty reasonable from my point of view. Good
>>>> suggestion.
>>>>
>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>>
>>>>> Interesting discussion.
>>>>>
>>>>> I am wondering, if we cannot unify the advantage of both approaches:
>>>>>
>>>>>
>>>>>
>>>>> KStream#split() -> KBranchedStream
>>>>>
>>>>> // branch is not easily accessible in current scope
>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>   -> KBranchedStream
>>>>>
>>>>> // assign a name to the branch and
>>>>> // return the sub-stream to the current scope later
>>>>> //
>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>   -> KBranchedStream
>>>>>
>>>>> // default branch is not easily accessible
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>   -> Map<String,KStream>
>>>>>
>>>>> // assign custom name to default-branch
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>   -> Map<String,KStream>
>>>>>
>>>>> // assign a default name for default
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>   -> Map<String,KStream>
>>>>>
>>>>> // return map of all names sub-stream into current scope
>>>>> KBranchedStream#noDefaultBranch()
>>>>>   -> Map<String,KStream>
>>>>>
>>>>>
>>>>>
>>>>> Hence, for each sub-stream, the user can pick to add a name and return
>>>>> the branch "result" to the calling scope or not. The implementation can
>>>>> also check at runtime that all returned names are unique. The returned
>>>>> Map can be empty and it's also optional to use the Map.
>>>>>
>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>
>>>>> Thoughts?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>> Ivan,
>>>>>>
>>>>>> That's a very good point about the "start" operator in the dynamic case.
>>>>>> I had no problem with "split()"; I was just questioning the necessity.
>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>> "split()" start operator. Thanks!
>>>>>>
>>>>>> Separately, I'm interested to see where the present discussion leads.
>>>>>> I've written enough Javascript code in my life to be suspicious of
>>>>>> nested closures. You have a good point about using method references (or
>>>>>> indeed function literals also work). It should be validating that this
>>>>>> was also the JS community's first approach to flattening the logic when
>>>>>> their nested closure situation got out of hand. Unfortunately, it's
>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>> readability (but in different ways for different reasons). In other
>>>>>> words, I agree that function references is *the* first-order solution if
>>>>>> the nested code does indeed become a problem.
>>>>>>
>>>>>> However, the history of JS also tells us that function references aren't
>>>>>> the end of the story either, and you can see that by observing that
>>>>>> there have been two follow-on eras, as they continue trying to cope with
>>>>>> the consequences of living in such a callback-heavy language. First, you
>>>>>> have Futures/Promises, which essentially let you convert nested code to
>>>>>> method-chained code (Observables/FP is a popular variation on this).
>>>>>> Most lately, you have async/await, which is an effort to apply language
>>>>>> (not just API) syntax to the problem, and offer the "flattest" possible
>>>>>> programming style to solve the problem (because you get back to just one
>>>>>> code block per functional unit).
>>>>>>
>>>>>> Stream-processing is a different domain, and Java+KStreams is nowhere
>>>>>> near as callback heavy as JS, so I don't think we have to take the JS
>>>>>> story for granted, but then again, I think we can derive some valuable
>>>>>> lessons by looking sideways to adjacent domains. I'm just bringing this
>>>>>> up to inspire further/deeper discussion. At the same time, just like JS,
>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>
>>>>>> Separately again, I'm interested in the post-branch merge (and I'd also
>>>>>> add join) problem that Paul brought up. We can clearly punt on it, by
>>>>>> terminating the nested branches with sink operators. But is there a DSL
>>>>>> way to do it?
>>>>>>
>>>>>> Thanks again for your driving this,
>>>>>> -John
>>>>>>
>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>> <ma...@gmail.com>> wrote:
>>>>>>
>>>>>>     Ivan, I’ll definitely forfeit my point on the clumsiness of the
>>>>>>     branch(predicate, consumer) solution, I don’t see any real drawbacks
>>>>>>     for the dynamic case.
>>>>>>
>>>>>>     IMO the one trade off to consider at this point is the scope
>>>>>>     question. I don’t know if I totally agree that “we rarely need them
>>>>>>     in the same scope” since merging the branches back together later
>>>>>>     seems like a perfectly plausible use case that can be a lot nicer
>>>>>>     when the branched streams are in the same scope. That being said,
>>>>>>     for the reasons Ivan listed, I think it is overall the better
>>>>>>     solution - working around the scope thing is easy enough if you need
>>>>>>     to.
>>>>>>
>>>>>>     > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>     <ip...@mail.ru.invalid> wrote:
>>>>>>     >
>>>>>>     > Hello everyone, thank you all for joining the discussion!
>>>>>>     >
>>>>>>     > Well, I don't think the idea of named branches, be it a
>>>>>>     LinkedHashMap (no other Map will do, because order of definition
>>>>>>     matters) or `branch` method  taking name and Consumer has more
>>>>>>     advantages than drawbacks.
>>>>>>     >
>>>>>>     > In my opinion, the only real positive outcome from Michael's
>>>>>>     proposal is that all the returned branches are in the same scope.
>>>>>>     But 1) we rarely need them in the same scope 2) there is a
>>>>>>     workaround for the scope problem, described in the KIP.
>>>>>>     >
>>>>>>     > 'Inlining the complex logic' is not a problem, because we can use
>>>>>>     method references instead of lambdas. In real world scenarios you
>>>>>>     tend to split the complex logic to methods anyway, so the code is
>>>>>>     going to be clean.
>>>>>>     >
>>>>>>     > The drawbacks are strong. The cohesion between predicates and
>>>>>>     handlers is lost. We have to define predicates in one place, and
>>>>>>     handlers in another. This opens the door for bugs:
>>>>>>     >
>>>>>>     > - what if we forget to define a handler for a name? or a name for
>>>>>>     a handler?
>>>>>>     > - what if we misspell a name?
>>>>>>     > - what if we copy-paste and duplicate a name?
>>>>>>     >
>>>>>>     > What Michael propose would have been totally OK if we had been
>>>>>>     writing the API in Lua, Ruby or Python. In those languages the
>>>>>>     "dynamic naming" approach would have looked most concise and
>>>>>>     beautiful. But in Java we expect all the problems related to
>>>>>>     identifiers to be eliminated in compile time.
>>>>>>     >
>>>>>>     > Do we have to invent duck-typing for the Java API?
>>>>>>     >
>>>>>>     > And if we do, what advantage are we supposed to get besides having
>>>>>>     all the branches in the same scope? Michael, maybe I'm missing your
>>>>>>     point?
>>>>>>     >
>>>>>>     > ---
>>>>>>     >
>>>>>>     > Earlier in this discussion John Roesler also proposed to do
>>>>>>     without "start branching" operator, and later Paul mentioned that in
>>>>>>     the case when we have to add a dynamic number of branches, the
>>>>>>     current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
>>>>>>     me address both comments here.
>>>>>>     >
>>>>>>     > 1) "Start branching" operator (I think that *split* is a good name
>>>>>>     for it indeed) is critical when we need to do a dynamic branching,
>>>>>>     see example below.
>>>>>>     >
>>>>>>     > 2) No, dynamic branching in current KIP is not clumsy at all.
>>>>>>     Imagine a real-world scenario when you need one branch per enum
>>>>>>     value (say, RecordType). You can have something like this:
>>>>>>     >
>>>>>>     > /*John:if we had to start with stream.branch(...) here, it would
>>>>>>     have been much messier.*/
>>>>>>     > KBranchedStream branched = stream.split();
>>>>>>     >
>>>>>>     > /*Not clumsy at all :-)*/
>>>>>>     > for (RecordType recordType : RecordType.values())
>>>>>>     >             branched = branched.branch((k, v) -> v.getRecType() ==
>>>>>>     recordType,
>>>>>>     >                     recordType::processRecords);
>>>>>>     >
>>>>>>     > Regards,
>>>>>>     >
>>>>>>     > Ivan
>>>>>>     >
>>>>>>     >
>>>>>>     > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>     >> I also agree with Michael's observation about the core problem of
>>>>>>     >> current `branch()` implementation.
>>>>>>     >>
>>>>>>     >> However, I also don't like to pass in a clumsy Map object. My
>>>>>>     thinking
>>>>>>     >> was more aligned with Paul's proposal to just add a name to each
>>>>>>     >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>     >>
>>>>>>     >> It makes the code easier to read, and also make the order of
>>>>>>     >> `Predicates` (that is essential) easier to grasp.
>>>>>>     >>
>>>>>>     >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>     >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>     >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>     >>>>>>    .defaultBranch("defaultBranch");
>>>>>>     >> An open question is the case for which no defaultBranch() should
>>>>> be
>>>>>>     >> specified. Atm, `split()` and `branch()` would return
>>>>>>     `BranchedKStream`
>>>>>>     >> and the call to `defaultBranch()` that returns the `Map` is
>>>>> mandatory
>>>>>>     >> (what is not the case atm). Or is this actually not a real
>>>>> problem,
>>>>>>     >> because users can just ignore the branch returned by
>>>>>>     `defaultBranch()`
>>>>>>     >> in the result `Map` ?
>>>>>>     >>
>>>>>>     >>
>>>>>>     >> About "inlining": So far, it seems to be a matter of personal
>>>>>>     >> preference. I can see arguments for both, but no "killer
>>>>>>     argument" yet
>>>>>>     >> that clearly make the case for one or the other.
>>>>>>     >>
>>>>>>     >>
>>>>>>     >> -Matthias
>>>>>>     >>
>>>>>>     >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>     >>> Perhaps inlining is the wrong terminology. It doesn’t require
>>>>>>     that a lambda with the full downstream topology be defined inline -
>>>>>>     it can be a method reference as with Ivan’s original suggestion.
>>>>>>     The advantage of putting the predicate and its downstream logic
>>>>>>     (Consumer) together in branch() is that they are required to be near
>>>>>>     to each other.
>>>>>>     >>>
>>>>>>     >>> Ultimately the downstream code has to live somewhere, and deep
>>>>>>     branch trees will be hard to read regardless.
>>>>>>     >>>
>>>>>>     >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>     <michael.drogalis@confluent.io
>>>>>>     <ma...@confluent.io>> wrote:
>>>>>>     >>>>
>>>>>>     >>>> I'm less enthusiastic about inlining the branch logic with its
>>>>>>     downstream
>>>>>>     >>>> functionality. Programs that have deep branch trees will
>>>>>>     quickly become
>>>>>>     >>>> harder to read as a single unit.
>>>>>>     >>>>
>>>>>>     >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>     <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>     >>>>>
>>>>>>     >>>>> Also +1 on the issues/goals as Michael outlined them, I think
>>>>>>     that sets a
>>>>>>     >>>>> great framework for the discussion.
>>>>>>     >>>>>
>>>>>>     >>>>> Regarding the SortedMap solution, my understanding is that the
>>>>>>     current
>>>>>>     >>>>> proposal in the KIP is what is in my PR which (pending naming
>>>>>>     decisions) is
>>>>>>     >>>>> roughly this:
>>>>>>     >>>>>
>>>>>>     >>>>> stream.split()
>>>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>     >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>     >>>>>
>>>>>>     >>>>> Obviously some ordering is necessary, since branching as a
>>>>>>     construct
>>>>>>     >>>>> doesn't work without it, but this solution seems like it
>>>>>>     provides as much
>>>>>>     >>>>> associativity as the SortedMap solution, because each branch()
>>>>>>     call
>>>>>>     >>>>> directly associates the "conditional" with the "code block."
>>>>>>     The value it
>>>>>>     >>>>> provides over the KIP solution is the accessing of streams in
>>>>>>     the same
>>>>>>     >>>>> scope.
>>>>>>     >>>>>
>>>>>>     >>>>> The KIP solution is less "dynamic" than the SortedMap solution
>>>>>>     in the sense
>>>>>>     >>>>> that it is slightly clumsier to add a dynamic number of
>>>>>>     branches, but it is
>>>>>>     >>>>> certainly possible.  It seems to me like the API should favor
>>>>>>     the "static"
>>>>>>     >>>>> case anyway, and should make it simple and readable to
>>>>>>     fluently declare and
>>>>>>     >>>>> access your branches in-line.  It also makes it impossible to
>>>>>>     ignore a
>>>>>>     >>>>> branch, and it is possible to build an (almost) identical
>>>>>>     SortedMap
>>>>>>     >>>>> solution on top of it.
>>>>>>     >>>>>
>>>>>>     >>>>> I could also see a middle ground where instead of a raw
>>>>>>     SortedMap being
>>>>>>     >>>>> taken in, branch() takes a name and not a Consumer.  Something
>>>>>>     like this:
>>>>>>     >>>>>
>>>>>>     >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>     >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>     >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>     >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>>>>     >>>>>
>>>>>>     >>>>> Pros for that solution:
>>>>>>     >>>>> - accessing branched KStreams in same scope
>>>>>>     >>>>> - no double brace initialization, hopefully slightly more
>>>>>>     readable than
>>>>>>     >>>>> SortedMap
>>>>>>     >>>>>
>>>>>>     >>>>> Cons
>>>>>>     >>>>> - downstream branch logic cannot be specified inline which
>>>>>>     makes it harder
>>>>>>     >>>>> to read top to bottom (like existing API and SortedMap, but
>>>>>>     unlike the KIP)
>>>>>>     >>>>> - you can forget to "handle" one of the branched streams (like
>>>>>>     existing
>>>>>>     >>>>> API and SortedMap, but unlike the KIP)
>>>>>>     >>>>>
>>>>>>     >>>>> (KBranchedStreams could even work *both* ways but perhaps
>>>>>>     that's overdoing
>>>>>>     >>>>> it).
>>>>>>     >>>>>
>>>>>>     >>>>> Overall I'm curious how important it is to be able to easily
>>>>>>     access the
>>>>>>     >>>>> branched KStream in the same scope as the original.  It's
>>>>>>     possible that it
>>>>>>     >>>>> doesn't need to be handled directly by the API, but instead
>>>>>>     left up to the
>>>>>>     >>>>> user.  I'm sort of in the middle on it.
>>>>>>     >>>>>
>>>>>>     >>>>> Paul
>>>>>>     >>>>>
>>>>>>     >>>>>
>>>>>>     >>>>>
>>>>>>     >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>     <sophie@confluent.io <ma...@confluent.io>>
>>>>>>     >>>>> wrote:
>>>>>>     >>>>>
>>>>>>     >>>>>> I'd like to +1 what Michael said about the issues with the
>>>>>>     existing
>>>>>>     >>>>> branch
>>>>>>     >>>>>> method, I agree with what he's outlined and I think we should
>>>>>>     proceed by
>>>>>>     >>>>>> trying to alleviate these problems. Specifically it seems
>>>>>>     important to be
>>>>>>     >>>>>> able to cleanly access the individual branches (eg by mapping
>>>>>>     >>>>>> name->stream), which I thought was the original intention of
>>>>>>     this KIP.
>>>>>>     >>>>>>
>>>>>>     >>>>>> That said, I don't think we should so easily give in to the
>>>>>>     double brace
>>>>>>     >>>>>> anti-pattern or force ours users into it if at all possible to
>>>>>>     >>>>> avoid...just
>>>>>>     >>>>>> my two cents.
>>>>>>     >>>>>>
>>>>>>     >>>>>> Cheers,
>>>>>>     >>>>>> Sophie
>>>>>>     >>>>>>
>>>>>>     >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>     >>>>>> michael.drogalis@confluent.io
>>>>>>     <ma...@confluent.io>> wrote:
>>>>>>     >>>>>>
>>>>>>     >>>>>>> I’d like to propose a different way of thinking about this.
>>>>>>     To me,
>>>>>>     >>>>> there
>>>>>>     >>>>>>> are three problems with the existing branch signature:
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> 1. If you use it the way most people do, Java raises unsafe
>>>>> type
>>>>>>     >>>>>> warnings.
>>>>>>     >>>>>>> 2. The way in which you use the stream branches is
>>>>>>     positionally coupled
>>>>>>     >>>>>> to
>>>>>>     >>>>>>> the ordering of the conditionals.
>>>>>>     >>>>>>> 3. It is brittle to extend existing branch calls with
>>>>>>     additional code
>>>>>>     >>>>>>> paths.
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> Using associative constructs instead of relying on ordered
>>>>>>     constructs
>>>>>>     >>>>>> would
>>>>>>     >>>>>>> be a stronger approach. Consider a signature that instead
>>>>>>     looks like
>>>>>>     >>>>>> this:
>>>>>>     >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
>>>>>>     Predicate<?
>>>>>>     >>>>>>> super K,? super V>>);
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> Branches are given names in a map, and as a result, the API
>>>>>>     returns a
>>>>>>     >>>>>>> mapping of names to streams. The ordering of the
>>>>> conditionals is
>>>>>>     >>>>>> maintained
>>>>>>     >>>>>>> because it’s a sorted map. Insert order determines the order
>>>>> of
>>>>>>     >>>>>> evaluation.
>>>>>>     >>>>>>> This solves problem 1 because there are no more varargs. It
>>>>>>     solves
>>>>>>     >>>>>> problem
>>>>>>     >>>>>>> 2 because you no longer lean on ordering to access the
>>>>>>     branch you’re
>>>>>>     >>>>>>> interested in. It solves problem 3 because you can introduce
>>>>>>     another
>>>>>>     >>>>>>> conditional by simply attaching another name to the
>>>>>>     structure, rather
>>>>>>     >>>>>> than
>>>>>>     >>>>>>> messing with the existing indices.
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> One of the drawbacks is that creating the map inline is
>>>>>>     historically
>>>>>>     >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>     voluminously, but
>>>>>>     >>>>>>> double brace initialization would clean up the aesthetics.
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>     <john@confluent.io <ma...@confluent.io>>
>>>>>>     >>>>> wrote:
>>>>>>     >>>>>>>> Hi Ivan,
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Thanks for the update.
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> FWIW, I agree with Matthias that the current "start
>>>>> branching"
>>>>>>     >>>>> operator
>>>>>>     >>>>>>> is
>>>>>>     >>>>>>>> confusing when named the same way as the actual branches.
>>>>>>     "Split"
>>>>>>     >>>>> seems
>>>>>>     >>>>>>>> like a good name. Alternatively, we can do without a "start
>>>>>>     >>>>> branching"
>>>>>>     >>>>>>>> operator at all, and just do:
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> stream
>>>>>>     >>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>      .defaultBranch();
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Tentatively, I think that this branching operation should be
>>>>>>     >>>>> terminal.
>>>>>>     >>>>>>> That
>>>>>>     >>>>>>>> way, we don't create ambiguity about how to use it. That
>>>>>>     is, `branch`
>>>>>>     >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
>>>>>>     `void`, to
>>>>>>     >>>>>>>> enforce that it comes last, and that there is only one
>>>>>>     definition of
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>> default branch. Potentially, we should log a warning if
>>>>>>     there's no
>>>>>>     >>>>>>> default,
>>>>>>     >>>>>>>> and additionally log a warning (or throw an exception) if a
>>>>>>     record
>>>>>>     >>>>>> falls
>>>>>>     >>>>>>>> though with no default.
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Thoughts?
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Thanks,
>>>>>>     >>>>>>>> -John
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>     >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>     >>>>>>>> wrote:
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>> this is to make the name similar to String#split
>>>>>>     >>>>>>>>>>> that also returns an array, right?
>>>>>>     >>>>>>>>> The intend was to avoid name duplication. The return type
>>>>>>     should
>>>>>>     >>>>>> _not_
>>>>>>     >>>>>>>>> be an array.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> The current proposal is
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> stream.branch()
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .defaultBranch();
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>     `branch()` does
>>>>>>     >>>>> not
>>>>>>     >>>>>>>>> take any parameters and has different semantics than the
>>>>> later
>>>>>>     >>>>>>>>> `branch()` calls. Note, that from the code snippet above,
>>>>> it's
>>>>>>     >>>>> hidden
>>>>>>     >>>>>>>>> that the first call is `KStream#branch()` while the others
>>>>> are
>>>>>>     >>>>>>>>> `KBranchedStream#branch()` what makes reading the code
>>>>> harder.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
>>>>>>     I though
>>>>>>     >>>>>> it
>>>>>>     >>>>>>>>> might be better to also rename `KStream#branch()` to avoid
>>>>> the
>>>>>>     >>>>> naming
>>>>>>     >>>>>>>>> overlap that seems to be confusing. The following reads
>>>>> much
>>>>>>     >>>>> cleaner
>>>>>>     >>>>>> to
>>>>>>     >>>>>>>> me:
>>>>>>     >>>>>>>>> stream.split()
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .defaultBranch();
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Maybe there is a better alternative to `split()` though to
>>>>>>     avoid
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> naming overlap.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately
>>>>> we
>>>>>>     >>>>> cannot
>>>>>>     >>>>>>> have
>>>>>>     >>>>>>>>> a method with such name :-)
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
>>>>>>     with a
>>>>>>     >>>>> short
>>>>>>     >>>>>>>> name?
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
>>>>>>     with all
>>>>>>     >>>>> it's
>>>>>>     >>>>>>>>> methods? It will be part of public API and should be
>>>>>>     contained in
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>>     >>>>>>>>> `defaultBranch()` is.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> You did not comment on the idea to add a
>>>>>>     `KBranchedStream#get(int
>>>>>>     >>>>>>> index)
>>>>>>     >>>>>>>>> -> KStream` method to get the individually
>>>>>>     branched-KStreams. Would
>>>>>>     >>>>>> be
>>>>>>     >>>>>>>>> nice to get your feedback about it. It seems you suggest
>>>>>>     that users
>>>>>>     >>>>>>>>> would need to write custom utility code otherwise, to
>>>>>>     access them.
>>>>>>     >>>>> We
>>>>>>     >>>>>>>>> should discuss the pros and cons of both approaches. It
>>>>> feels
>>>>>>     >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
>>>>>>     to get
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> branched-KStreams directly.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> -Matthias
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>     >>>>>>>>>> Hi all!
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>     >>>>>>>>>> I can see your point: this is to make the name similar to
>>>>>>     >>>>>>> String#split
>>>>>>     >>>>>>>>>> that also returns an array, right? But is it worth the
>>>>>>     loss of
>>>>>>     >>>>>>>> backwards
>>>>>>     >>>>>>>>>> compatibility? We can have overloaded branch() as well
>>>>>>     without
>>>>>>     >>>>>>>> affecting
>>>>>>     >>>>>>>>>> the existing code. Maybe the old array-based `branch`
>>>>> method
>>>>>>     >>>>> should
>>>>>>     >>>>>>> be
>>>>>>     >>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>     >>>>> BranchingKStream#branch(),
>>>>>>     >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>> BranchingKStream#default()
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default'
>>>>> is,
>>>>>>     >>>>>>> however, a
>>>>>>     >>>>>>>>>> reserved word, so unfortunately we cannot have a method
>>>>>>     with such
>>>>>>     >>>>>>> name
>>>>>>     >>>>>>>>> :-)
>>>>>>     >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument,
>>>>> but I
>>>>>>     >>>>> think
>>>>>>     >>>>>>> that
>>>>>>     >>>>>>>>>> is not required?
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Absolutely! I think that was just copy-paste error or
>>>>>>     something.
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Dear colleagues,
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>>     >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Any new suggestions/objections?
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>     >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems
>>>>> that
>>>>>>     >>>>>>> everybody
>>>>>>     >>>>>>>>>>> agrees that the current branch() method using arrays is
>>>>> not
>>>>>>     >>>>>> optimal.
>>>>>>     >>>>>>>>>>> I had a quick look into the PR and I like the overall
>>>>>>     proposal.
>>>>>>     >>>>>>> There
>>>>>>     >>>>>>>>>>> are some minor things we need to consider. I would
>>>>>>     recommend the
>>>>>>     >>>>>>>>>>> following renaming:
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>     >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>>     >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>     BranchingKStream#default()
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> It's just a suggestion to get slightly shorter method
>>>>> names.
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>     `Predicate` as
>>>>>>     >>>>>>>> argument,
>>>>>>     >>>>>>>>>>> but I think that is not required?
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> Also, we should consider KIP-307, that was recently
>>>>>>     accepted and
>>>>>>     >>>>>> is
>>>>>>     >>>>>>>>>>> currently implemented:
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>
>>>>>>
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>     >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
>>>>>>     parameter.
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> For the issue that the created `KStream` object are in
>>>>>>     different
>>>>>>     >>>>>>>> scopes:
>>>>>>     >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>>>>>>     index)` method
>>>>>>     >>>>>>> that
>>>>>>     >>>>>>>>>>> returns the corresponding "branched" result `KStream`
>>>>>>     object?
>>>>>>     >>>>>> Maybe,
>>>>>>     >>>>>>>> the
>>>>>>     >>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>>>     >>>>>> `Consumer<KStream>`
>>>>>>     >>>>>>>> but
>>>>>>     >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
>>>>>>     whatever
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>>>>> `Function` returns?
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
>>>>>>     current
>>>>>>     >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> -Matthias
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>     >>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
>>>>>>     makes sense
>>>>>>     >>>>>> for
>>>>>>     >>>>>>>> you
>>>>>>     >>>>>>>>> to
>>>>>>     >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
>>>>>>     we'll
>>>>>>     >>>>> need
>>>>>>     >>>>>>>> some
>>>>>>     >>>>>>>>>>>> buy-in from committers that have actual binding votes on
>>>>>>     >>>>> whether
>>>>>>     >>>>>>> the
>>>>>>     >>>>>>>>> KIP
>>>>>>     >>>>>>>>>>>> could be adopted.  It would be great to hear if they
>>>>>>     think this
>>>>>>     >>>>>> is
>>>>>>     >>>>>>> a
>>>>>>     >>>>>>>>> good
>>>>>>     >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
>>>>>>     starting a
>>>>>>     >>>>>>> vote,
>>>>>>     >>>>>>>>> or if
>>>>>>     >>>>>>>>>>>> there is generally some indication of interest
>>>>> beforehand.
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
>>>>>>     assuming
>>>>>>     >>>>> we
>>>>>>     >>>>>> do
>>>>>>     >>>>>>>>> move
>>>>>>     >>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>>>     >>>>>> KBranchedStream",
>>>>>>     >>>>>>> do
>>>>>>     >>>>>>>>> we
>>>>>>     >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I
>>>>> would
>>>>>>     >>>>> favor
>>>>>>     >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs
>>>>> that
>>>>>>     >>>>>>> accomplish
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>> same thing is confusing, especially when they're fairly
>>>>>>     similar
>>>>>>     >>>>>>>>> anyway.  We
>>>>>>     >>>>>>>>>>>> just need to be sure we're not making something
>>>>>>     >>>>>>> impossible/difficult
>>>>>>     >>>>>>>>> that
>>>>>>     >>>>>>>>>>>> is currently possible/easy.
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> Regarding my PR - I think the general structure would
>>>>> work,
>>>>>>     >>>>> it's
>>>>>>     >>>>>>>> just a
>>>>>>     >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>>>     >>>>>>> particular,
>>>>>>     >>>>>>>>>>>> passing in the "predicates" and "children" lists which
>>>>> get
>>>>>>     >>>>>> modified
>>>>>>     >>>>>>>> in
>>>>>>     >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>     KStreamLazyBranch is
>>>>>>     >>>>> a
>>>>>>     >>>>>>> bit
>>>>>>     >>>>>>>>>>>> complicated to follow.
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> Thanks,
>>>>>>     >>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>     >>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>> Hi Paul!
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>     convinced: your
>>>>>>     >>>>>>>> proposal
>>>>>>     >>>>>>>>>>>>> looks better and should work. We just have to document
>>>>> the
>>>>>>     >>>>>> crucial
>>>>>>     >>>>>>>>> fact
>>>>>>     >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
>>>>>>     And then
>>>>>>     >>>>>> it's
>>>>>>     >>>>>>>> all
>>>>>>     >>>>>>>>>>>>> going to be very nice.
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
>>>>>>     resume the
>>>>>>     >>>>>>>>>>>>> discussion here, right?
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>>>>>     >>>>> starting
>>>>>>     >>>>>>>> point
>>>>>>     >>>>>>>>> if
>>>>>>     >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
>>>>>>     starting
>>>>>>     >>>>>>> point.
>>>>>>     >>>>>>>>> But
>>>>>>     >>>>>>>>>>>>> as a novice in this project I might miss some important
>>>>>>     >>>>> details.
>>>>>>     >>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>>>     >>>>> stream.branch()
>>>>>>     >>>>>>>>> solution
>>>>>>     >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>>>>>     >>>>> invoked
>>>>>>     >>>>>> as
>>>>>>     >>>>>>>>> they’re
>>>>>>     >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
>>>>>>     still
>>>>>>     >>>>>> ought
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>> be
>>>>>>     >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
>>>>>>     depend on
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> branched
>>>>>>     >>>>>>>>>>>>> streams having been set.
>>>>>>     >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
>>>>>>     access
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> branched
>>>>>>     >>>>>>>>>>>>> streams in the same scope as the original stream (that
>>>>>>     is, not
>>>>>>     >>>>>>>> inside
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>>>>>     >>>>> solutions.
>>>>>>     >>>>>> It
>>>>>>     >>>>>>>>> can be
>>>>>>     >>>>>>>>>>>>> worked around though.
>>>>>>     >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
>>>>>>     excited
>>>>>>     >>>>> to
>>>>>>     >>>>>>>> hear
>>>>>>     >>>>>>>>>>>>> your thoughts!]
>>>>>>     >>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>     >>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>> Hi Paul!
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>>     >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at
>>>>> first
>>>>>>     >>>>> glance,
>>>>>>     >>>>>>> but
>>>>>>     >>>>>>>>> ---
>>>>>>     >>>>>>>>>>>>>>>> the newly branched streams are not available in the
>>>>>>     same
>>>>>>     >>>>>> scope
>>>>>>     >>>>>>> as
>>>>>>     >>>>>>>>> each
>>>>>>     >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
>>>>> together
>>>>>>     >>>>> again
>>>>>>     >>>>>> I
>>>>>>     >>>>>>>>> don't see
>>>>>>     >>>>>>>>>>>>> a way to do that.
>>>>>>     >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
>>>>>>     just
>>>>>>     >>>>>> going
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>>>>>> write in details about this issue.
>>>>>>     >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
>>>>>>     we need
>>>>>>     >>>>> to
>>>>>>     >>>>>>>>> identify
>>>>>>     >>>>>>>>>>>>> customers who have bought coffee and made a purchase
>>>>>>     in the
>>>>>>     >>>>>>>>> electronics
>>>>>>     >>>>>>>>>>>>> store to give them coupons.
>>>>>>     >>>>>>>>>>>>>>> This is the code I usually write under these
>>>>>>     circumstances
>>>>>>     >>>>>> using
>>>>>>     >>>>>>>> my
>>>>>>     >>>>>>>>>>>>> 'brancher' class:
>>>>>>     >>>>>>>>>>>>>>> @Setter
>>>>>>     >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>     >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>     >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>     >>>>>>>>>>>>>>>       return
>>>>>>     >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>     >>>>>>>>>>>>>>>       /*In the real world the code here can be
>>>>>>     complex, so
>>>>>>     >>>>>>>>> creation of
>>>>>>     >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
>>>>>>     order to
>>>>>>     >>>>>>>> separate
>>>>>>     >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>     >>>>>>>>>>>>>>>  }
>>>>>>     >>>>>>>>>>>>>>> }
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>     >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>> couponIssuer::setCoffePurchases)
>>>>>>     >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>     >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>     >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
>>>>>>     everything
>>>>>>     >>>>>>>> later,
>>>>>>     >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>     >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>     initialize the
>>>>>>     >>>>>>>>> CouponIssuer
>>>>>>     >>>>>>>>>>>>> we need the terminal operation to be called before
>>>>>>     >>>>>>>>> streamsBuilder.build()
>>>>>>     >>>>>>>>>>>>> is called.
>>>>>>     >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
>>>>>>     essentially
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>>> next
>>>>>>     >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
>>>>>>     based on
>>>>>>     >>>>> my
>>>>>>     >>>>>>>>> experience,
>>>>>>     >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>     >>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
>>>>>>     fluent
>>>>>>     >>>>> API
>>>>>>     >>>>>>>> based
>>>>>>     >>>>>>>>>>>>> off of
>>>>>>     >>>>>>>>>>>>>>>> KStream here
>>>>>>     (https://github.com/apache/kafka/pull/6512),
>>>>>>     >>>>>> and
>>>>>>     >>>>>>> I
>>>>>>     >>>>>>>>> think
>>>>>>     >>>>>>>>>>>>> I
>>>>>>     >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>     >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>>>>     >>>>>>> compatibility
>>>>>>     >>>>>>>>>>>>> issues,
>>>>>>     >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
>>>>>>     that Java
>>>>>>     >>>>> is
>>>>>>     >>>>>>>> smart
>>>>>>     >>>>>>>>>>>>> enough to
>>>>>>     >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>>>>>>     returning one
>>>>>>     >>>>>>> thing
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>> branch()
>>>>>>     >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>     >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
>>>>>>     need
>>>>>>     >>>>> it.
>>>>>>     >>>>>>> We
>>>>>>     >>>>>>>>> can
>>>>>>     >>>>>>>>>>>>> just
>>>>>>     >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
>>>>>>     shares
>>>>>>     >>>>>> its
>>>>>>     >>>>>>>>> state
>>>>>>     >>>>>>>>>>>>> with the
>>>>>>     >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
>>>>>>     branching.
>>>>>>     >>>>>>> It's
>>>>>>     >>>>>>>>> not
>>>>>>     >>>>>>>>>>>>> terribly
>>>>>>     >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>>>>>>     demonstrates
>>>>>>     >>>>>> its
>>>>>>     >>>>>>>>>>>>> feasibility.
>>>>>>     >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should
>>>>> be
>>>>>>     >>>>> final
>>>>>>     >>>>>> or
>>>>>>     >>>>>>>>> even a
>>>>>>     >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
>>>>>>     wanted to
>>>>>>     >>>>>> see
>>>>>>     >>>>>>>> how
>>>>>>     >>>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>>     >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
>>>>>>     solution
>>>>>>     >>>>>>> could
>>>>>>     >>>>>>>> be
>>>>>>     >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>>>>>     >>>>> suggested
>>>>>>     >>>>>>>> was a
>>>>>>     >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
>>>>>>     streams
>>>>>>     >>>>>> are
>>>>>>     >>>>>>>> not
>>>>>>     >>>>>>>>>>>>>>>> available in the same scope as each other.  That
>>>>>>     is, if we
>>>>>>     >>>>>>> wanted
>>>>>>     >>>>>>>>> to
>>>>>>     >>>>>>>>>>>>> merge
>>>>>>     >>>>>>>>>>>>>>>> them back together again I don't see a way to do
>>>>>>     that.  The
>>>>>>     >>>>>> KIP
>>>>>>     >>>>>>>>>>>>> proposal
>>>>>>     >>>>>>>>>>>>>>>> has the same issue, though - all this means is that
>>>>> for
>>>>>>     >>>>>> either
>>>>>>     >>>>>>>>>>>>> solution,
>>>>>>     >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
>>>>>>     table.
>>>>>>     >>>>>>>>>>>>>>>> Thanks,
>>>>>>     >>>>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>     >>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to
>>>>> this
>>>>>>     >>>>>> point.
>>>>>>     >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
>>>>>>     branch API
>>>>>>     >>>>>>> needs
>>>>>>     >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>     >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
>>>>>>     returns
>>>>>>     >>>>>> its
>>>>>>     >>>>>>>>> argument
>>>>>>     >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code
>>>>> won't
>>>>>>     >>>>> make
>>>>>>     >>>>>>>> sense
>>>>>>     >>>>>>>>>>>>> until
>>>>>>     >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
>>>>>>     instance
>>>>>>     >>>>>>>>> contrasts the
>>>>>>     >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> stream
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>>>     >>>>>>>>> defaultBranch(..)
>>>>>>     >>>>>>>>>>>>> and
>>>>>>     >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface
>>>>> is
>>>>>>     >>>>>> defined.
>>>>>>     >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>>     >>>>>>>> (defaultBranch(ks->)
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
>>>>>>     miss the
>>>>>>     >>>>>> fact
>>>>>>     >>>>>>>>> that one
>>>>>>     >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
>>>>>>     methods
>>>>>>     >>>>>> are
>>>>>>     >>>>>>>> not
>>>>>>     >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do
>>>>> better?
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>     >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>>     >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>>>>>>     implemented the
>>>>>>     >>>>>>> easy
>>>>>>     >>>>>>>>> way.
>>>>>>     >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
>>>>> assumes
>>>>>>     >>>>> nothing
>>>>>>     >>>>>>>> will
>>>>>>     >>>>>>>>>>>>> reach
>>>>>>     >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>     >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only
>>>>> option
>>>>>>     >>>>>> besides
>>>>>>     >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
>>>>>>     want to
>>>>>>     >>>>>> just
>>>>>>     >>>>>>>>> silently
>>>>>>     >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>     predicate. 2)
>>>>>>     >>>>>>> Throwing
>>>>>>     >>>>>>>>> an
>>>>>>     >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
>>>>>>     looks
>>>>>>     >>>>>> like a
>>>>>>     >>>>>>>> bad
>>>>>>     >>>>>>>>>>>>> idea.
>>>>>>     >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
>>>>>>     emit a
>>>>>>     >>>>>>>> special
>>>>>>     >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly
>>>>> where
>>>>>>     >>>>>>> `default`
>>>>>>     >>>>>>>>> can
>>>>>>     >>>>>>>>>>>>> be
>>>>>>     >>>>>>>>>>>>>>>>>>> used.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>     >>>>> InternalTopologyBuilder
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>> track
>>>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>     >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>>     a clear
>>>>>>     >>>>>>> error
>>>>>>     >>>>>>>>>>>>> before it
>>>>>>     >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>>>>>     >>>>> compiled
>>>>>>     >>>>>>> and
>>>>>>     >>>>>>>>> run?
>>>>>>     >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>>>>>>     compile if
>>>>>>     >>>>> used
>>>>>>     >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>>>>>>     method chain
>>>>>>     >>>>>>>> starting
>>>>>>     >>>>>>>>>>>>> from
>>>>>>     >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
>>>>>>     between
>>>>>>     >>>>>>>> runtime
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>>>>>     >>>>> instantly
>>>>>>     >>>>>> on
>>>>>>     >>>>>>>>> unit
>>>>>>     >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
>>>>>>     compilation
>>>>>>     >>>>>>>> failure.
>>>>>>     >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
>>>>>>     required.
>>>>>>     >>>>>>> But
>>>>>>     >>>>>>>> is
>>>>>>     >>>>>>>>>>>>> that
>>>>>>     >>>>>>>>>>>>>>>>>>>> really
>>>>>>     >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>>>>     >>>>>> defaultBranch
>>>>>>     >>>>>>>>> they
>>>>>>     >>>>>>>>>>>>> can
>>>>>>     >>>>>>>>>>>>>>>>>>>> call
>>>>>>     >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
>>>>>>     just as
>>>>>>     >>>>>>>>> easily.  In
>>>>>>     >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>     >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API
>>>>> - a
>>>>>>     >>>>> user
>>>>>>     >>>>>>>> could
>>>>>>     >>>>>>>>>>>>> specify
>>>>>>     >>>>>>>>>>>>>>>>> a
>>>>>>     >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach
>>>>> the
>>>>>>     >>>>>> default
>>>>>>     >>>>>>>>> branch,
>>>>>>     >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>> That
>>>>>>     >>>>> seems
>>>>>>     >>>>>>> like
>>>>>>     >>>>>>>>> an
>>>>>>     >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
>>>>>>     which allows
>>>>>>     >>>>>> for
>>>>>>     >>>>>>>> the
>>>>>>     >>>>>>>>>>>>> more
>>>>>>     >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>     >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
>>>>> dropped.
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
>>>>>>     to be
>>>>>>     >>>>>> well
>>>>>>     >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>     >>>>> InternalTopologyBuilder
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>> track
>>>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>     >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>>     a clear
>>>>>>     >>>>>>> error
>>>>>>     >>>>>>>>>>>>> before it
>>>>>>     >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is
>>>>> a
>>>>>>     >>>>> "build
>>>>>>     >>>>>>>> step"
>>>>>>     >>>>>>>>>>>>> where
>>>>>>     >>>>>>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>     >>>>>> StreamsBuilder.build()
>>>>>>     >>>>>>> is
>>>>>>     >>>>>>>>>>>>> called.
>>>>>>     >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I
>>>>> agree
>>>>>>     >>>>> that
>>>>>>     >>>>>>> it's
>>>>>>     >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>     >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
>>>>>>     stream.
>>>>>>     >>>>>>> With
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>> fluent
>>>>>>     >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all
>>>>> other
>>>>>>     >>>>>>> operations
>>>>>>     >>>>>>>>> do -
>>>>>>     >>>>>>>>>>>>> if
>>>>>>     >>>>>>>>>>>>>>>>> you
>>>>>>     >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
>>>>> multiple
>>>>>>     >>>>> times,
>>>>>>     >>>>>>> you
>>>>>>     >>>>>>>>> just
>>>>>>     >>>>>>>>>>>>>>>>>>>> need the
>>>>>>     >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
>>>>>>     operations
>>>>>>     >>>>>> on
>>>>>>     >>>>>>> it
>>>>>>     >>>>>>>>> as
>>>>>>     >>>>>>>>>>>>> you
>>>>>>     >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>     >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>     >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
>>>>>>     always need
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>> operation we
>>>>>>     >>>>> don't
>>>>>>     >>>>>>>> know
>>>>>>     >>>>>>>>>>>>> when to
>>>>>>     >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
>>>>>>     so we
>>>>>>     >>>>> can
>>>>>>     >>>>>> do
>>>>>>     >>>>>>>>>>>>> something
>>>>>>     >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
>>>>> special
>>>>>>     >>>>> object
>>>>>>     >>>>>>>>>>>>> construction
>>>>>>     >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods.
>>>>> But
>>>>>>     >>>>> here
>>>>>>     >>>>>> we
>>>>>>     >>>>>>>>> have a
>>>>>>     >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
>>>>>>     flow,
>>>>>>     >>>>> so
>>>>>>     >>>>>> I
>>>>>>     >>>>>>>>> think
>>>>>>     >>>>>>>>>>>>> this
>>>>>>     >>>>>>>>>>>>>>>>> is
>>>>>>     >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
>>>>>>     API, but I
>>>>>>     >>>>>> find
>>>>>>     >>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>>>>>>     contrasts the
>>>>>>     >>>>>>> fluency
>>>>>>     >>>>>>>>> of
>>>>>>     >>>>>>>>>>>>> other
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
>>>>>>     just call
>>>>>>     >>>>> a
>>>>>>     >>>>>>>>> method on
>>>>>>     >>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
>>>>>>     cases
>>>>>>     >>>>> are
>>>>>>     >>>>>>>>> defined
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
>>>>>>     is very
>>>>>>     >>>>>> nice
>>>>>>     >>>>>>>>> and the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>     >>>>>>>>>>>>>>>>>>>>> way
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
>>>>>>     how we
>>>>>>     >>>>>>> specify
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>>> source
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>> this::handle1)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>> this::handle2)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>>>     >>>>>>>> KStreamBrancher
>>>>>>     >>>>>>>>> or
>>>>>>     >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>     terminated by
>>>>>>     >>>>>>>>>>>>> defaultBranch()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>     incompatible with
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> current
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
>>>>>>     different
>>>>>>     >>>>>> name,
>>>>>>     >>>>>>>> but
>>>>>>     >>>>>>>>> that
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>>>>     >>>>>> something
>>>>>>     >>>>>>>> like
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>     >>>>>>>>>>>>>>>>>>>>> or
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
>>>>>>     KIP?  It
>>>>>>     >>>>>> seems
>>>>>>     >>>>>>>>> like it
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
>>>>>>     while also
>>>>>>     >>>>>>>> allowing
>>>>>>     >>>>>>>>> you
>>>>>>     >>>>>>>>>>>>> to
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>     KBranchedStreams
>>>>>>     >>>>>> if
>>>>>>     >>>>>>>>> desired.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>     >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
>>>>>>     ks){
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>>>>>>     String> ks){
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>     this::handleFirstCase)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>     this::handleSecondCase)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>> KafkaStreamsBrancher
>>>>>>     >>>>> takes a
>>>>>>     >>>>>>>>> Consumer
>>>>>>     >>>>>>>>>>>>> as a
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
>>>>>>     example in
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>> KIP
>>>>>>     >>>>>>>>>>>>> shows
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>>     >>>>>>>>> (KafkaStreams#to()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would
>>>>> we
>>>>>>     >>>>> handle
>>>>>>     >>>>>>> the
>>>>>>     >>>>>>>>> case
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to
>>>>> continue
>>>>>>     >>>>>>>> processing
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>> not
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the
>>>>> branched
>>>>>>     >>>>>> stream
>>>>>>     >>>>>>>>>>>>> immediately?
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
>>>>>>     we had
>>>>>>     >>>>>>>> something
>>>>>>     >>>>>>>>> like
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck
>>>>> <
>>>>>>     >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for
>>>>> KIP-
>>>>>>     >>>>> 418.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
>>>>> KIP-418.
>>>>>>     >>>>> Please
>>>>>>     >>>>>>>> take
>>>>>>     >>>>>>>>> a
>>>>>>     >>>>>>>>>>>>> look
>>>>>>     >>>>>>>>>>>>>>>>> at
>>>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
>>>>>>     feedback :)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>     >>>>>
>>>>>>
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>     >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>     >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >
>>>>>>
>>>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <ma...@confluent.io>.
What is the status of this KIP?

-Matthias

On 7/11/19 4:00 PM, Matthias J. Sax wrote:
> Ivan,
> 
> did you see my last reply? What do you think about my proposal to mix
> both approaches and try to get best-of-both worlds?
> 
> 
> -Matthias
> 
> On 6/11/19 3:56 PM, Matthias J. Sax wrote:
>> Thanks for the input John!
>>
>>> under your suggestion, it seems that the name is required
>>
>> If you want to get the `KStream` as part of the `Map` back using a
>> `Function`, yes. If you follow the "embedded chaining" pattern using a
>> `Consumer`, no.
>>
>> Allowing for a default name via `split()` can of course be done.
>> Similarly, using `Named` instead of `String` is possible.
>>
>> I wanted to sketch out a high level proposal to merge both patterns
>> only. Your suggestions to align the new API with the existing API make
>> totally sense.
>>
>>
>>
>> One follow up question: Would `Named` be optional or required in
>> `split()` and `branch()`? It's unclear from your example.
>>
>> If both are mandatory, what do we gain by it? The returned `Map` only
>> contains the corresponding branches, so why should we prefix all of
>> them? If only `Named` is mandatory in `branch()`, but optional in
>> `split()`, the same question raises?
>>
>> Requiring `Named` in `split()` seems only to make sense, if `Named` is
>> optional in `branch()` and we generate `-X` suffix using a counter for
>> different branch name. However, this might lead to the problem of
>> changing names if branches are added/removed. Also, how would the names
>> be generated if `Consumer` is mixed in (ie, not all branches are
>> returned in the `Map`).
>>
>> If `Named` is optional for both, it could happen that a user misses to
>> specify a name for a branch what would lead to runtime issues.
>>
>>
>> Hence, I am actually in favor to not allow a default name but keep
>> `split()` without parameter and make `Named` in `branch()` required if a
>> `Function` is used. This makes it explicit to the user that specifying a
>> name is required if a `Function` is used.
>>
>>
>>
>> About
>>
>>> KBranchedStream#branch(BranchConfig)
>>
>> I don't think that the branching predicate is a configuration and hence
>> would not include it in a configuration object.
>>
>>>     withChain(...);
>>
>> Similar, `withChain()` (that would only take a `Consumer`?) does not
>> seem to be a configuration. We can also not prevent a user to call
>> `withName()` in combination of `withChain()` what does not make sense
>> IMHO. We could of course throw an RTE but not have a compile time check
>> seems less appealing. Also, it could happen that neither `withChain()`
>> not `withName()` is called and the branch is missing in the returned
>> `Map` what lead to runtime issues, too.
>>
>> Hence, I don't think that we should add `BranchConfig`. A config object
>> is helpful if each configuration can be set independently of all others,
>> but this seems not to be the case here. If we add new configuration
>> later, we can also just move forward by deprecating the methods that
>> accept `Named` and add new methods that accepted `BranchConfig` (that
>> would of course implement `Named`).
>>
>>
>> Thoughts?
>>
>>
>> @Ivan, what do you think about the general idea to blend the two main
>> approaches of returning a `Map` plus an "embedded chaining"?
>>
>>
>>
>> -Matthias
>>
>>
>>
>> On 6/4/19 10:33 AM, John Roesler wrote:
>>> Thanks for the idea, Matthias, it does seem like this would satisfy
>>> everyone. Returning the map from the terminal operations also solves
>>> the problem of merging/joining the branched streams, if we want to add
>>> support for the compliment later on.
>>>
>>> Under your suggestion, it seems that the name is required. Otherwise,
>>> we wouldn't have keys for the map to return. I this this is actually
>>> not too bad, since experience has taught us that, although names for
>>> operations are not required to define stream processing logic, it does
>>> significantly improve the operational experience when you can map the
>>> topology, logs, metrics, etc. back to the source code. Since you
>>> wouldn't (have to) reference the name to chain extra processing onto
>>> the branch (thanks to the second argument), you can avoid the
>>> "unchecked name" problem that Ivan pointed out.
>>>
>>> In the current implementation of Branch, you can name the branch
>>> operator itself, and then all the branches get index-suffixed names
>>> built from the branch operator name. I guess under this proposal, we
>>> could naturally append the branch name to the branching operator name,
>>> like this:
>>>
>>>    stream.split(Named.withName("mysplit")) //creates node "mysplit"
>>>               .branch(..., ..., "abranch") // creates node "mysplit-abranch"
>>>               .defaultBranch(...) // creates node "mysplit-default"
>>>
>>> It does make me wonder about the DSL syntax itself, though.
>>>
>>> We don't have a defined grammar, so there's plenty of room to debate
>>> the "best" syntax in the context of each operation, but in general,
>>> the KStream DSL operators follow this pattern:
>>>
>>>     operator(function, config_object?) OR operator(config_object)
>>>
>>> where config_object is often just Named in the "function" variant.
>>> Even when the config_object isn't a Named, but some other config
>>> class, that config class _always_ implements NamedOperation.
>>>
>>> Here, we're introducing a totally different pattern:
>>>
>>>   operator(function, function, string)
>>>
>>> where the string is the name.
>>> My first question is whether the name should instead be specified with
>>> the NamedOperation interface.
>>>
>>> My second question is whether we should just roll all these arguments
>>> up into a config object like:
>>>
>>>    KBranchedStream#branch(BranchConfig)
>>>
>>>    interface BranchConfig extends NamedOperation {
>>>     withPredicate(...);
>>>     withChain(...);
>>>     withName(...);
>>>   }
>>>
>>> Although I guess we'd like to call BranchConfig something more like
>>> "Branched", even if I don't particularly like that pattern.
>>>
>>> This makes the source code a little noisier, but it also makes us more
>>> future-proof, as we can deal with a wide range of alternatives purely
>>> in the config interface, and never have to deal with adding overloads
>>> to the KBranchedStream if/when we decide we want the name to be
>>> optional, or the KStream->KStream to be optional.
>>>
>>> WDYT?
>>>
>>> Thanks,
>>> -John
>>>
>>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>>> <mi...@confluent.io> wrote:
>>>>
>>>> Matthias: I think that's pretty reasonable from my point of view. Good
>>>> suggestion.
>>>>
>>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>>
>>>>> Interesting discussion.
>>>>>
>>>>> I am wondering, if we cannot unify the advantage of both approaches:
>>>>>
>>>>>
>>>>>
>>>>> KStream#split() -> KBranchedStream
>>>>>
>>>>> // branch is not easily accessible in current scope
>>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>>   -> KBranchedStream
>>>>>
>>>>> // assign a name to the branch and
>>>>> // return the sub-stream to the current scope later
>>>>> //
>>>>> // can be simple as `#branch(p, s->s, "name")`
>>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>>   -> KBranchedStream
>>>>>
>>>>> // default branch is not easily accessible
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>>   -> Map<String,KStream>
>>>>>
>>>>> // assign custom name to default-branch
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>>   -> Map<String,KStream>
>>>>>
>>>>> // assign a default name for default
>>>>> // return map of all named sub-stream into current scope
>>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>>   -> Map<String,KStream>
>>>>>
>>>>> // return map of all names sub-stream into current scope
>>>>> KBranchedStream#noDefaultBranch()
>>>>>   -> Map<String,KStream>
>>>>>
>>>>>
>>>>>
>>>>> Hence, for each sub-stream, the user can pick to add a name and return
>>>>> the branch "result" to the calling scope or not. The implementation can
>>>>> also check at runtime that all returned names are unique. The returned
>>>>> Map can be empty and it's also optional to use the Map.
>>>>>
>>>>> To me, it seems like a good way to get best of both worlds.
>>>>>
>>>>> Thoughts?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>>> Ivan,
>>>>>>
>>>>>> That's a very good point about the "start" operator in the dynamic case.
>>>>>> I had no problem with "split()"; I was just questioning the necessity.
>>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>>> "split()" start operator. Thanks!
>>>>>>
>>>>>> Separately, I'm interested to see where the present discussion leads.
>>>>>> I've written enough Javascript code in my life to be suspicious of
>>>>>> nested closures. You have a good point about using method references (or
>>>>>> indeed function literals also work). It should be validating that this
>>>>>> was also the JS community's first approach to flattening the logic when
>>>>>> their nested closure situation got out of hand. Unfortunately, it's
>>>>>> replacing nesting with redirection, both of which disrupt code
>>>>>> readability (but in different ways for different reasons). In other
>>>>>> words, I agree that function references is *the* first-order solution if
>>>>>> the nested code does indeed become a problem.
>>>>>>
>>>>>> However, the history of JS also tells us that function references aren't
>>>>>> the end of the story either, and you can see that by observing that
>>>>>> there have been two follow-on eras, as they continue trying to cope with
>>>>>> the consequences of living in such a callback-heavy language. First, you
>>>>>> have Futures/Promises, which essentially let you convert nested code to
>>>>>> method-chained code (Observables/FP is a popular variation on this).
>>>>>> Most lately, you have async/await, which is an effort to apply language
>>>>>> (not just API) syntax to the problem, and offer the "flattest" possible
>>>>>> programming style to solve the problem (because you get back to just one
>>>>>> code block per functional unit).
>>>>>>
>>>>>> Stream-processing is a different domain, and Java+KStreams is nowhere
>>>>>> near as callback heavy as JS, so I don't think we have to take the JS
>>>>>> story for granted, but then again, I think we can derive some valuable
>>>>>> lessons by looking sideways to adjacent domains. I'm just bringing this
>>>>>> up to inspire further/deeper discussion. At the same time, just like JS,
>>>>>> we can afford to take an iterative approach to the problem.
>>>>>>
>>>>>> Separately again, I'm interested in the post-branch merge (and I'd also
>>>>>> add join) problem that Paul brought up. We can clearly punt on it, by
>>>>>> terminating the nested branches with sink operators. But is there a DSL
>>>>>> way to do it?
>>>>>>
>>>>>> Thanks again for your driving this,
>>>>>> -John
>>>>>>
>>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>>> <ma...@gmail.com>> wrote:
>>>>>>
>>>>>>     Ivan, I’ll definitely forfeit my point on the clumsiness of the
>>>>>>     branch(predicate, consumer) solution, I don’t see any real drawbacks
>>>>>>     for the dynamic case.
>>>>>>
>>>>>>     IMO the one trade off to consider at this point is the scope
>>>>>>     question. I don’t know if I totally agree that “we rarely need them
>>>>>>     in the same scope” since merging the branches back together later
>>>>>>     seems like a perfectly plausible use case that can be a lot nicer
>>>>>>     when the branched streams are in the same scope. That being said,
>>>>>>     for the reasons Ivan listed, I think it is overall the better
>>>>>>     solution - working around the scope thing is easy enough if you need
>>>>>>     to.
>>>>>>
>>>>>>     > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>>     <ip...@mail.ru.invalid> wrote:
>>>>>>     >
>>>>>>     > Hello everyone, thank you all for joining the discussion!
>>>>>>     >
>>>>>>     > Well, I don't think the idea of named branches, be it a
>>>>>>     LinkedHashMap (no other Map will do, because order of definition
>>>>>>     matters) or `branch` method  taking name and Consumer has more
>>>>>>     advantages than drawbacks.
>>>>>>     >
>>>>>>     > In my opinion, the only real positive outcome from Michael's
>>>>>>     proposal is that all the returned branches are in the same scope.
>>>>>>     But 1) we rarely need them in the same scope 2) there is a
>>>>>>     workaround for the scope problem, described in the KIP.
>>>>>>     >
>>>>>>     > 'Inlining the complex logic' is not a problem, because we can use
>>>>>>     method references instead of lambdas. In real world scenarios you
>>>>>>     tend to split the complex logic to methods anyway, so the code is
>>>>>>     going to be clean.
>>>>>>     >
>>>>>>     > The drawbacks are strong. The cohesion between predicates and
>>>>>>     handlers is lost. We have to define predicates in one place, and
>>>>>>     handlers in another. This opens the door for bugs:
>>>>>>     >
>>>>>>     > - what if we forget to define a handler for a name? or a name for
>>>>>>     a handler?
>>>>>>     > - what if we misspell a name?
>>>>>>     > - what if we copy-paste and duplicate a name?
>>>>>>     >
>>>>>>     > What Michael propose would have been totally OK if we had been
>>>>>>     writing the API in Lua, Ruby or Python. In those languages the
>>>>>>     "dynamic naming" approach would have looked most concise and
>>>>>>     beautiful. But in Java we expect all the problems related to
>>>>>>     identifiers to be eliminated in compile time.
>>>>>>     >
>>>>>>     > Do we have to invent duck-typing for the Java API?
>>>>>>     >
>>>>>>     > And if we do, what advantage are we supposed to get besides having
>>>>>>     all the branches in the same scope? Michael, maybe I'm missing your
>>>>>>     point?
>>>>>>     >
>>>>>>     > ---
>>>>>>     >
>>>>>>     > Earlier in this discussion John Roesler also proposed to do
>>>>>>     without "start branching" operator, and later Paul mentioned that in
>>>>>>     the case when we have to add a dynamic number of branches, the
>>>>>>     current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
>>>>>>     me address both comments here.
>>>>>>     >
>>>>>>     > 1) "Start branching" operator (I think that *split* is a good name
>>>>>>     for it indeed) is critical when we need to do a dynamic branching,
>>>>>>     see example below.
>>>>>>     >
>>>>>>     > 2) No, dynamic branching in current KIP is not clumsy at all.
>>>>>>     Imagine a real-world scenario when you need one branch per enum
>>>>>>     value (say, RecordType). You can have something like this:
>>>>>>     >
>>>>>>     > /*John:if we had to start with stream.branch(...) here, it would
>>>>>>     have been much messier.*/
>>>>>>     > KBranchedStream branched = stream.split();
>>>>>>     >
>>>>>>     > /*Not clumsy at all :-)*/
>>>>>>     > for (RecordType recordType : RecordType.values())
>>>>>>     >             branched = branched.branch((k, v) -> v.getRecType() ==
>>>>>>     recordType,
>>>>>>     >                     recordType::processRecords);
>>>>>>     >
>>>>>>     > Regards,
>>>>>>     >
>>>>>>     > Ivan
>>>>>>     >
>>>>>>     >
>>>>>>     > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>>     >> I also agree with Michael's observation about the core problem of
>>>>>>     >> current `branch()` implementation.
>>>>>>     >>
>>>>>>     >> However, I also don't like to pass in a clumsy Map object. My
>>>>>>     thinking
>>>>>>     >> was more aligned with Paul's proposal to just add a name to each
>>>>>>     >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>>     >>
>>>>>>     >> It makes the code easier to read, and also make the order of
>>>>>>     >> `Predicates` (that is essential) easier to grasp.
>>>>>>     >>
>>>>>>     >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>     >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>     >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>     >>>>>>    .defaultBranch("defaultBranch");
>>>>>>     >> An open question is the case for which no defaultBranch() should
>>>>> be
>>>>>>     >> specified. Atm, `split()` and `branch()` would return
>>>>>>     `BranchedKStream`
>>>>>>     >> and the call to `defaultBranch()` that returns the `Map` is
>>>>> mandatory
>>>>>>     >> (what is not the case atm). Or is this actually not a real
>>>>> problem,
>>>>>>     >> because users can just ignore the branch returned by
>>>>>>     `defaultBranch()`
>>>>>>     >> in the result `Map` ?
>>>>>>     >>
>>>>>>     >>
>>>>>>     >> About "inlining": So far, it seems to be a matter of personal
>>>>>>     >> preference. I can see arguments for both, but no "killer
>>>>>>     argument" yet
>>>>>>     >> that clearly make the case for one or the other.
>>>>>>     >>
>>>>>>     >>
>>>>>>     >> -Matthias
>>>>>>     >>
>>>>>>     >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>>     >>> Perhaps inlining is the wrong terminology. It doesn’t require
>>>>>>     that a lambda with the full downstream topology be defined inline -
>>>>>>     it can be a method reference as with Ivan’s original suggestion.
>>>>>>     The advantage of putting the predicate and its downstream logic
>>>>>>     (Consumer) together in branch() is that they are required to be near
>>>>>>     to each other.
>>>>>>     >>>
>>>>>>     >>> Ultimately the downstream code has to live somewhere, and deep
>>>>>>     branch trees will be hard to read regardless.
>>>>>>     >>>
>>>>>>     >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>>     <michael.drogalis@confluent.io
>>>>>>     <ma...@confluent.io>> wrote:
>>>>>>     >>>>
>>>>>>     >>>> I'm less enthusiastic about inlining the branch logic with its
>>>>>>     downstream
>>>>>>     >>>> functionality. Programs that have deep branch trees will
>>>>>>     quickly become
>>>>>>     >>>> harder to read as a single unit.
>>>>>>     >>>>
>>>>>>     >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>>     <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>     >>>>>
>>>>>>     >>>>> Also +1 on the issues/goals as Michael outlined them, I think
>>>>>>     that sets a
>>>>>>     >>>>> great framework for the discussion.
>>>>>>     >>>>>
>>>>>>     >>>>> Regarding the SortedMap solution, my understanding is that the
>>>>>>     current
>>>>>>     >>>>> proposal in the KIP is what is in my PR which (pending naming
>>>>>>     decisions) is
>>>>>>     >>>>> roughly this:
>>>>>>     >>>>>
>>>>>>     >>>>> stream.split()
>>>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>>     >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>>     >>>>>
>>>>>>     >>>>> Obviously some ordering is necessary, since branching as a
>>>>>>     construct
>>>>>>     >>>>> doesn't work without it, but this solution seems like it
>>>>>>     provides as much
>>>>>>     >>>>> associativity as the SortedMap solution, because each branch()
>>>>>>     call
>>>>>>     >>>>> directly associates the "conditional" with the "code block."
>>>>>>     The value it
>>>>>>     >>>>> provides over the KIP solution is the accessing of streams in
>>>>>>     the same
>>>>>>     >>>>> scope.
>>>>>>     >>>>>
>>>>>>     >>>>> The KIP solution is less "dynamic" than the SortedMap solution
>>>>>>     in the sense
>>>>>>     >>>>> that it is slightly clumsier to add a dynamic number of
>>>>>>     branches, but it is
>>>>>>     >>>>> certainly possible.  It seems to me like the API should favor
>>>>>>     the "static"
>>>>>>     >>>>> case anyway, and should make it simple and readable to
>>>>>>     fluently declare and
>>>>>>     >>>>> access your branches in-line.  It also makes it impossible to
>>>>>>     ignore a
>>>>>>     >>>>> branch, and it is possible to build an (almost) identical
>>>>>>     SortedMap
>>>>>>     >>>>> solution on top of it.
>>>>>>     >>>>>
>>>>>>     >>>>> I could also see a middle ground where instead of a raw
>>>>>>     SortedMap being
>>>>>>     >>>>> taken in, branch() takes a name and not a Consumer.  Something
>>>>>>     like this:
>>>>>>     >>>>>
>>>>>>     >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>     >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>     >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>     >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>>>>     >>>>>
>>>>>>     >>>>> Pros for that solution:
>>>>>>     >>>>> - accessing branched KStreams in same scope
>>>>>>     >>>>> - no double brace initialization, hopefully slightly more
>>>>>>     readable than
>>>>>>     >>>>> SortedMap
>>>>>>     >>>>>
>>>>>>     >>>>> Cons
>>>>>>     >>>>> - downstream branch logic cannot be specified inline which
>>>>>>     makes it harder
>>>>>>     >>>>> to read top to bottom (like existing API and SortedMap, but
>>>>>>     unlike the KIP)
>>>>>>     >>>>> - you can forget to "handle" one of the branched streams (like
>>>>>>     existing
>>>>>>     >>>>> API and SortedMap, but unlike the KIP)
>>>>>>     >>>>>
>>>>>>     >>>>> (KBranchedStreams could even work *both* ways but perhaps
>>>>>>     that's overdoing
>>>>>>     >>>>> it).
>>>>>>     >>>>>
>>>>>>     >>>>> Overall I'm curious how important it is to be able to easily
>>>>>>     access the
>>>>>>     >>>>> branched KStream in the same scope as the original.  It's
>>>>>>     possible that it
>>>>>>     >>>>> doesn't need to be handled directly by the API, but instead
>>>>>>     left up to the
>>>>>>     >>>>> user.  I'm sort of in the middle on it.
>>>>>>     >>>>>
>>>>>>     >>>>> Paul
>>>>>>     >>>>>
>>>>>>     >>>>>
>>>>>>     >>>>>
>>>>>>     >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>>     <sophie@confluent.io <ma...@confluent.io>>
>>>>>>     >>>>> wrote:
>>>>>>     >>>>>
>>>>>>     >>>>>> I'd like to +1 what Michael said about the issues with the
>>>>>>     existing
>>>>>>     >>>>> branch
>>>>>>     >>>>>> method, I agree with what he's outlined and I think we should
>>>>>>     proceed by
>>>>>>     >>>>>> trying to alleviate these problems. Specifically it seems
>>>>>>     important to be
>>>>>>     >>>>>> able to cleanly access the individual branches (eg by mapping
>>>>>>     >>>>>> name->stream), which I thought was the original intention of
>>>>>>     this KIP.
>>>>>>     >>>>>>
>>>>>>     >>>>>> That said, I don't think we should so easily give in to the
>>>>>>     double brace
>>>>>>     >>>>>> anti-pattern or force ours users into it if at all possible to
>>>>>>     >>>>> avoid...just
>>>>>>     >>>>>> my two cents.
>>>>>>     >>>>>>
>>>>>>     >>>>>> Cheers,
>>>>>>     >>>>>> Sophie
>>>>>>     >>>>>>
>>>>>>     >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>>     >>>>>> michael.drogalis@confluent.io
>>>>>>     <ma...@confluent.io>> wrote:
>>>>>>     >>>>>>
>>>>>>     >>>>>>> I’d like to propose a different way of thinking about this.
>>>>>>     To me,
>>>>>>     >>>>> there
>>>>>>     >>>>>>> are three problems with the existing branch signature:
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> 1. If you use it the way most people do, Java raises unsafe
>>>>> type
>>>>>>     >>>>>> warnings.
>>>>>>     >>>>>>> 2. The way in which you use the stream branches is
>>>>>>     positionally coupled
>>>>>>     >>>>>> to
>>>>>>     >>>>>>> the ordering of the conditionals.
>>>>>>     >>>>>>> 3. It is brittle to extend existing branch calls with
>>>>>>     additional code
>>>>>>     >>>>>>> paths.
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> Using associative constructs instead of relying on ordered
>>>>>>     constructs
>>>>>>     >>>>>> would
>>>>>>     >>>>>>> be a stronger approach. Consider a signature that instead
>>>>>>     looks like
>>>>>>     >>>>>> this:
>>>>>>     >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
>>>>>>     Predicate<?
>>>>>>     >>>>>>> super K,? super V>>);
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> Branches are given names in a map, and as a result, the API
>>>>>>     returns a
>>>>>>     >>>>>>> mapping of names to streams. The ordering of the
>>>>> conditionals is
>>>>>>     >>>>>> maintained
>>>>>>     >>>>>>> because it’s a sorted map. Insert order determines the order
>>>>> of
>>>>>>     >>>>>> evaluation.
>>>>>>     >>>>>>> This solves problem 1 because there are no more varargs. It
>>>>>>     solves
>>>>>>     >>>>>> problem
>>>>>>     >>>>>>> 2 because you no longer lean on ordering to access the
>>>>>>     branch you’re
>>>>>>     >>>>>>> interested in. It solves problem 3 because you can introduce
>>>>>>     another
>>>>>>     >>>>>>> conditional by simply attaching another name to the
>>>>>>     structure, rather
>>>>>>     >>>>>> than
>>>>>>     >>>>>>> messing with the existing indices.
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> One of the drawbacks is that creating the map inline is
>>>>>>     historically
>>>>>>     >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>>     voluminously, but
>>>>>>     >>>>>>> double brace initialization would clean up the aesthetics.
>>>>>>     >>>>>>>
>>>>>>     >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>>     <john@confluent.io <ma...@confluent.io>>
>>>>>>     >>>>> wrote:
>>>>>>     >>>>>>>> Hi Ivan,
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Thanks for the update.
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> FWIW, I agree with Matthias that the current "start
>>>>> branching"
>>>>>>     >>>>> operator
>>>>>>     >>>>>>> is
>>>>>>     >>>>>>>> confusing when named the same way as the actual branches.
>>>>>>     "Split"
>>>>>>     >>>>> seems
>>>>>>     >>>>>>>> like a good name. Alternatively, we can do without a "start
>>>>>>     >>>>> branching"
>>>>>>     >>>>>>>> operator at all, and just do:
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> stream
>>>>>>     >>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>      .defaultBranch();
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Tentatively, I think that this branching operation should be
>>>>>>     >>>>> terminal.
>>>>>>     >>>>>>> That
>>>>>>     >>>>>>>> way, we don't create ambiguity about how to use it. That
>>>>>>     is, `branch`
>>>>>>     >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
>>>>>>     `void`, to
>>>>>>     >>>>>>>> enforce that it comes last, and that there is only one
>>>>>>     definition of
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>> default branch. Potentially, we should log a warning if
>>>>>>     there's no
>>>>>>     >>>>>>> default,
>>>>>>     >>>>>>>> and additionally log a warning (or throw an exception) if a
>>>>>>     record
>>>>>>     >>>>>> falls
>>>>>>     >>>>>>>> though with no default.
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Thoughts?
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> Thanks,
>>>>>>     >>>>>>>> -John
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>>     >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>>     >>>>>>>> wrote:
>>>>>>     >>>>>>>>
>>>>>>     >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>> this is to make the name similar to String#split
>>>>>>     >>>>>>>>>>> that also returns an array, right?
>>>>>>     >>>>>>>>> The intend was to avoid name duplication. The return type
>>>>>>     should
>>>>>>     >>>>>> _not_
>>>>>>     >>>>>>>>> be an array.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> The current proposal is
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> stream.branch()
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .defaultBranch();
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>>     `branch()` does
>>>>>>     >>>>> not
>>>>>>     >>>>>>>>> take any parameters and has different semantics than the
>>>>> later
>>>>>>     >>>>>>>>> `branch()` calls. Note, that from the code snippet above,
>>>>> it's
>>>>>>     >>>>> hidden
>>>>>>     >>>>>>>>> that the first call is `KStream#branch()` while the others
>>>>> are
>>>>>>     >>>>>>>>> `KBranchedStream#branch()` what makes reading the code
>>>>> harder.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
>>>>>>     I though
>>>>>>     >>>>>> it
>>>>>>     >>>>>>>>> might be better to also rename `KStream#branch()` to avoid
>>>>> the
>>>>>>     >>>>> naming
>>>>>>     >>>>>>>>> overlap that seems to be confusing. The following reads
>>>>> much
>>>>>>     >>>>> cleaner
>>>>>>     >>>>>> to
>>>>>>     >>>>>>>> me:
>>>>>>     >>>>>>>>> stream.split()
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>>     >>>>>>>>>      .defaultBranch();
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Maybe there is a better alternative to `split()` though to
>>>>>>     avoid
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> naming overlap.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately
>>>>> we
>>>>>>     >>>>> cannot
>>>>>>     >>>>>>> have
>>>>>>     >>>>>>>>> a method with such name :-)
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
>>>>>>     with a
>>>>>>     >>>>> short
>>>>>>     >>>>>>>> name?
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
>>>>>>     with all
>>>>>>     >>>>> it's
>>>>>>     >>>>>>>>> methods? It will be part of public API and should be
>>>>>>     contained in
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>>     >>>>>>>>> `defaultBranch()` is.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> You did not comment on the idea to add a
>>>>>>     `KBranchedStream#get(int
>>>>>>     >>>>>>> index)
>>>>>>     >>>>>>>>> -> KStream` method to get the individually
>>>>>>     branched-KStreams. Would
>>>>>>     >>>>>> be
>>>>>>     >>>>>>>>> nice to get your feedback about it. It seems you suggest
>>>>>>     that users
>>>>>>     >>>>>>>>> would need to write custom utility code otherwise, to
>>>>>>     access them.
>>>>>>     >>>>> We
>>>>>>     >>>>>>>>> should discuss the pros and cons of both approaches. It
>>>>> feels
>>>>>>     >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
>>>>>>     to get
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> branched-KStreams directly.
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>> -Matthias
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>     >>>>>>>>>> Hi all!
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Matthias, thanks for your comment!
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>     >>>>>>>>>> I can see your point: this is to make the name similar to
>>>>>>     >>>>>>> String#split
>>>>>>     >>>>>>>>>> that also returns an array, right? But is it worth the
>>>>>>     loss of
>>>>>>     >>>>>>>> backwards
>>>>>>     >>>>>>>>>> compatibility? We can have overloaded branch() as well
>>>>>>     without
>>>>>>     >>>>>>>> affecting
>>>>>>     >>>>>>>>>> the existing code. Maybe the old array-based `branch`
>>>>> method
>>>>>>     >>>>> should
>>>>>>     >>>>>>> be
>>>>>>     >>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>>     >>>>> BranchingKStream#branch(),
>>>>>>     >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>> BranchingKStream#default()
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default'
>>>>> is,
>>>>>>     >>>>>>> however, a
>>>>>>     >>>>>>>>>> reserved word, so unfortunately we cannot have a method
>>>>>>     with such
>>>>>>     >>>>>>> name
>>>>>>     >>>>>>>>> :-)
>>>>>>     >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument,
>>>>> but I
>>>>>>     >>>>> think
>>>>>>     >>>>>>> that
>>>>>>     >>>>>>>>>> is not required?
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Absolutely! I think that was just copy-paste error or
>>>>>>     something.
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Dear colleagues,
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>>     >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Any new suggestions/objections?
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>>
>>>>>>     >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>     >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems
>>>>> that
>>>>>>     >>>>>>> everybody
>>>>>>     >>>>>>>>>>> agrees that the current branch() method using arrays is
>>>>> not
>>>>>>     >>>>>> optimal.
>>>>>>     >>>>>>>>>>> I had a quick look into the PR and I like the overall
>>>>>>     proposal.
>>>>>>     >>>>>>> There
>>>>>>     >>>>>>>>>>> are some minor things we need to consider. I would
>>>>>>     recommend the
>>>>>>     >>>>>>>>>>> following renaming:
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> KStream#branch() -> #split()
>>>>>>     >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>>     >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>>     BranchingKStream#default()
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> It's just a suggestion to get slightly shorter method
>>>>> names.
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>>     `Predicate` as
>>>>>>     >>>>>>>> argument,
>>>>>>     >>>>>>>>>>> but I think that is not required?
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> Also, we should consider KIP-307, that was recently
>>>>>>     accepted and
>>>>>>     >>>>>> is
>>>>>>     >>>>>>>>>>> currently implemented:
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>
>>>>>>
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>     >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
>>>>>>     parameter.
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> For the issue that the created `KStream` object are in
>>>>>>     different
>>>>>>     >>>>>>>> scopes:
>>>>>>     >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>>>>>>     index)` method
>>>>>>     >>>>>>> that
>>>>>>     >>>>>>>>>>> returns the corresponding "branched" result `KStream`
>>>>>>     object?
>>>>>>     >>>>>> Maybe,
>>>>>>     >>>>>>>> the
>>>>>>     >>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>>>     >>>>>> `Consumer<KStream>`
>>>>>>     >>>>>>>> but
>>>>>>     >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
>>>>>>     whatever
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>>>>> `Function` returns?
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
>>>>>>     current
>>>>>>     >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>> -Matthias
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>     >>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
>>>>>>     makes sense
>>>>>>     >>>>>> for
>>>>>>     >>>>>>>> you
>>>>>>     >>>>>>>>> to
>>>>>>     >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
>>>>>>     we'll
>>>>>>     >>>>> need
>>>>>>     >>>>>>>> some
>>>>>>     >>>>>>>>>>>> buy-in from committers that have actual binding votes on
>>>>>>     >>>>> whether
>>>>>>     >>>>>>> the
>>>>>>     >>>>>>>>> KIP
>>>>>>     >>>>>>>>>>>> could be adopted.  It would be great to hear if they
>>>>>>     think this
>>>>>>     >>>>>> is
>>>>>>     >>>>>>> a
>>>>>>     >>>>>>>>> good
>>>>>>     >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
>>>>>>     starting a
>>>>>>     >>>>>>> vote,
>>>>>>     >>>>>>>>> or if
>>>>>>     >>>>>>>>>>>> there is generally some indication of interest
>>>>> beforehand.
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
>>>>>>     assuming
>>>>>>     >>>>> we
>>>>>>     >>>>>> do
>>>>>>     >>>>>>>>> move
>>>>>>     >>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>>>     >>>>>> KBranchedStream",
>>>>>>     >>>>>>> do
>>>>>>     >>>>>>>>> we
>>>>>>     >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I
>>>>> would
>>>>>>     >>>>> favor
>>>>>>     >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs
>>>>> that
>>>>>>     >>>>>>> accomplish
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>> same thing is confusing, especially when they're fairly
>>>>>>     similar
>>>>>>     >>>>>>>>> anyway.  We
>>>>>>     >>>>>>>>>>>> just need to be sure we're not making something
>>>>>>     >>>>>>> impossible/difficult
>>>>>>     >>>>>>>>> that
>>>>>>     >>>>>>>>>>>> is currently possible/easy.
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> Regarding my PR - I think the general structure would
>>>>> work,
>>>>>>     >>>>> it's
>>>>>>     >>>>>>>> just a
>>>>>>     >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>>>     >>>>>>> particular,
>>>>>>     >>>>>>>>>>>> passing in the "predicates" and "children" lists which
>>>>> get
>>>>>>     >>>>>> modified
>>>>>>     >>>>>>>> in
>>>>>>     >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>>     KStreamLazyBranch is
>>>>>>     >>>>> a
>>>>>>     >>>>>>> bit
>>>>>>     >>>>>>>>>>>> complicated to follow.
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> Thanks,
>>>>>>     >>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>     >>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>> Hi Paul!
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>>     convinced: your
>>>>>>     >>>>>>>> proposal
>>>>>>     >>>>>>>>>>>>> looks better and should work. We just have to document
>>>>> the
>>>>>>     >>>>>> crucial
>>>>>>     >>>>>>>>> fact
>>>>>>     >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
>>>>>>     And then
>>>>>>     >>>>>> it's
>>>>>>     >>>>>>>> all
>>>>>>     >>>>>>>>>>>>> going to be very nice.
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
>>>>>>     resume the
>>>>>>     >>>>>>>>>>>>> discussion here, right?
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>>>>>     >>>>> starting
>>>>>>     >>>>>>>> point
>>>>>>     >>>>>>>>> if
>>>>>>     >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
>>>>>>     starting
>>>>>>     >>>>>>> point.
>>>>>>     >>>>>>>>> But
>>>>>>     >>>>>>>>>>>>> as a novice in this project I might miss some important
>>>>>>     >>>>> details.
>>>>>>     >>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>>>     >>>>> stream.branch()
>>>>>>     >>>>>>>>> solution
>>>>>>     >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>>>>>     >>>>> invoked
>>>>>>     >>>>>> as
>>>>>>     >>>>>>>>> they’re
>>>>>>     >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
>>>>>>     still
>>>>>>     >>>>>> ought
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>> be
>>>>>>     >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
>>>>>>     depend on
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> branched
>>>>>>     >>>>>>>>>>>>> streams having been set.
>>>>>>     >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
>>>>>>     access
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> branched
>>>>>>     >>>>>>>>>>>>> streams in the same scope as the original stream (that
>>>>>>     is, not
>>>>>>     >>>>>>>> inside
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>>>>>     >>>>> solutions.
>>>>>>     >>>>>> It
>>>>>>     >>>>>>>>> can be
>>>>>>     >>>>>>>>>>>>> worked around though.
>>>>>>     >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
>>>>>>     excited
>>>>>>     >>>>> to
>>>>>>     >>>>>>>> hear
>>>>>>     >>>>>>>>>>>>> your thoughts!]
>>>>>>     >>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>     >>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>> Hi Paul!
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>>     >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at
>>>>> first
>>>>>>     >>>>> glance,
>>>>>>     >>>>>>> but
>>>>>>     >>>>>>>>> ---
>>>>>>     >>>>>>>>>>>>>>>> the newly branched streams are not available in the
>>>>>>     same
>>>>>>     >>>>>> scope
>>>>>>     >>>>>>> as
>>>>>>     >>>>>>>>> each
>>>>>>     >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
>>>>> together
>>>>>>     >>>>> again
>>>>>>     >>>>>> I
>>>>>>     >>>>>>>>> don't see
>>>>>>     >>>>>>>>>>>>> a way to do that.
>>>>>>     >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
>>>>>>     just
>>>>>>     >>>>>> going
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>>>>>> write in details about this issue.
>>>>>>     >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
>>>>>>     we need
>>>>>>     >>>>> to
>>>>>>     >>>>>>>>> identify
>>>>>>     >>>>>>>>>>>>> customers who have bought coffee and made a purchase
>>>>>>     in the
>>>>>>     >>>>>>>>> electronics
>>>>>>     >>>>>>>>>>>>> store to give them coupons.
>>>>>>     >>>>>>>>>>>>>>> This is the code I usually write under these
>>>>>>     circumstances
>>>>>>     >>>>>> using
>>>>>>     >>>>>>>> my
>>>>>>     >>>>>>>>>>>>> 'brancher' class:
>>>>>>     >>>>>>>>>>>>>>> @Setter
>>>>>>     >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>     >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>     >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>     >>>>>>>>>>>>>>>       return
>>>>>>     >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>     >>>>>>>>>>>>>>>       /*In the real world the code here can be
>>>>>>     complex, so
>>>>>>     >>>>>>>>> creation of
>>>>>>     >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
>>>>>>     order to
>>>>>>     >>>>>>>> separate
>>>>>>     >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>     >>>>>>>>>>>>>>>  }
>>>>>>     >>>>>>>>>>>>>>> }
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>     >>>>>>>>>>>>>>>     .branch(predicate1,
>>>>> couponIssuer::setCoffePurchases)
>>>>>>     >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>>     >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>     >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
>>>>>>     everything
>>>>>>     >>>>>>>> later,
>>>>>>     >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>     >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>>     initialize the
>>>>>>     >>>>>>>>> CouponIssuer
>>>>>>     >>>>>>>>>>>>> we need the terminal operation to be called before
>>>>>>     >>>>>>>>> streamsBuilder.build()
>>>>>>     >>>>>>>>>>>>> is called.
>>>>>>     >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
>>>>>>     essentially
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>>> next
>>>>>>     >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
>>>>>>     based on
>>>>>>     >>>>> my
>>>>>>     >>>>>>>>> experience,
>>>>>>     >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>     >>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
>>>>>>     fluent
>>>>>>     >>>>> API
>>>>>>     >>>>>>>> based
>>>>>>     >>>>>>>>>>>>> off of
>>>>>>     >>>>>>>>>>>>>>>> KStream here
>>>>>>     (https://github.com/apache/kafka/pull/6512),
>>>>>>     >>>>>> and
>>>>>>     >>>>>>> I
>>>>>>     >>>>>>>>> think
>>>>>>     >>>>>>>>>>>>> I
>>>>>>     >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>     >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>>>>     >>>>>>> compatibility
>>>>>>     >>>>>>>>>>>>> issues,
>>>>>>     >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
>>>>>>     that Java
>>>>>>     >>>>> is
>>>>>>     >>>>>>>> smart
>>>>>>     >>>>>>>>>>>>> enough to
>>>>>>     >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>>>>>>     returning one
>>>>>>     >>>>>>> thing
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>> branch()
>>>>>>     >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>     >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
>>>>>>     need
>>>>>>     >>>>> it.
>>>>>>     >>>>>>> We
>>>>>>     >>>>>>>>> can
>>>>>>     >>>>>>>>>>>>> just
>>>>>>     >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
>>>>>>     shares
>>>>>>     >>>>>> its
>>>>>>     >>>>>>>>> state
>>>>>>     >>>>>>>>>>>>> with the
>>>>>>     >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
>>>>>>     branching.
>>>>>>     >>>>>>> It's
>>>>>>     >>>>>>>>> not
>>>>>>     >>>>>>>>>>>>> terribly
>>>>>>     >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>>>>>>     demonstrates
>>>>>>     >>>>>> its
>>>>>>     >>>>>>>>>>>>> feasibility.
>>>>>>     >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should
>>>>> be
>>>>>>     >>>>> final
>>>>>>     >>>>>> or
>>>>>>     >>>>>>>>> even a
>>>>>>     >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
>>>>>>     wanted to
>>>>>>     >>>>>> see
>>>>>>     >>>>>>>> how
>>>>>>     >>>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>>     >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
>>>>>>     solution
>>>>>>     >>>>>>> could
>>>>>>     >>>>>>>> be
>>>>>>     >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>>>>>     >>>>> suggested
>>>>>>     >>>>>>>> was a
>>>>>>     >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
>>>>>>     streams
>>>>>>     >>>>>> are
>>>>>>     >>>>>>>> not
>>>>>>     >>>>>>>>>>>>>>>> available in the same scope as each other.  That
>>>>>>     is, if we
>>>>>>     >>>>>>> wanted
>>>>>>     >>>>>>>>> to
>>>>>>     >>>>>>>>>>>>> merge
>>>>>>     >>>>>>>>>>>>>>>> them back together again I don't see a way to do
>>>>>>     that.  The
>>>>>>     >>>>>> KIP
>>>>>>     >>>>>>>>>>>>> proposal
>>>>>>     >>>>>>>>>>>>>>>> has the same issue, though - all this means is that
>>>>> for
>>>>>>     >>>>>> either
>>>>>>     >>>>>>>>>>>>> solution,
>>>>>>     >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
>>>>>>     table.
>>>>>>     >>>>>>>>>>>>>>>> Thanks,
>>>>>>     >>>>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>>     >>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to
>>>>> this
>>>>>>     >>>>>> point.
>>>>>>     >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
>>>>>>     branch API
>>>>>>     >>>>>>> needs
>>>>>>     >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>     >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
>>>>>>     returns
>>>>>>     >>>>>> its
>>>>>>     >>>>>>>>> argument
>>>>>>     >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code
>>>>> won't
>>>>>>     >>>>> make
>>>>>>     >>>>>>>> sense
>>>>>>     >>>>>>>>>>>>> until
>>>>>>     >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
>>>>>>     instance
>>>>>>     >>>>>>>>> contrasts the
>>>>>>     >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> stream
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>>>     >>>>>>>>> defaultBranch(..)
>>>>>>     >>>>>>>>>>>>> and
>>>>>>     >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface
>>>>> is
>>>>>>     >>>>>> defined.
>>>>>>     >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>>     >>>>>>>> (defaultBranch(ks->)
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
>>>>>>     miss the
>>>>>>     >>>>>> fact
>>>>>>     >>>>>>>>> that one
>>>>>>     >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
>>>>>>     methods
>>>>>>     >>>>>> are
>>>>>>     >>>>>>>> not
>>>>>>     >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do
>>>>> better?
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>     >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>> Paul,
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>>     >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>>>>>>     implemented the
>>>>>>     >>>>>>> easy
>>>>>>     >>>>>>>>> way.
>>>>>>     >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
>>>>> assumes
>>>>>>     >>>>> nothing
>>>>>>     >>>>>>>> will
>>>>>>     >>>>>>>>>>>>> reach
>>>>>>     >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>     >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only
>>>>> option
>>>>>>     >>>>>> besides
>>>>>>     >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
>>>>>>     want to
>>>>>>     >>>>>> just
>>>>>>     >>>>>>>>> silently
>>>>>>     >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>>     predicate. 2)
>>>>>>     >>>>>>> Throwing
>>>>>>     >>>>>>>>> an
>>>>>>     >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
>>>>>>     looks
>>>>>>     >>>>>> like a
>>>>>>     >>>>>>>> bad
>>>>>>     >>>>>>>>>>>>> idea.
>>>>>>     >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
>>>>>>     emit a
>>>>>>     >>>>>>>> special
>>>>>>     >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly
>>>>> where
>>>>>>     >>>>>>> `default`
>>>>>>     >>>>>>>>> can
>>>>>>     >>>>>>>>>>>>> be
>>>>>>     >>>>>>>>>>>>>>>>>>> used.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>     >>>>> InternalTopologyBuilder
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>> track
>>>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>     >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>>     a clear
>>>>>>     >>>>>>> error
>>>>>>     >>>>>>>>>>>>> before it
>>>>>>     >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>>>>>     >>>>> compiled
>>>>>>     >>>>>>> and
>>>>>>     >>>>>>>>> run?
>>>>>>     >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>>>>>>     compile if
>>>>>>     >>>>> used
>>>>>>     >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>>>>>>     method chain
>>>>>>     >>>>>>>> starting
>>>>>>     >>>>>>>>>>>>> from
>>>>>>     >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
>>>>>>     between
>>>>>>     >>>>>>>> runtime
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>>>>>     >>>>> instantly
>>>>>>     >>>>>> on
>>>>>>     >>>>>>>>> unit
>>>>>>     >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
>>>>>>     compilation
>>>>>>     >>>>>>>> failure.
>>>>>>     >>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
>>>>>>     required.
>>>>>>     >>>>>>> But
>>>>>>     >>>>>>>> is
>>>>>>     >>>>>>>>>>>>> that
>>>>>>     >>>>>>>>>>>>>>>>>>>> really
>>>>>>     >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>>>>     >>>>>> defaultBranch
>>>>>>     >>>>>>>>> they
>>>>>>     >>>>>>>>>>>>> can
>>>>>>     >>>>>>>>>>>>>>>>>>>> call
>>>>>>     >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
>>>>>>     just as
>>>>>>     >>>>>>>>> easily.  In
>>>>>>     >>>>>>>>>>>>>>>>>>>> fact I
>>>>>>     >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API
>>>>> - a
>>>>>>     >>>>> user
>>>>>>     >>>>>>>> could
>>>>>>     >>>>>>>>>>>>> specify
>>>>>>     >>>>>>>>>>>>>>>>> a
>>>>>>     >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach
>>>>> the
>>>>>>     >>>>>> default
>>>>>>     >>>>>>>>> branch,
>>>>>>     >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>> That
>>>>>>     >>>>> seems
>>>>>>     >>>>>>> like
>>>>>>     >>>>>>>>> an
>>>>>>     >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
>>>>>>     which allows
>>>>>>     >>>>>> for
>>>>>>     >>>>>>>> the
>>>>>>     >>>>>>>>>>>>> more
>>>>>>     >>>>>>>>>>>>>>>>>>>> subtle
>>>>>>     >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
>>>>> dropped.
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
>>>>>>     to be
>>>>>>     >>>>>> well
>>>>>>     >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>>     >>>>> InternalTopologyBuilder
>>>>>>     >>>>>>> to
>>>>>>     >>>>>>>>> track
>>>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>>>     >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>>     a clear
>>>>>>     >>>>>>> error
>>>>>>     >>>>>>>>>>>>> before it
>>>>>>     >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is
>>>>> a
>>>>>>     >>>>> "build
>>>>>>     >>>>>>>> step"
>>>>>>     >>>>>>>>>>>>> where
>>>>>>     >>>>>>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>>     >>>>>> StreamsBuilder.build()
>>>>>>     >>>>>>> is
>>>>>>     >>>>>>>>>>>>> called.
>>>>>>     >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I
>>>>> agree
>>>>>>     >>>>> that
>>>>>>     >>>>>>> it's
>>>>>>     >>>>>>>>>>>>>>>>>>>> critical to
>>>>>>     >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
>>>>>>     stream.
>>>>>>     >>>>>>> With
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>> fluent
>>>>>>     >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all
>>>>> other
>>>>>>     >>>>>>> operations
>>>>>>     >>>>>>>>> do -
>>>>>>     >>>>>>>>>>>>> if
>>>>>>     >>>>>>>>>>>>>>>>> you
>>>>>>     >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
>>>>> multiple
>>>>>>     >>>>> times,
>>>>>>     >>>>>>> you
>>>>>>     >>>>>>>>> just
>>>>>>     >>>>>>>>>>>>>>>>>>>> need the
>>>>>>     >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
>>>>>>     operations
>>>>>>     >>>>>> on
>>>>>>     >>>>>>> it
>>>>>>     >>>>>>>>> as
>>>>>>     >>>>>>>>>>>>> you
>>>>>>     >>>>>>>>>>>>>>>>>>>> desire.
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> Best,
>>>>>>     >>>>>>>>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>>     >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
>>>>>>     always need
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>>> operation we
>>>>>>     >>>>> don't
>>>>>>     >>>>>>>> know
>>>>>>     >>>>>>>>>>>>> when to
>>>>>>     >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
>>>>>>     so we
>>>>>>     >>>>> can
>>>>>>     >>>>>> do
>>>>>>     >>>>>>>>>>>>> something
>>>>>>     >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
>>>>> special
>>>>>>     >>>>> object
>>>>>>     >>>>>>>>>>>>> construction
>>>>>>     >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods.
>>>>> But
>>>>>>     >>>>> here
>>>>>>     >>>>>> we
>>>>>>     >>>>>>>>> have a
>>>>>>     >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
>>>>>>     flow,
>>>>>>     >>>>> so
>>>>>>     >>>>>> I
>>>>>>     >>>>>>>>> think
>>>>>>     >>>>>>>>>>>>> this
>>>>>>     >>>>>>>>>>>>>>>>> is
>>>>>>     >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
>>>>>>     API, but I
>>>>>>     >>>>>> find
>>>>>>     >>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>>>>>>     contrasts the
>>>>>>     >>>>>>> fluency
>>>>>>     >>>>>>>>> of
>>>>>>     >>>>>>>>>>>>> other
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
>>>>>>     just call
>>>>>>     >>>>> a
>>>>>>     >>>>>>>>> method on
>>>>>>     >>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>> stream
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
>>>>>>     cases
>>>>>>     >>>>> are
>>>>>>     >>>>>>>>> defined
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
>>>>>>     is very
>>>>>>     >>>>>> nice
>>>>>>     >>>>>>>>> and the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> right
>>>>>>     >>>>>>>>>>>>>>>>>>>>> way
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
>>>>>>     how we
>>>>>>     >>>>>>> specify
>>>>>>     >>>>>>>>> the
>>>>>>     >>>>>>>>>>>>> source
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>>> this::handle1)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>>> this::handle2)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>>>     >>>>>>>> KStreamBrancher
>>>>>>     >>>>>>>>> or
>>>>>>     >>>>>>>>>>>>>>>>>>>>> something,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>>     terminated by
>>>>>>     >>>>>>>>>>>>> defaultBranch()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>>     incompatible with
>>>>>>     >>>>> the
>>>>>>     >>>>>>>>> current
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
>>>>>>     different
>>>>>>     >>>>>> name,
>>>>>>     >>>>>>>> but
>>>>>>     >>>>>>>>> that
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>>>>     >>>>>> something
>>>>>>     >>>>>>>> like
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>     >>>>>>>>>>>>>>>>>>>>> or
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
>>>>>>     KIP?  It
>>>>>>     >>>>>> seems
>>>>>>     >>>>>>>>> like it
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
>>>>>>     while also
>>>>>>     >>>>>>>> allowing
>>>>>>     >>>>>>>>> you
>>>>>>     >>>>>>>>>>>>> to
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>>     KBranchedStreams
>>>>>>     >>>>>> if
>>>>>>     >>>>>>>>> desired.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>     >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
>>>>>>     ks){
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>>>>>>     String> ks){
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>>     this::handleFirstCase)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>>     this::handleSecondCase)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>>> KafkaStreamsBrancher
>>>>>>     >>>>> takes a
>>>>>>     >>>>>>>>> Consumer
>>>>>>     >>>>>>>>>>>>> as a
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
>>>>>>     example in
>>>>>>     >>>>>> the
>>>>>>     >>>>>>>> KIP
>>>>>>     >>>>>>>>>>>>> shows
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>>     >>>>>>>>> (KafkaStreams#to()
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would
>>>>> we
>>>>>>     >>>>> handle
>>>>>>     >>>>>>> the
>>>>>>     >>>>>>>>> case
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to
>>>>> continue
>>>>>>     >>>>>>>> processing
>>>>>>     >>>>>>>>> and
>>>>>>     >>>>>>>>>>>>> not
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the
>>>>> branched
>>>>>>     >>>>>> stream
>>>>>>     >>>>>>>>>>>>> immediately?
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
>>>>>>     we had
>>>>>>     >>>>>>>> something
>>>>>>     >>>>>>>>> like
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck
>>>>> <
>>>>>>     >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for
>>>>> KIP-
>>>>>>     >>>>> 418.
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
>>>>> KIP-418.
>>>>>>     >>>>> Please
>>>>>>     >>>>>>>> take
>>>>>>     >>>>>>>>> a
>>>>>>     >>>>>>>>>>>>> look
>>>>>>     >>>>>>>>>>>>>>>>> at
>>>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
>>>>>>     feedback :)
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>     >>>>>
>>>>>>
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>     >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>>     >>>>> https://github.com/apache/kafka/pull/6164
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>     >>>>>>>>>
>>>>>>     >
>>>>>>
>>>>>
>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Ivan,

did you see my last reply? What do you think about my proposal to mix
both approaches and try to get best-of-both worlds?


-Matthias

On 6/11/19 3:56 PM, Matthias J. Sax wrote:
> Thanks for the input John!
> 
>> under your suggestion, it seems that the name is required
> 
> If you want to get the `KStream` as part of the `Map` back using a
> `Function`, yes. If you follow the "embedded chaining" pattern using a
> `Consumer`, no.
> 
> Allowing for a default name via `split()` can of course be done.
> Similarly, using `Named` instead of `String` is possible.
> 
> I wanted to sketch out a high level proposal to merge both patterns
> only. Your suggestions to align the new API with the existing API make
> totally sense.
> 
> 
> 
> One follow up question: Would `Named` be optional or required in
> `split()` and `branch()`? It's unclear from your example.
> 
> If both are mandatory, what do we gain by it? The returned `Map` only
> contains the corresponding branches, so why should we prefix all of
> them? If only `Named` is mandatory in `branch()`, but optional in
> `split()`, the same question raises?
> 
> Requiring `Named` in `split()` seems only to make sense, if `Named` is
> optional in `branch()` and we generate `-X` suffix using a counter for
> different branch name. However, this might lead to the problem of
> changing names if branches are added/removed. Also, how would the names
> be generated if `Consumer` is mixed in (ie, not all branches are
> returned in the `Map`).
> 
> If `Named` is optional for both, it could happen that a user misses to
> specify a name for a branch what would lead to runtime issues.
> 
> 
> Hence, I am actually in favor to not allow a default name but keep
> `split()` without parameter and make `Named` in `branch()` required if a
> `Function` is used. This makes it explicit to the user that specifying a
> name is required if a `Function` is used.
> 
> 
> 
> About
> 
>> KBranchedStream#branch(BranchConfig)
> 
> I don't think that the branching predicate is a configuration and hence
> would not include it in a configuration object.
> 
>>     withChain(...);
> 
> Similar, `withChain()` (that would only take a `Consumer`?) does not
> seem to be a configuration. We can also not prevent a user to call
> `withName()` in combination of `withChain()` what does not make sense
> IMHO. We could of course throw an RTE but not have a compile time check
> seems less appealing. Also, it could happen that neither `withChain()`
> not `withName()` is called and the branch is missing in the returned
> `Map` what lead to runtime issues, too.
> 
> Hence, I don't think that we should add `BranchConfig`. A config object
> is helpful if each configuration can be set independently of all others,
> but this seems not to be the case here. If we add new configuration
> later, we can also just move forward by deprecating the methods that
> accept `Named` and add new methods that accepted `BranchConfig` (that
> would of course implement `Named`).
> 
> 
> Thoughts?
> 
> 
> @Ivan, what do you think about the general idea to blend the two main
> approaches of returning a `Map` plus an "embedded chaining"?
> 
> 
> 
> -Matthias
> 
> 
> 
> On 6/4/19 10:33 AM, John Roesler wrote:
>> Thanks for the idea, Matthias, it does seem like this would satisfy
>> everyone. Returning the map from the terminal operations also solves
>> the problem of merging/joining the branched streams, if we want to add
>> support for the compliment later on.
>>
>> Under your suggestion, it seems that the name is required. Otherwise,
>> we wouldn't have keys for the map to return. I this this is actually
>> not too bad, since experience has taught us that, although names for
>> operations are not required to define stream processing logic, it does
>> significantly improve the operational experience when you can map the
>> topology, logs, metrics, etc. back to the source code. Since you
>> wouldn't (have to) reference the name to chain extra processing onto
>> the branch (thanks to the second argument), you can avoid the
>> "unchecked name" problem that Ivan pointed out.
>>
>> In the current implementation of Branch, you can name the branch
>> operator itself, and then all the branches get index-suffixed names
>> built from the branch operator name. I guess under this proposal, we
>> could naturally append the branch name to the branching operator name,
>> like this:
>>
>>    stream.split(Named.withName("mysplit")) //creates node "mysplit"
>>               .branch(..., ..., "abranch") // creates node "mysplit-abranch"
>>               .defaultBranch(...) // creates node "mysplit-default"
>>
>> It does make me wonder about the DSL syntax itself, though.
>>
>> We don't have a defined grammar, so there's plenty of room to debate
>> the "best" syntax in the context of each operation, but in general,
>> the KStream DSL operators follow this pattern:
>>
>>     operator(function, config_object?) OR operator(config_object)
>>
>> where config_object is often just Named in the "function" variant.
>> Even when the config_object isn't a Named, but some other config
>> class, that config class _always_ implements NamedOperation.
>>
>> Here, we're introducing a totally different pattern:
>>
>>   operator(function, function, string)
>>
>> where the string is the name.
>> My first question is whether the name should instead be specified with
>> the NamedOperation interface.
>>
>> My second question is whether we should just roll all these arguments
>> up into a config object like:
>>
>>    KBranchedStream#branch(BranchConfig)
>>
>>    interface BranchConfig extends NamedOperation {
>>     withPredicate(...);
>>     withChain(...);
>>     withName(...);
>>   }
>>
>> Although I guess we'd like to call BranchConfig something more like
>> "Branched", even if I don't particularly like that pattern.
>>
>> This makes the source code a little noisier, but it also makes us more
>> future-proof, as we can deal with a wide range of alternatives purely
>> in the config interface, and never have to deal with adding overloads
>> to the KBranchedStream if/when we decide we want the name to be
>> optional, or the KStream->KStream to be optional.
>>
>> WDYT?
>>
>> Thanks,
>> -John
>>
>> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
>> <mi...@confluent.io> wrote:
>>>
>>> Matthias: I think that's pretty reasonable from my point of view. Good
>>> suggestion.
>>>
>>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>> Interesting discussion.
>>>>
>>>> I am wondering, if we cannot unify the advantage of both approaches:
>>>>
>>>>
>>>>
>>>> KStream#split() -> KBranchedStream
>>>>
>>>> // branch is not easily accessible in current scope
>>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>>   -> KBranchedStream
>>>>
>>>> // assign a name to the branch and
>>>> // return the sub-stream to the current scope later
>>>> //
>>>> // can be simple as `#branch(p, s->s, "name")`
>>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>>   -> KBranchedStream
>>>>
>>>> // default branch is not easily accessible
>>>> // return map of all named sub-stream into current scope
>>>> KBranchedStream#default(Cosumer<KStream>)
>>>>   -> Map<String,KStream>
>>>>
>>>> // assign custom name to default-branch
>>>> // return map of all named sub-stream into current scope
>>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>>   -> Map<String,KStream>
>>>>
>>>> // assign a default name for default
>>>> // return map of all named sub-stream into current scope
>>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>>   -> Map<String,KStream>
>>>>
>>>> // return map of all names sub-stream into current scope
>>>> KBranchedStream#noDefaultBranch()
>>>>   -> Map<String,KStream>
>>>>
>>>>
>>>>
>>>> Hence, for each sub-stream, the user can pick to add a name and return
>>>> the branch "result" to the calling scope or not. The implementation can
>>>> also check at runtime that all returned names are unique. The returned
>>>> Map can be empty and it's also optional to use the Map.
>>>>
>>>> To me, it seems like a good way to get best of both worlds.
>>>>
>>>> Thoughts?
>>>>
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>>
>>>>
>>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>>> Ivan,
>>>>>
>>>>> That's a very good point about the "start" operator in the dynamic case.
>>>>> I had no problem with "split()"; I was just questioning the necessity.
>>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>>> "split()" start operator. Thanks!
>>>>>
>>>>> Separately, I'm interested to see where the present discussion leads.
>>>>> I've written enough Javascript code in my life to be suspicious of
>>>>> nested closures. You have a good point about using method references (or
>>>>> indeed function literals also work). It should be validating that this
>>>>> was also the JS community's first approach to flattening the logic when
>>>>> their nested closure situation got out of hand. Unfortunately, it's
>>>>> replacing nesting with redirection, both of which disrupt code
>>>>> readability (but in different ways for different reasons). In other
>>>>> words, I agree that function references is *the* first-order solution if
>>>>> the nested code does indeed become a problem.
>>>>>
>>>>> However, the history of JS also tells us that function references aren't
>>>>> the end of the story either, and you can see that by observing that
>>>>> there have been two follow-on eras, as they continue trying to cope with
>>>>> the consequences of living in such a callback-heavy language. First, you
>>>>> have Futures/Promises, which essentially let you convert nested code to
>>>>> method-chained code (Observables/FP is a popular variation on this).
>>>>> Most lately, you have async/await, which is an effort to apply language
>>>>> (not just API) syntax to the problem, and offer the "flattest" possible
>>>>> programming style to solve the problem (because you get back to just one
>>>>> code block per functional unit).
>>>>>
>>>>> Stream-processing is a different domain, and Java+KStreams is nowhere
>>>>> near as callback heavy as JS, so I don't think we have to take the JS
>>>>> story for granted, but then again, I think we can derive some valuable
>>>>> lessons by looking sideways to adjacent domains. I'm just bringing this
>>>>> up to inspire further/deeper discussion. At the same time, just like JS,
>>>>> we can afford to take an iterative approach to the problem.
>>>>>
>>>>> Separately again, I'm interested in the post-branch merge (and I'd also
>>>>> add join) problem that Paul brought up. We can clearly punt on it, by
>>>>> terminating the nested branches with sink operators. But is there a DSL
>>>>> way to do it?
>>>>>
>>>>> Thanks again for your driving this,
>>>>> -John
>>>>>
>>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>>> <ma...@gmail.com>> wrote:
>>>>>
>>>>>     Ivan, I’ll definitely forfeit my point on the clumsiness of the
>>>>>     branch(predicate, consumer) solution, I don’t see any real drawbacks
>>>>>     for the dynamic case.
>>>>>
>>>>>     IMO the one trade off to consider at this point is the scope
>>>>>     question. I don’t know if I totally agree that “we rarely need them
>>>>>     in the same scope” since merging the branches back together later
>>>>>     seems like a perfectly plausible use case that can be a lot nicer
>>>>>     when the branched streams are in the same scope. That being said,
>>>>>     for the reasons Ivan listed, I think it is overall the better
>>>>>     solution - working around the scope thing is easy enough if you need
>>>>>     to.
>>>>>
>>>>>     > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>>     <ip...@mail.ru.invalid> wrote:
>>>>>     >
>>>>>     > Hello everyone, thank you all for joining the discussion!
>>>>>     >
>>>>>     > Well, I don't think the idea of named branches, be it a
>>>>>     LinkedHashMap (no other Map will do, because order of definition
>>>>>     matters) or `branch` method  taking name and Consumer has more
>>>>>     advantages than drawbacks.
>>>>>     >
>>>>>     > In my opinion, the only real positive outcome from Michael's
>>>>>     proposal is that all the returned branches are in the same scope.
>>>>>     But 1) we rarely need them in the same scope 2) there is a
>>>>>     workaround for the scope problem, described in the KIP.
>>>>>     >
>>>>>     > 'Inlining the complex logic' is not a problem, because we can use
>>>>>     method references instead of lambdas. In real world scenarios you
>>>>>     tend to split the complex logic to methods anyway, so the code is
>>>>>     going to be clean.
>>>>>     >
>>>>>     > The drawbacks are strong. The cohesion between predicates and
>>>>>     handlers is lost. We have to define predicates in one place, and
>>>>>     handlers in another. This opens the door for bugs:
>>>>>     >
>>>>>     > - what if we forget to define a handler for a name? or a name for
>>>>>     a handler?
>>>>>     > - what if we misspell a name?
>>>>>     > - what if we copy-paste and duplicate a name?
>>>>>     >
>>>>>     > What Michael propose would have been totally OK if we had been
>>>>>     writing the API in Lua, Ruby or Python. In those languages the
>>>>>     "dynamic naming" approach would have looked most concise and
>>>>>     beautiful. But in Java we expect all the problems related to
>>>>>     identifiers to be eliminated in compile time.
>>>>>     >
>>>>>     > Do we have to invent duck-typing for the Java API?
>>>>>     >
>>>>>     > And if we do, what advantage are we supposed to get besides having
>>>>>     all the branches in the same scope? Michael, maybe I'm missing your
>>>>>     point?
>>>>>     >
>>>>>     > ---
>>>>>     >
>>>>>     > Earlier in this discussion John Roesler also proposed to do
>>>>>     without "start branching" operator, and later Paul mentioned that in
>>>>>     the case when we have to add a dynamic number of branches, the
>>>>>     current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
>>>>>     me address both comments here.
>>>>>     >
>>>>>     > 1) "Start branching" operator (I think that *split* is a good name
>>>>>     for it indeed) is critical when we need to do a dynamic branching,
>>>>>     see example below.
>>>>>     >
>>>>>     > 2) No, dynamic branching in current KIP is not clumsy at all.
>>>>>     Imagine a real-world scenario when you need one branch per enum
>>>>>     value (say, RecordType). You can have something like this:
>>>>>     >
>>>>>     > /*John:if we had to start with stream.branch(...) here, it would
>>>>>     have been much messier.*/
>>>>>     > KBranchedStream branched = stream.split();
>>>>>     >
>>>>>     > /*Not clumsy at all :-)*/
>>>>>     > for (RecordType recordType : RecordType.values())
>>>>>     >             branched = branched.branch((k, v) -> v.getRecType() ==
>>>>>     recordType,
>>>>>     >                     recordType::processRecords);
>>>>>     >
>>>>>     > Regards,
>>>>>     >
>>>>>     > Ivan
>>>>>     >
>>>>>     >
>>>>>     > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>>     >> I also agree with Michael's observation about the core problem of
>>>>>     >> current `branch()` implementation.
>>>>>     >>
>>>>>     >> However, I also don't like to pass in a clumsy Map object. My
>>>>>     thinking
>>>>>     >> was more aligned with Paul's proposal to just add a name to each
>>>>>     >> `branch()` statement and return a `Map<String,KStream>`.
>>>>>     >>
>>>>>     >> It makes the code easier to read, and also make the order of
>>>>>     >> `Predicates` (that is essential) easier to grasp.
>>>>>     >>
>>>>>     >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>     >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>     >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>     >>>>>>    .defaultBranch("defaultBranch");
>>>>>     >> An open question is the case for which no defaultBranch() should
>>>> be
>>>>>     >> specified. Atm, `split()` and `branch()` would return
>>>>>     `BranchedKStream`
>>>>>     >> and the call to `defaultBranch()` that returns the `Map` is
>>>> mandatory
>>>>>     >> (what is not the case atm). Or is this actually not a real
>>>> problem,
>>>>>     >> because users can just ignore the branch returned by
>>>>>     `defaultBranch()`
>>>>>     >> in the result `Map` ?
>>>>>     >>
>>>>>     >>
>>>>>     >> About "inlining": So far, it seems to be a matter of personal
>>>>>     >> preference. I can see arguments for both, but no "killer
>>>>>     argument" yet
>>>>>     >> that clearly make the case for one or the other.
>>>>>     >>
>>>>>     >>
>>>>>     >> -Matthias
>>>>>     >>
>>>>>     >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>>     >>> Perhaps inlining is the wrong terminology. It doesn’t require
>>>>>     that a lambda with the full downstream topology be defined inline -
>>>>>     it can be a method reference as with Ivan’s original suggestion.
>>>>>     The advantage of putting the predicate and its downstream logic
>>>>>     (Consumer) together in branch() is that they are required to be near
>>>>>     to each other.
>>>>>     >>>
>>>>>     >>> Ultimately the downstream code has to live somewhere, and deep
>>>>>     branch trees will be hard to read regardless.
>>>>>     >>>
>>>>>     >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>>     <michael.drogalis@confluent.io
>>>>>     <ma...@confluent.io>> wrote:
>>>>>     >>>>
>>>>>     >>>> I'm less enthusiastic about inlining the branch logic with its
>>>>>     downstream
>>>>>     >>>> functionality. Programs that have deep branch trees will
>>>>>     quickly become
>>>>>     >>>> harder to read as a single unit.
>>>>>     >>>>
>>>>>     >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>>     <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>>     >>>>>
>>>>>     >>>>> Also +1 on the issues/goals as Michael outlined them, I think
>>>>>     that sets a
>>>>>     >>>>> great framework for the discussion.
>>>>>     >>>>>
>>>>>     >>>>> Regarding the SortedMap solution, my understanding is that the
>>>>>     current
>>>>>     >>>>> proposal in the KIP is what is in my PR which (pending naming
>>>>>     decisions) is
>>>>>     >>>>> roughly this:
>>>>>     >>>>>
>>>>>     >>>>> stream.split()
>>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>     >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>>     >>>>>
>>>>>     >>>>> Obviously some ordering is necessary, since branching as a
>>>>>     construct
>>>>>     >>>>> doesn't work without it, but this solution seems like it
>>>>>     provides as much
>>>>>     >>>>> associativity as the SortedMap solution, because each branch()
>>>>>     call
>>>>>     >>>>> directly associates the "conditional" with the "code block."
>>>>>     The value it
>>>>>     >>>>> provides over the KIP solution is the accessing of streams in
>>>>>     the same
>>>>>     >>>>> scope.
>>>>>     >>>>>
>>>>>     >>>>> The KIP solution is less "dynamic" than the SortedMap solution
>>>>>     in the sense
>>>>>     >>>>> that it is slightly clumsier to add a dynamic number of
>>>>>     branches, but it is
>>>>>     >>>>> certainly possible.  It seems to me like the API should favor
>>>>>     the "static"
>>>>>     >>>>> case anyway, and should make it simple and readable to
>>>>>     fluently declare and
>>>>>     >>>>> access your branches in-line.  It also makes it impossible to
>>>>>     ignore a
>>>>>     >>>>> branch, and it is possible to build an (almost) identical
>>>>>     SortedMap
>>>>>     >>>>> solution on top of it.
>>>>>     >>>>>
>>>>>     >>>>> I could also see a middle ground where instead of a raw
>>>>>     SortedMap being
>>>>>     >>>>> taken in, branch() takes a name and not a Consumer.  Something
>>>>>     like this:
>>>>>     >>>>>
>>>>>     >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>     >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>     >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>     >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>>>     >>>>>
>>>>>     >>>>> Pros for that solution:
>>>>>     >>>>> - accessing branched KStreams in same scope
>>>>>     >>>>> - no double brace initialization, hopefully slightly more
>>>>>     readable than
>>>>>     >>>>> SortedMap
>>>>>     >>>>>
>>>>>     >>>>> Cons
>>>>>     >>>>> - downstream branch logic cannot be specified inline which
>>>>>     makes it harder
>>>>>     >>>>> to read top to bottom (like existing API and SortedMap, but
>>>>>     unlike the KIP)
>>>>>     >>>>> - you can forget to "handle" one of the branched streams (like
>>>>>     existing
>>>>>     >>>>> API and SortedMap, but unlike the KIP)
>>>>>     >>>>>
>>>>>     >>>>> (KBranchedStreams could even work *both* ways but perhaps
>>>>>     that's overdoing
>>>>>     >>>>> it).
>>>>>     >>>>>
>>>>>     >>>>> Overall I'm curious how important it is to be able to easily
>>>>>     access the
>>>>>     >>>>> branched KStream in the same scope as the original.  It's
>>>>>     possible that it
>>>>>     >>>>> doesn't need to be handled directly by the API, but instead
>>>>>     left up to the
>>>>>     >>>>> user.  I'm sort of in the middle on it.
>>>>>     >>>>>
>>>>>     >>>>> Paul
>>>>>     >>>>>
>>>>>     >>>>>
>>>>>     >>>>>
>>>>>     >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>>     <sophie@confluent.io <ma...@confluent.io>>
>>>>>     >>>>> wrote:
>>>>>     >>>>>
>>>>>     >>>>>> I'd like to +1 what Michael said about the issues with the
>>>>>     existing
>>>>>     >>>>> branch
>>>>>     >>>>>> method, I agree with what he's outlined and I think we should
>>>>>     proceed by
>>>>>     >>>>>> trying to alleviate these problems. Specifically it seems
>>>>>     important to be
>>>>>     >>>>>> able to cleanly access the individual branches (eg by mapping
>>>>>     >>>>>> name->stream), which I thought was the original intention of
>>>>>     this KIP.
>>>>>     >>>>>>
>>>>>     >>>>>> That said, I don't think we should so easily give in to the
>>>>>     double brace
>>>>>     >>>>>> anti-pattern or force ours users into it if at all possible to
>>>>>     >>>>> avoid...just
>>>>>     >>>>>> my two cents.
>>>>>     >>>>>>
>>>>>     >>>>>> Cheers,
>>>>>     >>>>>> Sophie
>>>>>     >>>>>>
>>>>>     >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>     >>>>>> michael.drogalis@confluent.io
>>>>>     <ma...@confluent.io>> wrote:
>>>>>     >>>>>>
>>>>>     >>>>>>> I’d like to propose a different way of thinking about this.
>>>>>     To me,
>>>>>     >>>>> there
>>>>>     >>>>>>> are three problems with the existing branch signature:
>>>>>     >>>>>>>
>>>>>     >>>>>>> 1. If you use it the way most people do, Java raises unsafe
>>>> type
>>>>>     >>>>>> warnings.
>>>>>     >>>>>>> 2. The way in which you use the stream branches is
>>>>>     positionally coupled
>>>>>     >>>>>> to
>>>>>     >>>>>>> the ordering of the conditionals.
>>>>>     >>>>>>> 3. It is brittle to extend existing branch calls with
>>>>>     additional code
>>>>>     >>>>>>> paths.
>>>>>     >>>>>>>
>>>>>     >>>>>>> Using associative constructs instead of relying on ordered
>>>>>     constructs
>>>>>     >>>>>> would
>>>>>     >>>>>>> be a stronger approach. Consider a signature that instead
>>>>>     looks like
>>>>>     >>>>>> this:
>>>>>     >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
>>>>>     Predicate<?
>>>>>     >>>>>>> super K,? super V>>);
>>>>>     >>>>>>>
>>>>>     >>>>>>> Branches are given names in a map, and as a result, the API
>>>>>     returns a
>>>>>     >>>>>>> mapping of names to streams. The ordering of the
>>>> conditionals is
>>>>>     >>>>>> maintained
>>>>>     >>>>>>> because it’s a sorted map. Insert order determines the order
>>>> of
>>>>>     >>>>>> evaluation.
>>>>>     >>>>>>> This solves problem 1 because there are no more varargs. It
>>>>>     solves
>>>>>     >>>>>> problem
>>>>>     >>>>>>> 2 because you no longer lean on ordering to access the
>>>>>     branch you’re
>>>>>     >>>>>>> interested in. It solves problem 3 because you can introduce
>>>>>     another
>>>>>     >>>>>>> conditional by simply attaching another name to the
>>>>>     structure, rather
>>>>>     >>>>>> than
>>>>>     >>>>>>> messing with the existing indices.
>>>>>     >>>>>>>
>>>>>     >>>>>>> One of the drawbacks is that creating the map inline is
>>>>>     historically
>>>>>     >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>>     voluminously, but
>>>>>     >>>>>>> double brace initialization would clean up the aesthetics.
>>>>>     >>>>>>>
>>>>>     >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>>     <john@confluent.io <ma...@confluent.io>>
>>>>>     >>>>> wrote:
>>>>>     >>>>>>>> Hi Ivan,
>>>>>     >>>>>>>>
>>>>>     >>>>>>>> Thanks for the update.
>>>>>     >>>>>>>>
>>>>>     >>>>>>>> FWIW, I agree with Matthias that the current "start
>>>> branching"
>>>>>     >>>>> operator
>>>>>     >>>>>>> is
>>>>>     >>>>>>>> confusing when named the same way as the actual branches.
>>>>>     "Split"
>>>>>     >>>>> seems
>>>>>     >>>>>>>> like a good name. Alternatively, we can do without a "start
>>>>>     >>>>> branching"
>>>>>     >>>>>>>> operator at all, and just do:
>>>>>     >>>>>>>>
>>>>>     >>>>>>>> stream
>>>>>     >>>>>>>>      .branch(Predicate)
>>>>>     >>>>>>>>      .branch(Predicate)
>>>>>     >>>>>>>>      .defaultBranch();
>>>>>     >>>>>>>>
>>>>>     >>>>>>>> Tentatively, I think that this branching operation should be
>>>>>     >>>>> terminal.
>>>>>     >>>>>>> That
>>>>>     >>>>>>>> way, we don't create ambiguity about how to use it. That
>>>>>     is, `branch`
>>>>>     >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
>>>>>     `void`, to
>>>>>     >>>>>>>> enforce that it comes last, and that there is only one
>>>>>     definition of
>>>>>     >>>>>> the
>>>>>     >>>>>>>> default branch. Potentially, we should log a warning if
>>>>>     there's no
>>>>>     >>>>>>> default,
>>>>>     >>>>>>>> and additionally log a warning (or throw an exception) if a
>>>>>     record
>>>>>     >>>>>> falls
>>>>>     >>>>>>>> though with no default.
>>>>>     >>>>>>>>
>>>>>     >>>>>>>> Thoughts?
>>>>>     >>>>>>>>
>>>>>     >>>>>>>> Thanks,
>>>>>     >>>>>>>> -John
>>>>>     >>>>>>>>
>>>>>     >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>>     >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>>     >>>>>>>> wrote:
>>>>>     >>>>>>>>
>>>>>     >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>> this is to make the name similar to String#split
>>>>>     >>>>>>>>>>> that also returns an array, right?
>>>>>     >>>>>>>>> The intend was to avoid name duplication. The return type
>>>>>     should
>>>>>     >>>>>> _not_
>>>>>     >>>>>>>>> be an array.
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> The current proposal is
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> stream.branch()
>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>     >>>>>>>>>      .defaultBranch();
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>>     `branch()` does
>>>>>     >>>>> not
>>>>>     >>>>>>>>> take any parameters and has different semantics than the
>>>> later
>>>>>     >>>>>>>>> `branch()` calls. Note, that from the code snippet above,
>>>> it's
>>>>>     >>>>> hidden
>>>>>     >>>>>>>>> that the first call is `KStream#branch()` while the others
>>>> are
>>>>>     >>>>>>>>> `KBranchedStream#branch()` what makes reading the code
>>>> harder.
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
>>>>>     I though
>>>>>     >>>>>> it
>>>>>     >>>>>>>>> might be better to also rename `KStream#branch()` to avoid
>>>> the
>>>>>     >>>>> naming
>>>>>     >>>>>>>>> overlap that seems to be confusing. The following reads
>>>> much
>>>>>     >>>>> cleaner
>>>>>     >>>>>> to
>>>>>     >>>>>>>> me:
>>>>>     >>>>>>>>> stream.split()
>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>     >>>>>>>>>      .branch(Predicate)
>>>>>     >>>>>>>>>      .defaultBranch();
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> Maybe there is a better alternative to `split()` though to
>>>>>     avoid
>>>>>     >>>>> the
>>>>>     >>>>>>>>> naming overlap.
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately
>>>> we
>>>>>     >>>>> cannot
>>>>>     >>>>>>> have
>>>>>     >>>>>>>>> a method with such name :-)
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
>>>>>     with a
>>>>>     >>>>> short
>>>>>     >>>>>>>> name?
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
>>>>>     with all
>>>>>     >>>>> it's
>>>>>     >>>>>>>>> methods? It will be part of public API and should be
>>>>>     contained in
>>>>>     >>>>> the
>>>>>     >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>     >>>>>>>>> `defaultBranch()` is.
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> You did not comment on the idea to add a
>>>>>     `KBranchedStream#get(int
>>>>>     >>>>>>> index)
>>>>>     >>>>>>>>> -> KStream` method to get the individually
>>>>>     branched-KStreams. Would
>>>>>     >>>>>> be
>>>>>     >>>>>>>>> nice to get your feedback about it. It seems you suggest
>>>>>     that users
>>>>>     >>>>>>>>> would need to write custom utility code otherwise, to
>>>>>     access them.
>>>>>     >>>>> We
>>>>>     >>>>>>>>> should discuss the pros and cons of both approaches. It
>>>> feels
>>>>>     >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
>>>>>     to get
>>>>>     >>>>> the
>>>>>     >>>>>>>>> branched-KStreams directly.
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>> -Matthias
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>
>>>>>     >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>     >>>>>>>>>> Hi all!
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> Matthias, thanks for your comment!
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>     >>>>>>>>>> I can see your point: this is to make the name similar to
>>>>>     >>>>>>> String#split
>>>>>     >>>>>>>>>> that also returns an array, right? But is it worth the
>>>>>     loss of
>>>>>     >>>>>>>> backwards
>>>>>     >>>>>>>>>> compatibility? We can have overloaded branch() as well
>>>>>     without
>>>>>     >>>>>>>> affecting
>>>>>     >>>>>>>>>> the existing code. Maybe the old array-based `branch`
>>>> method
>>>>>     >>>>> should
>>>>>     >>>>>>> be
>>>>>     >>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>>     >>>>> BranchingKStream#branch(),
>>>>>     >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>> BranchingKStream#default()
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default'
>>>> is,
>>>>>     >>>>>>> however, a
>>>>>     >>>>>>>>>> reserved word, so unfortunately we cannot have a method
>>>>>     with such
>>>>>     >>>>>>> name
>>>>>     >>>>>>>>> :-)
>>>>>     >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument,
>>>> but I
>>>>>     >>>>> think
>>>>>     >>>>>>> that
>>>>>     >>>>>>>>>> is not required?
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> Absolutely! I think that was just copy-paste error or
>>>>>     something.
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> Dear colleagues,
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>     >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> Any new suggestions/objections?
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> Ivan
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>>
>>>>>     >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>     >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems
>>>> that
>>>>>     >>>>>>> everybody
>>>>>     >>>>>>>>>>> agrees that the current branch() method using arrays is
>>>> not
>>>>>     >>>>>> optimal.
>>>>>     >>>>>>>>>>> I had a quick look into the PR and I like the overall
>>>>>     proposal.
>>>>>     >>>>>>> There
>>>>>     >>>>>>>>>>> are some minor things we need to consider. I would
>>>>>     recommend the
>>>>>     >>>>>>>>>>> following renaming:
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>> KStream#branch() -> #split()
>>>>>     >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>     >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>>     BranchingKStream#default()
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>> It's just a suggestion to get slightly shorter method
>>>> names.
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>>     `Predicate` as
>>>>>     >>>>>>>> argument,
>>>>>     >>>>>>>>>>> but I think that is not required?
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>> Also, we should consider KIP-307, that was recently
>>>>>     accepted and
>>>>>     >>>>>> is
>>>>>     >>>>>>>>>>> currently implemented:
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>
>>>>>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>     >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
>>>>>     parameter.
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>> For the issue that the created `KStream` object are in
>>>>>     different
>>>>>     >>>>>>>> scopes:
>>>>>     >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>>>>>     index)` method
>>>>>     >>>>>>> that
>>>>>     >>>>>>>>>>> returns the corresponding "branched" result `KStream`
>>>>>     object?
>>>>>     >>>>>> Maybe,
>>>>>     >>>>>>>> the
>>>>>     >>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>>     >>>>>> `Consumer<KStream>`
>>>>>     >>>>>>>> but
>>>>>     >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
>>>>>     whatever
>>>>>     >>>>>> the
>>>>>     >>>>>>>>>>> `Function` returns?
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
>>>>>     current
>>>>>     >>>>>>>>>>> proposal. That makes it easier to review.
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>> -Matthias
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>>
>>>>>     >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>     >>>>>>>>>>>> Ivan,
>>>>>     >>>>>>>>>>>>
>>>>>     >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
>>>>>     makes sense
>>>>>     >>>>>> for
>>>>>     >>>>>>>> you
>>>>>     >>>>>>>>> to
>>>>>     >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
>>>>>     we'll
>>>>>     >>>>> need
>>>>>     >>>>>>>> some
>>>>>     >>>>>>>>>>>> buy-in from committers that have actual binding votes on
>>>>>     >>>>> whether
>>>>>     >>>>>>> the
>>>>>     >>>>>>>>> KIP
>>>>>     >>>>>>>>>>>> could be adopted.  It would be great to hear if they
>>>>>     think this
>>>>>     >>>>>> is
>>>>>     >>>>>>> a
>>>>>     >>>>>>>>> good
>>>>>     >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
>>>>>     starting a
>>>>>     >>>>>>> vote,
>>>>>     >>>>>>>>> or if
>>>>>     >>>>>>>>>>>> there is generally some indication of interest
>>>> beforehand.
>>>>>     >>>>>>>>>>>>
>>>>>     >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
>>>>>     assuming
>>>>>     >>>>> we
>>>>>     >>>>>> do
>>>>>     >>>>>>>>> move
>>>>>     >>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>>     >>>>>> KBranchedStream",
>>>>>     >>>>>>> do
>>>>>     >>>>>>>>> we
>>>>>     >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I
>>>> would
>>>>>     >>>>> favor
>>>>>     >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs
>>>> that
>>>>>     >>>>>>> accomplish
>>>>>     >>>>>>>>> the
>>>>>     >>>>>>>>>>>> same thing is confusing, especially when they're fairly
>>>>>     similar
>>>>>     >>>>>>>>> anyway.  We
>>>>>     >>>>>>>>>>>> just need to be sure we're not making something
>>>>>     >>>>>>> impossible/difficult
>>>>>     >>>>>>>>> that
>>>>>     >>>>>>>>>>>> is currently possible/easy.
>>>>>     >>>>>>>>>>>>
>>>>>     >>>>>>>>>>>> Regarding my PR - I think the general structure would
>>>> work,
>>>>>     >>>>> it's
>>>>>     >>>>>>>> just a
>>>>>     >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>>     >>>>>>> particular,
>>>>>     >>>>>>>>>>>> passing in the "predicates" and "children" lists which
>>>> get
>>>>>     >>>>>> modified
>>>>>     >>>>>>>> in
>>>>>     >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>>     KStreamLazyBranch is
>>>>>     >>>>> a
>>>>>     >>>>>>> bit
>>>>>     >>>>>>>>>>>> complicated to follow.
>>>>>     >>>>>>>>>>>>
>>>>>     >>>>>>>>>>>> Thanks,
>>>>>     >>>>>>>>>>>> Paul
>>>>>     >>>>>>>>>>>>
>>>>>     >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>     >>>>>>>>> wrote:
>>>>>     >>>>>>>>>>>>> Hi Paul!
>>>>>     >>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>>     convinced: your
>>>>>     >>>>>>>> proposal
>>>>>     >>>>>>>>>>>>> looks better and should work. We just have to document
>>>> the
>>>>>     >>>>>> crucial
>>>>>     >>>>>>>>> fact
>>>>>     >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
>>>>>     And then
>>>>>     >>>>>> it's
>>>>>     >>>>>>>> all
>>>>>     >>>>>>>>>>>>> going to be very nice.
>>>>>     >>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
>>>>>     resume the
>>>>>     >>>>>>>>>>>>> discussion here, right?
>>>>>     >>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>>>>     >>>>> starting
>>>>>     >>>>>>>> point
>>>>>     >>>>>>>>> if
>>>>>     >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
>>>>>     starting
>>>>>     >>>>>>> point.
>>>>>     >>>>>>>>> But
>>>>>     >>>>>>>>>>>>> as a novice in this project I might miss some important
>>>>>     >>>>> details.
>>>>>     >>>>>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>> Ivan
>>>>>     >>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>     >>>>>>>>>>>>>> Ivan,
>>>>>     >>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>>     >>>>> stream.branch()
>>>>>     >>>>>>>>> solution
>>>>>     >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>>>>     >>>>> invoked
>>>>>     >>>>>> as
>>>>>     >>>>>>>>> they’re
>>>>>     >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
>>>>>     still
>>>>>     >>>>>> ought
>>>>>     >>>>>>> to
>>>>>     >>>>>>>>> be
>>>>>     >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
>>>>>     depend on
>>>>>     >>>>> the
>>>>>     >>>>>>>>> branched
>>>>>     >>>>>>>>>>>>> streams having been set.
>>>>>     >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
>>>>>     access
>>>>>     >>>>> the
>>>>>     >>>>>>>>> branched
>>>>>     >>>>>>>>>>>>> streams in the same scope as the original stream (that
>>>>>     is, not
>>>>>     >>>>>>>> inside
>>>>>     >>>>>>>>> the
>>>>>     >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>>>>     >>>>> solutions.
>>>>>     >>>>>> It
>>>>>     >>>>>>>>> can be
>>>>>     >>>>>>>>>>>>> worked around though.
>>>>>     >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
>>>>>     excited
>>>>>     >>>>> to
>>>>>     >>>>>>>> hear
>>>>>     >>>>>>>>>>>>> your thoughts!]
>>>>>     >>>>>>>>>>>>>> Paul
>>>>>     >>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>     >>>>>>>>> wrote:
>>>>>     >>>>>>>>>>>>>>> Hi Paul!
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>     >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at
>>>> first
>>>>>     >>>>> glance,
>>>>>     >>>>>>> but
>>>>>     >>>>>>>>> ---
>>>>>     >>>>>>>>>>>>>>>> the newly branched streams are not available in the
>>>>>     same
>>>>>     >>>>>> scope
>>>>>     >>>>>>> as
>>>>>     >>>>>>>>> each
>>>>>     >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
>>>> together
>>>>>     >>>>> again
>>>>>     >>>>>> I
>>>>>     >>>>>>>>> don't see
>>>>>     >>>>>>>>>>>>> a way to do that.
>>>>>     >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
>>>>>     just
>>>>>     >>>>>> going
>>>>>     >>>>>>> to
>>>>>     >>>>>>>>>>>>> write in details about this issue.
>>>>>     >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
>>>>>     we need
>>>>>     >>>>> to
>>>>>     >>>>>>>>> identify
>>>>>     >>>>>>>>>>>>> customers who have bought coffee and made a purchase
>>>>>     in the
>>>>>     >>>>>>>>> electronics
>>>>>     >>>>>>>>>>>>> store to give them coupons.
>>>>>     >>>>>>>>>>>>>>> This is the code I usually write under these
>>>>>     circumstances
>>>>>     >>>>>> using
>>>>>     >>>>>>>> my
>>>>>     >>>>>>>>>>>>> 'brancher' class:
>>>>>     >>>>>>>>>>>>>>> @Setter
>>>>>     >>>>>>>>>>>>>>> class CouponIssuer{
>>>>>     >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>     >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>     >>>>>>>>>>>>>>>       return
>>>>>     >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>     >>>>>>>>>>>>>>>       /*In the real world the code here can be
>>>>>     complex, so
>>>>>     >>>>>>>>> creation of
>>>>>     >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
>>>>>     order to
>>>>>     >>>>>>>> separate
>>>>>     >>>>>>>>>>>>> classes' responsibilities.*/
>>>>>     >>>>>>>>>>>>>>>  }
>>>>>     >>>>>>>>>>>>>>> }
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>     >>>>>>>>>>>>>>>     .branch(predicate1,
>>>> couponIssuer::setCoffePurchases)
>>>>>     >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>     >>>>>> couponIssuer::setElectronicsPurchases)
>>>>>     >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
>>>>>     everything
>>>>>     >>>>>>>> later,
>>>>>     >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>     >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>>     initialize the
>>>>>     >>>>>>>>> CouponIssuer
>>>>>     >>>>>>>>>>>>> we need the terminal operation to be called before
>>>>>     >>>>>>>>> streamsBuilder.build()
>>>>>     >>>>>>>>>>>>> is called.
>>>>>     >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
>>>>>     essentially
>>>>>     >>>>>> the
>>>>>     >>>>>>>>> next
>>>>>     >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
>>>>>     based on
>>>>>     >>>>> my
>>>>>     >>>>>>>>> experience,
>>>>>     >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>     >>>>>>>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> Ivan
>>>>>     >>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>     >>>>>>>>>>>>>>>> Ivan,
>>>>>     >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
>>>>>     fluent
>>>>>     >>>>> API
>>>>>     >>>>>>>> based
>>>>>     >>>>>>>>>>>>> off of
>>>>>     >>>>>>>>>>>>>>>> KStream here
>>>>>     (https://github.com/apache/kafka/pull/6512),
>>>>>     >>>>>> and
>>>>>     >>>>>>> I
>>>>>     >>>>>>>>> think
>>>>>     >>>>>>>>>>>>> I
>>>>>     >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>     >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>>>     >>>>>>> compatibility
>>>>>     >>>>>>>>>>>>> issues,
>>>>>     >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
>>>>>     that Java
>>>>>     >>>>> is
>>>>>     >>>>>>>> smart
>>>>>     >>>>>>>>>>>>> enough to
>>>>>     >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>>>>>     returning one
>>>>>     >>>>>>> thing
>>>>>     >>>>>>>>> and
>>>>>     >>>>>>>>>>>>> branch()
>>>>>     >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>     >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
>>>>>     need
>>>>>     >>>>> it.
>>>>>     >>>>>>> We
>>>>>     >>>>>>>>> can
>>>>>     >>>>>>>>>>>>> just
>>>>>     >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
>>>>>     shares
>>>>>     >>>>>> its
>>>>>     >>>>>>>>> state
>>>>>     >>>>>>>>>>>>> with the
>>>>>     >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
>>>>>     branching.
>>>>>     >>>>>>> It's
>>>>>     >>>>>>>>> not
>>>>>     >>>>>>>>>>>>> terribly
>>>>>     >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>>>>>     demonstrates
>>>>>     >>>>>> its
>>>>>     >>>>>>>>>>>>> feasibility.
>>>>>     >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should
>>>> be
>>>>>     >>>>> final
>>>>>     >>>>>> or
>>>>>     >>>>>>>>> even a
>>>>>     >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
>>>>>     wanted to
>>>>>     >>>>>> see
>>>>>     >>>>>>>> how
>>>>>     >>>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>     >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
>>>>>     solution
>>>>>     >>>>>>> could
>>>>>     >>>>>>>> be
>>>>>     >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>>>>     >>>>> suggested
>>>>>     >>>>>>>> was a
>>>>>     >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
>>>>>     streams
>>>>>     >>>>>> are
>>>>>     >>>>>>>> not
>>>>>     >>>>>>>>>>>>>>>> available in the same scope as each other.  That
>>>>>     is, if we
>>>>>     >>>>>>> wanted
>>>>>     >>>>>>>>> to
>>>>>     >>>>>>>>>>>>> merge
>>>>>     >>>>>>>>>>>>>>>> them back together again I don't see a way to do
>>>>>     that.  The
>>>>>     >>>>>> KIP
>>>>>     >>>>>>>>>>>>> proposal
>>>>>     >>>>>>>>>>>>>>>> has the same issue, though - all this means is that
>>>> for
>>>>>     >>>>>> either
>>>>>     >>>>>>>>>>>>> solution,
>>>>>     >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
>>>>>     table.
>>>>>     >>>>>>>>>>>>>>>> Thanks,
>>>>>     >>>>>>>>>>>>>>>> Paul
>>>>>     >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>>     >>>>>>>>>>>>> wrote:
>>>>>     >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to
>>>> this
>>>>>     >>>>>> point.
>>>>>     >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
>>>>>     branch API
>>>>>     >>>>>>> needs
>>>>>     >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>     >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
>>>>>     returns
>>>>>     >>>>>> its
>>>>>     >>>>>>>>> argument
>>>>>     >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code
>>>> won't
>>>>>     >>>>> make
>>>>>     >>>>>>>> sense
>>>>>     >>>>>>>>>>>>> until
>>>>>     >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
>>>>>     instance
>>>>>     >>>>>>>>> contrasts the
>>>>>     >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> stream
>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>>     >>>>>>>>> defaultBranch(..)
>>>>>     >>>>>>>>>>>>> and
>>>>>     >>>>>>>>>>>>>>>>> noDefault() return void
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface
>>>> is
>>>>>     >>>>>> defined.
>>>>>     >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>     >>>>>>>> (defaultBranch(ks->)
>>>>>     >>>>>>>>> and
>>>>>     >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
>>>>>     miss the
>>>>>     >>>>>> fact
>>>>>     >>>>>>>>> that one
>>>>>     >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
>>>>>     methods
>>>>>     >>>>>> are
>>>>>     >>>>>>>> not
>>>>>     >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do
>>>> better?
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> Ivan
>>>>>     >>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>     >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>     >>>>>>>>>>>>>>>>>>> Paul,
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>     >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>>>>>     implemented the
>>>>>     >>>>>>> easy
>>>>>     >>>>>>>>> way.
>>>>>     >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
>>>> assumes
>>>>>     >>>>> nothing
>>>>>     >>>>>>>> will
>>>>>     >>>>>>>>>>>>> reach
>>>>>     >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>     >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only
>>>> option
>>>>>     >>>>>> besides
>>>>>     >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
>>>>>     want to
>>>>>     >>>>>> just
>>>>>     >>>>>>>>> silently
>>>>>     >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>>     predicate. 2)
>>>>>     >>>>>>> Throwing
>>>>>     >>>>>>>>> an
>>>>>     >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
>>>>>     looks
>>>>>     >>>>>> like a
>>>>>     >>>>>>>> bad
>>>>>     >>>>>>>>>>>>> idea.
>>>>>     >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
>>>>>     emit a
>>>>>     >>>>>>>> special
>>>>>     >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly
>>>> where
>>>>>     >>>>>>> `default`
>>>>>     >>>>>>>>> can
>>>>>     >>>>>>>>>>>>> be
>>>>>     >>>>>>>>>>>>>>>>>>> used.
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>     >>>>> InternalTopologyBuilder
>>>>>     >>>>>>> to
>>>>>     >>>>>>>>> track
>>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>>     >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>     a clear
>>>>>     >>>>>>> error
>>>>>     >>>>>>>>>>>>> before it
>>>>>     >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>>>>     >>>>> compiled
>>>>>     >>>>>>> and
>>>>>     >>>>>>>>> run?
>>>>>     >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>>>>>     compile if
>>>>>     >>>>> used
>>>>>     >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>>>>>     method chain
>>>>>     >>>>>>>> starting
>>>>>     >>>>>>>>>>>>> from
>>>>>     >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
>>>>>     between
>>>>>     >>>>>>>> runtime
>>>>>     >>>>>>>>> and
>>>>>     >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>>>>     >>>>> instantly
>>>>>     >>>>>> on
>>>>>     >>>>>>>>> unit
>>>>>     >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
>>>>>     compilation
>>>>>     >>>>>>>> failure.
>>>>>     >>>>>>>>>>>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>> Ivan
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>     >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
>>>>>     required.
>>>>>     >>>>>>> But
>>>>>     >>>>>>>> is
>>>>>     >>>>>>>>>>>>> that
>>>>>     >>>>>>>>>>>>>>>>>>>> really
>>>>>     >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>>>     >>>>>> defaultBranch
>>>>>     >>>>>>>>> they
>>>>>     >>>>>>>>>>>>> can
>>>>>     >>>>>>>>>>>>>>>>>>>> call
>>>>>     >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
>>>>>     just as
>>>>>     >>>>>>>>> easily.  In
>>>>>     >>>>>>>>>>>>>>>>>>>> fact I
>>>>>     >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API
>>>> - a
>>>>>     >>>>> user
>>>>>     >>>>>>>> could
>>>>>     >>>>>>>>>>>>> specify
>>>>>     >>>>>>>>>>>>>>>>> a
>>>>>     >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach
>>>> the
>>>>>     >>>>>> default
>>>>>     >>>>>>>>> branch,
>>>>>     >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>> That
>>>>>     >>>>> seems
>>>>>     >>>>>>> like
>>>>>     >>>>>>>>> an
>>>>>     >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
>>>>>     which allows
>>>>>     >>>>>> for
>>>>>     >>>>>>>> the
>>>>>     >>>>>>>>>>>>> more
>>>>>     >>>>>>>>>>>>>>>>>>>> subtle
>>>>>     >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
>>>> dropped.
>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
>>>>>     to be
>>>>>     >>>>>> well
>>>>>     >>>>>>>>>>>>>>>>>>>> documented, but
>>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>>     >>>>> InternalTopologyBuilder
>>>>>     >>>>>>> to
>>>>>     >>>>>>>>> track
>>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>>     >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>>     a clear
>>>>>     >>>>>>> error
>>>>>     >>>>>>>>>>>>> before it
>>>>>     >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is
>>>> a
>>>>>     >>>>> "build
>>>>>     >>>>>>>> step"
>>>>>     >>>>>>>>>>>>> where
>>>>>     >>>>>>>>>>>>>>>>> the
>>>>>     >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>     >>>>>> StreamsBuilder.build()
>>>>>     >>>>>>> is
>>>>>     >>>>>>>>>>>>> called.
>>>>>     >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I
>>>> agree
>>>>>     >>>>> that
>>>>>     >>>>>>> it's
>>>>>     >>>>>>>>>>>>>>>>>>>> critical to
>>>>>     >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
>>>>>     stream.
>>>>>     >>>>>>> With
>>>>>     >>>>>>>>> the
>>>>>     >>>>>>>>>>>>>>>>> fluent
>>>>>     >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all
>>>> other
>>>>>     >>>>>>> operations
>>>>>     >>>>>>>>> do -
>>>>>     >>>>>>>>>>>>> if
>>>>>     >>>>>>>>>>>>>>>>> you
>>>>>     >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
>>>> multiple
>>>>>     >>>>> times,
>>>>>     >>>>>>> you
>>>>>     >>>>>>>>> just
>>>>>     >>>>>>>>>>>>>>>>>>>> need the
>>>>>     >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
>>>>>     operations
>>>>>     >>>>>> on
>>>>>     >>>>>>> it
>>>>>     >>>>>>>>> as
>>>>>     >>>>>>>>>>>>> you
>>>>>     >>>>>>>>>>>>>>>>>>>> desire.
>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>> Best,
>>>>>     >>>>>>>>>>>>>>>>>>>> Paul
>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>>     >>>>>>>>>>>>>>>>>>>> wrote:
>>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
>>>>>     always need
>>>>>     >>>>>> the
>>>>>     >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>>> operation we
>>>>>     >>>>> don't
>>>>>     >>>>>>>> know
>>>>>     >>>>>>>>>>>>> when to
>>>>>     >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
>>>>>     so we
>>>>>     >>>>> can
>>>>>     >>>>>> do
>>>>>     >>>>>>>>>>>>> something
>>>>>     >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
>>>> special
>>>>>     >>>>> object
>>>>>     >>>>>>>>>>>>> construction
>>>>>     >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods.
>>>> But
>>>>>     >>>>> here
>>>>>     >>>>>> we
>>>>>     >>>>>>>>> have a
>>>>>     >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
>>>>>     flow,
>>>>>     >>>>> so
>>>>>     >>>>>> I
>>>>>     >>>>>>>>> think
>>>>>     >>>>>>>>>>>>> this
>>>>>     >>>>>>>>>>>>>>>>> is
>>>>>     >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>     >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
>>>>>     API, but I
>>>>>     >>>>>> find
>>>>>     >>>>>>>> the
>>>>>     >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>     >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>>>>>     contrasts the
>>>>>     >>>>>>> fluency
>>>>>     >>>>>>>>> of
>>>>>     >>>>>>>>>>>>> other
>>>>>     >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
>>>>>     just call
>>>>>     >>>>> a
>>>>>     >>>>>>>>> method on
>>>>>     >>>>>>>>>>>>> the
>>>>>     >>>>>>>>>>>>>>>>>>>>> stream
>>>>>     >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
>>>>>     cases
>>>>>     >>>>> are
>>>>>     >>>>>>>>> defined
>>>>>     >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>     >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
>>>>>     is very
>>>>>     >>>>>> nice
>>>>>     >>>>>>>>> and the
>>>>>     >>>>>>>>>>>>>>>>>>>>>> right
>>>>>     >>>>>>>>>>>>>>>>>>>>> way
>>>>>     >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
>>>>>     how we
>>>>>     >>>>>>> specify
>>>>>     >>>>>>>>> the
>>>>>     >>>>>>>>>>>>> source
>>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>>> this::handle1)
>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>>> this::handle2)
>>>>>     >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>>     >>>>>>>> KStreamBrancher
>>>>>     >>>>>>>>> or
>>>>>     >>>>>>>>>>>>>>>>>>>>> something,
>>>>>     >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>>     terminated by
>>>>>     >>>>>>>>>>>>> defaultBranch()
>>>>>     >>>>>>>>>>>>>>>>>>>>>> (which
>>>>>     >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>>     incompatible with
>>>>>     >>>>> the
>>>>>     >>>>>>>>> current
>>>>>     >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>>     >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
>>>>>     different
>>>>>     >>>>>> name,
>>>>>     >>>>>>>> but
>>>>>     >>>>>>>>> that
>>>>>     >>>>>>>>>>>>>>>>>>>>>> seems
>>>>>     >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>>>     >>>>>> something
>>>>>     >>>>>>>> like
>>>>>     >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>     >>>>>>>>>>>>>>>>>>>>> or
>>>>>     >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
>>>>>     KIP?  It
>>>>>     >>>>>> seems
>>>>>     >>>>>>>>> like it
>>>>>     >>>>>>>>>>>>>>>>>>>>>> does to
>>>>>     >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
>>>>>     while also
>>>>>     >>>>>>>> allowing
>>>>>     >>>>>>>>> you
>>>>>     >>>>>>>>>>>>> to
>>>>>     >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>>     KBranchedStreams
>>>>>     >>>>>> if
>>>>>     >>>>>>>>> desired.
>>>>>     >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>     >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>     >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>     >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
>>>>>     ks){
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>>>>>     String> ks){
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>>     this::handleFirstCase)
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>>     this::handleSecondCase)
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>>> KafkaStreamsBrancher
>>>>>     >>>>> takes a
>>>>>     >>>>>>>>> Consumer
>>>>>     >>>>>>>>>>>>> as a
>>>>>     >>>>>>>>>>>>>>>>>>>>>>> second
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
>>>>>     example in
>>>>>     >>>>>> the
>>>>>     >>>>>>>> KIP
>>>>>     >>>>>>>>>>>>> shows
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>     >>>>>>>>> (KafkaStreams#to()
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would
>>>> we
>>>>>     >>>>> handle
>>>>>     >>>>>>> the
>>>>>     >>>>>>>>> case
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to
>>>> continue
>>>>>     >>>>>>>> processing
>>>>>     >>>>>>>>> and
>>>>>     >>>>>>>>>>>>> not
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the
>>>> branched
>>>>>     >>>>>> stream
>>>>>     >>>>>>>>>>>>> immediately?
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
>>>>>     we had
>>>>>     >>>>>>>> something
>>>>>     >>>>>>>>> like
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck
>>>> <
>>>>>     >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for
>>>> KIP-
>>>>>     >>>>> 418.
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
>>>> KIP-418.
>>>>>     >>>>> Please
>>>>>     >>>>>>>> take
>>>>>     >>>>>>>>> a
>>>>>     >>>>>>>>>>>>> look
>>>>>     >>>>>>>>>>>>>>>>> at
>>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
>>>>>     feedback :)
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>     >>>>>
>>>>>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>     >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>>     >>>>> https://github.com/apache/kafka/pull/6164
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>     >>>>>>>>>
>>>>>     >
>>>>>
>>>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks for the input John!

> under your suggestion, it seems that the name is required

If you want to get the `KStream` as part of the `Map` back using a
`Function`, yes. If you follow the "embedded chaining" pattern using a
`Consumer`, no.

Allowing for a default name via `split()` can of course be done.
Similarly, using `Named` instead of `String` is possible.

I wanted to sketch out a high level proposal to merge both patterns
only. Your suggestions to align the new API with the existing API make
totally sense.



One follow up question: Would `Named` be optional or required in
`split()` and `branch()`? It's unclear from your example.

If both are mandatory, what do we gain by it? The returned `Map` only
contains the corresponding branches, so why should we prefix all of
them? If only `Named` is mandatory in `branch()`, but optional in
`split()`, the same question raises?

Requiring `Named` in `split()` seems only to make sense, if `Named` is
optional in `branch()` and we generate `-X` suffix using a counter for
different branch name. However, this might lead to the problem of
changing names if branches are added/removed. Also, how would the names
be generated if `Consumer` is mixed in (ie, not all branches are
returned in the `Map`).

If `Named` is optional for both, it could happen that a user misses to
specify a name for a branch what would lead to runtime issues.


Hence, I am actually in favor to not allow a default name but keep
`split()` without parameter and make `Named` in `branch()` required if a
`Function` is used. This makes it explicit to the user that specifying a
name is required if a `Function` is used.



About

> KBranchedStream#branch(BranchConfig)

I don't think that the branching predicate is a configuration and hence
would not include it in a configuration object.

>     withChain(...);

Similar, `withChain()` (that would only take a `Consumer`?) does not
seem to be a configuration. We can also not prevent a user to call
`withName()` in combination of `withChain()` what does not make sense
IMHO. We could of course throw an RTE but not have a compile time check
seems less appealing. Also, it could happen that neither `withChain()`
not `withName()` is called and the branch is missing in the returned
`Map` what lead to runtime issues, too.

Hence, I don't think that we should add `BranchConfig`. A config object
is helpful if each configuration can be set independently of all others,
but this seems not to be the case here. If we add new configuration
later, we can also just move forward by deprecating the methods that
accept `Named` and add new methods that accepted `BranchConfig` (that
would of course implement `Named`).


Thoughts?


@Ivan, what do you think about the general idea to blend the two main
approaches of returning a `Map` plus an "embedded chaining"?



-Matthias



On 6/4/19 10:33 AM, John Roesler wrote:
> Thanks for the idea, Matthias, it does seem like this would satisfy
> everyone. Returning the map from the terminal operations also solves
> the problem of merging/joining the branched streams, if we want to add
> support for the compliment later on.
> 
> Under your suggestion, it seems that the name is required. Otherwise,
> we wouldn't have keys for the map to return. I this this is actually
> not too bad, since experience has taught us that, although names for
> operations are not required to define stream processing logic, it does
> significantly improve the operational experience when you can map the
> topology, logs, metrics, etc. back to the source code. Since you
> wouldn't (have to) reference the name to chain extra processing onto
> the branch (thanks to the second argument), you can avoid the
> "unchecked name" problem that Ivan pointed out.
> 
> In the current implementation of Branch, you can name the branch
> operator itself, and then all the branches get index-suffixed names
> built from the branch operator name. I guess under this proposal, we
> could naturally append the branch name to the branching operator name,
> like this:
> 
>    stream.split(Named.withName("mysplit")) //creates node "mysplit"
>               .branch(..., ..., "abranch") // creates node "mysplit-abranch"
>               .defaultBranch(...) // creates node "mysplit-default"
> 
> It does make me wonder about the DSL syntax itself, though.
> 
> We don't have a defined grammar, so there's plenty of room to debate
> the "best" syntax in the context of each operation, but in general,
> the KStream DSL operators follow this pattern:
> 
>     operator(function, config_object?) OR operator(config_object)
> 
> where config_object is often just Named in the "function" variant.
> Even when the config_object isn't a Named, but some other config
> class, that config class _always_ implements NamedOperation.
> 
> Here, we're introducing a totally different pattern:
> 
>   operator(function, function, string)
> 
> where the string is the name.
> My first question is whether the name should instead be specified with
> the NamedOperation interface.
> 
> My second question is whether we should just roll all these arguments
> up into a config object like:
> 
>    KBranchedStream#branch(BranchConfig)
> 
>    interface BranchConfig extends NamedOperation {
>     withPredicate(...);
>     withChain(...);
>     withName(...);
>   }
> 
> Although I guess we'd like to call BranchConfig something more like
> "Branched", even if I don't particularly like that pattern.
> 
> This makes the source code a little noisier, but it also makes us more
> future-proof, as we can deal with a wide range of alternatives purely
> in the config interface, and never have to deal with adding overloads
> to the KBranchedStream if/when we decide we want the name to be
> optional, or the KStream->KStream to be optional.
> 
> WDYT?
> 
> Thanks,
> -John
> 
> On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
> <mi...@confluent.io> wrote:
>>
>> Matthias: I think that's pretty reasonable from my point of view. Good
>> suggestion.
>>
>> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>>> Interesting discussion.
>>>
>>> I am wondering, if we cannot unify the advantage of both approaches:
>>>
>>>
>>>
>>> KStream#split() -> KBranchedStream
>>>
>>> // branch is not easily accessible in current scope
>>> KBranchedStream#branch(Predicate, Consumer<KStream>)
>>>   -> KBranchedStream
>>>
>>> // assign a name to the branch and
>>> // return the sub-stream to the current scope later
>>> //
>>> // can be simple as `#branch(p, s->s, "name")`
>>> // or also complex as `#branch(p, s->s.filter(...), "name")`
>>> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>>>   -> KBranchedStream
>>>
>>> // default branch is not easily accessible
>>> // return map of all named sub-stream into current scope
>>> KBranchedStream#default(Cosumer<KStream>)
>>>   -> Map<String,KStream>
>>>
>>> // assign custom name to default-branch
>>> // return map of all named sub-stream into current scope
>>> KBranchedStream#default(Function<KStream,KStream>, String)
>>>   -> Map<String,KStream>
>>>
>>> // assign a default name for default
>>> // return map of all named sub-stream into current scope
>>> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>>>   -> Map<String,KStream>
>>>
>>> // return map of all names sub-stream into current scope
>>> KBranchedStream#noDefaultBranch()
>>>   -> Map<String,KStream>
>>>
>>>
>>>
>>> Hence, for each sub-stream, the user can pick to add a name and return
>>> the branch "result" to the calling scope or not. The implementation can
>>> also check at runtime that all returned names are unique. The returned
>>> Map can be empty and it's also optional to use the Map.
>>>
>>> To me, it seems like a good way to get best of both worlds.
>>>
>>> Thoughts?
>>>
>>>
>>>
>>> -Matthias
>>>
>>>
>>>
>>>
>>> On 5/6/19 5:15 PM, John Roesler wrote:
>>>> Ivan,
>>>>
>>>> That's a very good point about the "start" operator in the dynamic case.
>>>> I had no problem with "split()"; I was just questioning the necessity.
>>>> Since you've provided a proof of necessity, I'm in favor of the
>>>> "split()" start operator. Thanks!
>>>>
>>>> Separately, I'm interested to see where the present discussion leads.
>>>> I've written enough Javascript code in my life to be suspicious of
>>>> nested closures. You have a good point about using method references (or
>>>> indeed function literals also work). It should be validating that this
>>>> was also the JS community's first approach to flattening the logic when
>>>> their nested closure situation got out of hand. Unfortunately, it's
>>>> replacing nesting with redirection, both of which disrupt code
>>>> readability (but in different ways for different reasons). In other
>>>> words, I agree that function references is *the* first-order solution if
>>>> the nested code does indeed become a problem.
>>>>
>>>> However, the history of JS also tells us that function references aren't
>>>> the end of the story either, and you can see that by observing that
>>>> there have been two follow-on eras, as they continue trying to cope with
>>>> the consequences of living in such a callback-heavy language. First, you
>>>> have Futures/Promises, which essentially let you convert nested code to
>>>> method-chained code (Observables/FP is a popular variation on this).
>>>> Most lately, you have async/await, which is an effort to apply language
>>>> (not just API) syntax to the problem, and offer the "flattest" possible
>>>> programming style to solve the problem (because you get back to just one
>>>> code block per functional unit).
>>>>
>>>> Stream-processing is a different domain, and Java+KStreams is nowhere
>>>> near as callback heavy as JS, so I don't think we have to take the JS
>>>> story for granted, but then again, I think we can derive some valuable
>>>> lessons by looking sideways to adjacent domains. I'm just bringing this
>>>> up to inspire further/deeper discussion. At the same time, just like JS,
>>>> we can afford to take an iterative approach to the problem.
>>>>
>>>> Separately again, I'm interested in the post-branch merge (and I'd also
>>>> add join) problem that Paul brought up. We can clearly punt on it, by
>>>> terminating the nested branches with sink operators. But is there a DSL
>>>> way to do it?
>>>>
>>>> Thanks again for your driving this,
>>>> -John
>>>>
>>>> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
>>>> <ma...@gmail.com>> wrote:
>>>>
>>>>     Ivan, I’ll definitely forfeit my point on the clumsiness of the
>>>>     branch(predicate, consumer) solution, I don’t see any real drawbacks
>>>>     for the dynamic case.
>>>>
>>>>     IMO the one trade off to consider at this point is the scope
>>>>     question. I don’t know if I totally agree that “we rarely need them
>>>>     in the same scope” since merging the branches back together later
>>>>     seems like a perfectly plausible use case that can be a lot nicer
>>>>     when the branched streams are in the same scope. That being said,
>>>>     for the reasons Ivan listed, I think it is overall the better
>>>>     solution - working around the scope thing is easy enough if you need
>>>>     to.
>>>>
>>>>     > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>>>>     <ip...@mail.ru.invalid> wrote:
>>>>     >
>>>>     > Hello everyone, thank you all for joining the discussion!
>>>>     >
>>>>     > Well, I don't think the idea of named branches, be it a
>>>>     LinkedHashMap (no other Map will do, because order of definition
>>>>     matters) or `branch` method  taking name and Consumer has more
>>>>     advantages than drawbacks.
>>>>     >
>>>>     > In my opinion, the only real positive outcome from Michael's
>>>>     proposal is that all the returned branches are in the same scope.
>>>>     But 1) we rarely need them in the same scope 2) there is a
>>>>     workaround for the scope problem, described in the KIP.
>>>>     >
>>>>     > 'Inlining the complex logic' is not a problem, because we can use
>>>>     method references instead of lambdas. In real world scenarios you
>>>>     tend to split the complex logic to methods anyway, so the code is
>>>>     going to be clean.
>>>>     >
>>>>     > The drawbacks are strong. The cohesion between predicates and
>>>>     handlers is lost. We have to define predicates in one place, and
>>>>     handlers in another. This opens the door for bugs:
>>>>     >
>>>>     > - what if we forget to define a handler for a name? or a name for
>>>>     a handler?
>>>>     > - what if we misspell a name?
>>>>     > - what if we copy-paste and duplicate a name?
>>>>     >
>>>>     > What Michael propose would have been totally OK if we had been
>>>>     writing the API in Lua, Ruby or Python. In those languages the
>>>>     "dynamic naming" approach would have looked most concise and
>>>>     beautiful. But in Java we expect all the problems related to
>>>>     identifiers to be eliminated in compile time.
>>>>     >
>>>>     > Do we have to invent duck-typing for the Java API?
>>>>     >
>>>>     > And if we do, what advantage are we supposed to get besides having
>>>>     all the branches in the same scope? Michael, maybe I'm missing your
>>>>     point?
>>>>     >
>>>>     > ---
>>>>     >
>>>>     > Earlier in this discussion John Roesler also proposed to do
>>>>     without "start branching" operator, and later Paul mentioned that in
>>>>     the case when we have to add a dynamic number of branches, the
>>>>     current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
>>>>     me address both comments here.
>>>>     >
>>>>     > 1) "Start branching" operator (I think that *split* is a good name
>>>>     for it indeed) is critical when we need to do a dynamic branching,
>>>>     see example below.
>>>>     >
>>>>     > 2) No, dynamic branching in current KIP is not clumsy at all.
>>>>     Imagine a real-world scenario when you need one branch per enum
>>>>     value (say, RecordType). You can have something like this:
>>>>     >
>>>>     > /*John:if we had to start with stream.branch(...) here, it would
>>>>     have been much messier.*/
>>>>     > KBranchedStream branched = stream.split();
>>>>     >
>>>>     > /*Not clumsy at all :-)*/
>>>>     > for (RecordType recordType : RecordType.values())
>>>>     >             branched = branched.branch((k, v) -> v.getRecType() ==
>>>>     recordType,
>>>>     >                     recordType::processRecords);
>>>>     >
>>>>     > Regards,
>>>>     >
>>>>     > Ivan
>>>>     >
>>>>     >
>>>>     > 02.05.2019 14:40, Matthias J. Sax пишет:
>>>>     >> I also agree with Michael's observation about the core problem of
>>>>     >> current `branch()` implementation.
>>>>     >>
>>>>     >> However, I also don't like to pass in a clumsy Map object. My
>>>>     thinking
>>>>     >> was more aligned with Paul's proposal to just add a name to each
>>>>     >> `branch()` statement and return a `Map<String,KStream>`.
>>>>     >>
>>>>     >> It makes the code easier to read, and also make the order of
>>>>     >> `Predicates` (that is essential) easier to grasp.
>>>>     >>
>>>>     >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>     >>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>     >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>     >>>>>>    .defaultBranch("defaultBranch");
>>>>     >> An open question is the case for which no defaultBranch() should
>>> be
>>>>     >> specified. Atm, `split()` and `branch()` would return
>>>>     `BranchedKStream`
>>>>     >> and the call to `defaultBranch()` that returns the `Map` is
>>> mandatory
>>>>     >> (what is not the case atm). Or is this actually not a real
>>> problem,
>>>>     >> because users can just ignore the branch returned by
>>>>     `defaultBranch()`
>>>>     >> in the result `Map` ?
>>>>     >>
>>>>     >>
>>>>     >> About "inlining": So far, it seems to be a matter of personal
>>>>     >> preference. I can see arguments for both, but no "killer
>>>>     argument" yet
>>>>     >> that clearly make the case for one or the other.
>>>>     >>
>>>>     >>
>>>>     >> -Matthias
>>>>     >>
>>>>     >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>>>     >>> Perhaps inlining is the wrong terminology. It doesn’t require
>>>>     that a lambda with the full downstream topology be defined inline -
>>>>     it can be a method reference as with Ivan’s original suggestion.
>>>>     The advantage of putting the predicate and its downstream logic
>>>>     (Consumer) together in branch() is that they are required to be near
>>>>     to each other.
>>>>     >>>
>>>>     >>> Ultimately the downstream code has to live somewhere, and deep
>>>>     branch trees will be hard to read regardless.
>>>>     >>>
>>>>     >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>>>>     <michael.drogalis@confluent.io
>>>>     <ma...@confluent.io>> wrote:
>>>>     >>>>
>>>>     >>>> I'm less enthusiastic about inlining the branch logic with its
>>>>     downstream
>>>>     >>>> functionality. Programs that have deep branch trees will
>>>>     quickly become
>>>>     >>>> harder to read as a single unit.
>>>>     >>>>
>>>>     >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>>>>     <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>>>>     >>>>>
>>>>     >>>>> Also +1 on the issues/goals as Michael outlined them, I think
>>>>     that sets a
>>>>     >>>>> great framework for the discussion.
>>>>     >>>>>
>>>>     >>>>> Regarding the SortedMap solution, my understanding is that the
>>>>     current
>>>>     >>>>> proposal in the KIP is what is in my PR which (pending naming
>>>>     decisions) is
>>>>     >>>>> roughly this:
>>>>     >>>>>
>>>>     >>>>> stream.split()
>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>     >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>     >>>>>
>>>>     >>>>> Obviously some ordering is necessary, since branching as a
>>>>     construct
>>>>     >>>>> doesn't work without it, but this solution seems like it
>>>>     provides as much
>>>>     >>>>> associativity as the SortedMap solution, because each branch()
>>>>     call
>>>>     >>>>> directly associates the "conditional" with the "code block."
>>>>     The value it
>>>>     >>>>> provides over the KIP solution is the accessing of streams in
>>>>     the same
>>>>     >>>>> scope.
>>>>     >>>>>
>>>>     >>>>> The KIP solution is less "dynamic" than the SortedMap solution
>>>>     in the sense
>>>>     >>>>> that it is slightly clumsier to add a dynamic number of
>>>>     branches, but it is
>>>>     >>>>> certainly possible.  It seems to me like the API should favor
>>>>     the "static"
>>>>     >>>>> case anyway, and should make it simple and readable to
>>>>     fluently declare and
>>>>     >>>>> access your branches in-line.  It also makes it impossible to
>>>>     ignore a
>>>>     >>>>> branch, and it is possible to build an (almost) identical
>>>>     SortedMap
>>>>     >>>>> solution on top of it.
>>>>     >>>>>
>>>>     >>>>> I could also see a middle ground where instead of a raw
>>>>     SortedMap being
>>>>     >>>>> taken in, branch() takes a name and not a Consumer.  Something
>>>>     like this:
>>>>     >>>>>
>>>>     >>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>     >>>>>    .branch("branchOne", Predicate<K, V>)
>>>>     >>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>     >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>>     >>>>>
>>>>     >>>>> Pros for that solution:
>>>>     >>>>> - accessing branched KStreams in same scope
>>>>     >>>>> - no double brace initialization, hopefully slightly more
>>>>     readable than
>>>>     >>>>> SortedMap
>>>>     >>>>>
>>>>     >>>>> Cons
>>>>     >>>>> - downstream branch logic cannot be specified inline which
>>>>     makes it harder
>>>>     >>>>> to read top to bottom (like existing API and SortedMap, but
>>>>     unlike the KIP)
>>>>     >>>>> - you can forget to "handle" one of the branched streams (like
>>>>     existing
>>>>     >>>>> API and SortedMap, but unlike the KIP)
>>>>     >>>>>
>>>>     >>>>> (KBranchedStreams could even work *both* ways but perhaps
>>>>     that's overdoing
>>>>     >>>>> it).
>>>>     >>>>>
>>>>     >>>>> Overall I'm curious how important it is to be able to easily
>>>>     access the
>>>>     >>>>> branched KStream in the same scope as the original.  It's
>>>>     possible that it
>>>>     >>>>> doesn't need to be handled directly by the API, but instead
>>>>     left up to the
>>>>     >>>>> user.  I'm sort of in the middle on it.
>>>>     >>>>>
>>>>     >>>>> Paul
>>>>     >>>>>
>>>>     >>>>>
>>>>     >>>>>
>>>>     >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>>>>     <sophie@confluent.io <ma...@confluent.io>>
>>>>     >>>>> wrote:
>>>>     >>>>>
>>>>     >>>>>> I'd like to +1 what Michael said about the issues with the
>>>>     existing
>>>>     >>>>> branch
>>>>     >>>>>> method, I agree with what he's outlined and I think we should
>>>>     proceed by
>>>>     >>>>>> trying to alleviate these problems. Specifically it seems
>>>>     important to be
>>>>     >>>>>> able to cleanly access the individual branches (eg by mapping
>>>>     >>>>>> name->stream), which I thought was the original intention of
>>>>     this KIP.
>>>>     >>>>>>
>>>>     >>>>>> That said, I don't think we should so easily give in to the
>>>>     double brace
>>>>     >>>>>> anti-pattern or force ours users into it if at all possible to
>>>>     >>>>> avoid...just
>>>>     >>>>>> my two cents.
>>>>     >>>>>>
>>>>     >>>>>> Cheers,
>>>>     >>>>>> Sophie
>>>>     >>>>>>
>>>>     >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>     >>>>>> michael.drogalis@confluent.io
>>>>     <ma...@confluent.io>> wrote:
>>>>     >>>>>>
>>>>     >>>>>>> I’d like to propose a different way of thinking about this.
>>>>     To me,
>>>>     >>>>> there
>>>>     >>>>>>> are three problems with the existing branch signature:
>>>>     >>>>>>>
>>>>     >>>>>>> 1. If you use it the way most people do, Java raises unsafe
>>> type
>>>>     >>>>>> warnings.
>>>>     >>>>>>> 2. The way in which you use the stream branches is
>>>>     positionally coupled
>>>>     >>>>>> to
>>>>     >>>>>>> the ordering of the conditionals.
>>>>     >>>>>>> 3. It is brittle to extend existing branch calls with
>>>>     additional code
>>>>     >>>>>>> paths.
>>>>     >>>>>>>
>>>>     >>>>>>> Using associative constructs instead of relying on ordered
>>>>     constructs
>>>>     >>>>>> would
>>>>     >>>>>>> be a stronger approach. Consider a signature that instead
>>>>     looks like
>>>>     >>>>>> this:
>>>>     >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
>>>>     Predicate<?
>>>>     >>>>>>> super K,? super V>>);
>>>>     >>>>>>>
>>>>     >>>>>>> Branches are given names in a map, and as a result, the API
>>>>     returns a
>>>>     >>>>>>> mapping of names to streams. The ordering of the
>>> conditionals is
>>>>     >>>>>> maintained
>>>>     >>>>>>> because it’s a sorted map. Insert order determines the order
>>> of
>>>>     >>>>>> evaluation.
>>>>     >>>>>>> This solves problem 1 because there are no more varargs. It
>>>>     solves
>>>>     >>>>>> problem
>>>>     >>>>>>> 2 because you no longer lean on ordering to access the
>>>>     branch you’re
>>>>     >>>>>>> interested in. It solves problem 3 because you can introduce
>>>>     another
>>>>     >>>>>>> conditional by simply attaching another name to the
>>>>     structure, rather
>>>>     >>>>>> than
>>>>     >>>>>>> messing with the existing indices.
>>>>     >>>>>>>
>>>>     >>>>>>> One of the drawbacks is that creating the map inline is
>>>>     historically
>>>>     >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>>>>     voluminously, but
>>>>     >>>>>>> double brace initialization would clean up the aesthetics.
>>>>     >>>>>>>
>>>>     >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>>>>     <john@confluent.io <ma...@confluent.io>>
>>>>     >>>>> wrote:
>>>>     >>>>>>>> Hi Ivan,
>>>>     >>>>>>>>
>>>>     >>>>>>>> Thanks for the update.
>>>>     >>>>>>>>
>>>>     >>>>>>>> FWIW, I agree with Matthias that the current "start
>>> branching"
>>>>     >>>>> operator
>>>>     >>>>>>> is
>>>>     >>>>>>>> confusing when named the same way as the actual branches.
>>>>     "Split"
>>>>     >>>>> seems
>>>>     >>>>>>>> like a good name. Alternatively, we can do without a "start
>>>>     >>>>> branching"
>>>>     >>>>>>>> operator at all, and just do:
>>>>     >>>>>>>>
>>>>     >>>>>>>> stream
>>>>     >>>>>>>>      .branch(Predicate)
>>>>     >>>>>>>>      .branch(Predicate)
>>>>     >>>>>>>>      .defaultBranch();
>>>>     >>>>>>>>
>>>>     >>>>>>>> Tentatively, I think that this branching operation should be
>>>>     >>>>> terminal.
>>>>     >>>>>>> That
>>>>     >>>>>>>> way, we don't create ambiguity about how to use it. That
>>>>     is, `branch`
>>>>     >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
>>>>     `void`, to
>>>>     >>>>>>>> enforce that it comes last, and that there is only one
>>>>     definition of
>>>>     >>>>>> the
>>>>     >>>>>>>> default branch. Potentially, we should log a warning if
>>>>     there's no
>>>>     >>>>>>> default,
>>>>     >>>>>>>> and additionally log a warning (or throw an exception) if a
>>>>     record
>>>>     >>>>>> falls
>>>>     >>>>>>>> though with no default.
>>>>     >>>>>>>>
>>>>     >>>>>>>> Thoughts?
>>>>     >>>>>>>>
>>>>     >>>>>>>> Thanks,
>>>>     >>>>>>>> -John
>>>>     >>>>>>>>
>>>>     >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>     >>>>> matthias@confluent.io <ma...@confluent.io>
>>>>     >>>>>>>> wrote:
>>>>     >>>>>>>>
>>>>     >>>>>>>>> Thanks for updating the KIP and your answers.
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>> this is to make the name similar to String#split
>>>>     >>>>>>>>>>> that also returns an array, right?
>>>>     >>>>>>>>> The intend was to avoid name duplication. The return type
>>>>     should
>>>>     >>>>>> _not_
>>>>     >>>>>>>>> be an array.
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> The current proposal is
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> stream.branch()
>>>>     >>>>>>>>>      .branch(Predicate)
>>>>     >>>>>>>>>      .branch(Predicate)
>>>>     >>>>>>>>>      .defaultBranch();
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> IMHO, this reads a little odd, because the first
>>>>     `branch()` does
>>>>     >>>>> not
>>>>     >>>>>>>>> take any parameters and has different semantics than the
>>> later
>>>>     >>>>>>>>> `branch()` calls. Note, that from the code snippet above,
>>> it's
>>>>     >>>>> hidden
>>>>     >>>>>>>>> that the first call is `KStream#branch()` while the others
>>> are
>>>>     >>>>>>>>> `KBranchedStream#branch()` what makes reading the code
>>> harder.
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
>>>>     I though
>>>>     >>>>>> it
>>>>     >>>>>>>>> might be better to also rename `KStream#branch()` to avoid
>>> the
>>>>     >>>>> naming
>>>>     >>>>>>>>> overlap that seems to be confusing. The following reads
>>> much
>>>>     >>>>> cleaner
>>>>     >>>>>> to
>>>>     >>>>>>>> me:
>>>>     >>>>>>>>> stream.split()
>>>>     >>>>>>>>>      .branch(Predicate)
>>>>     >>>>>>>>>      .branch(Predicate)
>>>>     >>>>>>>>>      .defaultBranch();
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> Maybe there is a better alternative to `split()` though to
>>>>     avoid
>>>>     >>>>> the
>>>>     >>>>>>>>> naming overlap.
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately
>>> we
>>>>     >>>>> cannot
>>>>     >>>>>>> have
>>>>     >>>>>>>>> a method with such name :-)
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
>>>>     with a
>>>>     >>>>> short
>>>>     >>>>>>>> name?
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
>>>>     with all
>>>>     >>>>> it's
>>>>     >>>>>>>>> methods? It will be part of public API and should be
>>>>     contained in
>>>>     >>>>> the
>>>>     >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>     >>>>>>>>> `defaultBranch()` is.
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> You did not comment on the idea to add a
>>>>     `KBranchedStream#get(int
>>>>     >>>>>>> index)
>>>>     >>>>>>>>> -> KStream` method to get the individually
>>>>     branched-KStreams. Would
>>>>     >>>>>> be
>>>>     >>>>>>>>> nice to get your feedback about it. It seems you suggest
>>>>     that users
>>>>     >>>>>>>>> would need to write custom utility code otherwise, to
>>>>     access them.
>>>>     >>>>> We
>>>>     >>>>>>>>> should discuss the pros and cons of both approaches. It
>>> feels
>>>>     >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
>>>>     to get
>>>>     >>>>> the
>>>>     >>>>>>>>> branched-KStreams directly.
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>
>>>>     >>>>>>>>> -Matthias
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>
>>>>     >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>     >>>>>>>>>> Hi all!
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> Matthias, thanks for your comment!
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>     >>>>>>>>>> I can see your point: this is to make the name similar to
>>>>     >>>>>>> String#split
>>>>     >>>>>>>>>> that also returns an array, right? But is it worth the
>>>>     loss of
>>>>     >>>>>>>> backwards
>>>>     >>>>>>>>>> compatibility? We can have overloaded branch() as well
>>>>     without
>>>>     >>>>>>>> affecting
>>>>     >>>>>>>>>> the existing code. Maybe the old array-based `branch`
>>> method
>>>>     >>>>> should
>>>>     >>>>>>> be
>>>>     >>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>     >>>>> BranchingKStream#branch(),
>>>>     >>>>>>>>>> KBranchedStream#defaultBranch() ->
>>> BranchingKStream#default()
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default'
>>> is,
>>>>     >>>>>>> however, a
>>>>     >>>>>>>>>> reserved word, so unfortunately we cannot have a method
>>>>     with such
>>>>     >>>>>>> name
>>>>     >>>>>>>>> :-)
>>>>     >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument,
>>> but I
>>>>     >>>>> think
>>>>     >>>>>>> that
>>>>     >>>>>>>>>> is not required?
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> Absolutely! I think that was just copy-paste error or
>>>>     something.
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> Dear colleagues,
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>     >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> Any new suggestions/objections?
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> Regards,
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> Ivan
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>>
>>>>     >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>     >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems
>>> that
>>>>     >>>>>>> everybody
>>>>     >>>>>>>>>>> agrees that the current branch() method using arrays is
>>> not
>>>>     >>>>>> optimal.
>>>>     >>>>>>>>>>> I had a quick look into the PR and I like the overall
>>>>     proposal.
>>>>     >>>>>>> There
>>>>     >>>>>>>>>>> are some minor things we need to consider. I would
>>>>     recommend the
>>>>     >>>>>>>>>>> following renaming:
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>> KStream#branch() -> #split()
>>>>     >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>     >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>>>>     BranchingKStream#default()
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>> It's just a suggestion to get slightly shorter method
>>> names.
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>> In the current PR, defaultBranch() does take an
>>>>     `Predicate` as
>>>>     >>>>>>>> argument,
>>>>     >>>>>>>>>>> but I think that is not required?
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>> Also, we should consider KIP-307, that was recently
>>>>     accepted and
>>>>     >>>>>> is
>>>>     >>>>>>>>>>> currently implemented:
>>>>     >>>>>>>>>>>
>>>>     >>>>>
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>     >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
>>>>     parameter.
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>> For the issue that the created `KStream` object are in
>>>>     different
>>>>     >>>>>>>> scopes:
>>>>     >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>>>>     index)` method
>>>>     >>>>>>> that
>>>>     >>>>>>>>>>> returns the corresponding "branched" result `KStream`
>>>>     object?
>>>>     >>>>>> Maybe,
>>>>     >>>>>>>> the
>>>>     >>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>     >>>>>> `Consumer<KStream>`
>>>>     >>>>>>>> but
>>>>     >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
>>>>     whatever
>>>>     >>>>>> the
>>>>     >>>>>>>>>>> `Function` returns?
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
>>>>     current
>>>>     >>>>>>>>>>> proposal. That makes it easier to review.
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>> -Matthias
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>>
>>>>     >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>     >>>>>>>>>>>> Ivan,
>>>>     >>>>>>>>>>>>
>>>>     >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
>>>>     makes sense
>>>>     >>>>>> for
>>>>     >>>>>>>> you
>>>>     >>>>>>>>> to
>>>>     >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
>>>>     we'll
>>>>     >>>>> need
>>>>     >>>>>>>> some
>>>>     >>>>>>>>>>>> buy-in from committers that have actual binding votes on
>>>>     >>>>> whether
>>>>     >>>>>>> the
>>>>     >>>>>>>>> KIP
>>>>     >>>>>>>>>>>> could be adopted.  It would be great to hear if they
>>>>     think this
>>>>     >>>>>> is
>>>>     >>>>>>> a
>>>>     >>>>>>>>> good
>>>>     >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
>>>>     starting a
>>>>     >>>>>>> vote,
>>>>     >>>>>>>>> or if
>>>>     >>>>>>>>>>>> there is generally some indication of interest
>>> beforehand.
>>>>     >>>>>>>>>>>>
>>>>     >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
>>>>     assuming
>>>>     >>>>> we
>>>>     >>>>>> do
>>>>     >>>>>>>>> move
>>>>     >>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>     >>>>>> KBranchedStream",
>>>>     >>>>>>> do
>>>>     >>>>>>>>> we
>>>>     >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I
>>> would
>>>>     >>>>> favor
>>>>     >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs
>>> that
>>>>     >>>>>>> accomplish
>>>>     >>>>>>>>> the
>>>>     >>>>>>>>>>>> same thing is confusing, especially when they're fairly
>>>>     similar
>>>>     >>>>>>>>> anyway.  We
>>>>     >>>>>>>>>>>> just need to be sure we're not making something
>>>>     >>>>>>> impossible/difficult
>>>>     >>>>>>>>> that
>>>>     >>>>>>>>>>>> is currently possible/easy.
>>>>     >>>>>>>>>>>>
>>>>     >>>>>>>>>>>> Regarding my PR - I think the general structure would
>>> work,
>>>>     >>>>> it's
>>>>     >>>>>>>> just a
>>>>     >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>     >>>>>>> particular,
>>>>     >>>>>>>>>>>> passing in the "predicates" and "children" lists which
>>> get
>>>>     >>>>>> modified
>>>>     >>>>>>>> in
>>>>     >>>>>>>>>>>> KBranchedStream but read from all the way
>>>>     KStreamLazyBranch is
>>>>     >>>>> a
>>>>     >>>>>>> bit
>>>>     >>>>>>>>>>>> complicated to follow.
>>>>     >>>>>>>>>>>>
>>>>     >>>>>>>>>>>> Thanks,
>>>>     >>>>>>>>>>>> Paul
>>>>     >>>>>>>>>>>>
>>>>     >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>     >>>>>>>>> wrote:
>>>>     >>>>>>>>>>>>> Hi Paul!
>>>>     >>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>> I read your code carefully and now I am fully
>>>>     convinced: your
>>>>     >>>>>>>> proposal
>>>>     >>>>>>>>>>>>> looks better and should work. We just have to document
>>> the
>>>>     >>>>>> crucial
>>>>     >>>>>>>>> fact
>>>>     >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
>>>>     And then
>>>>     >>>>>> it's
>>>>     >>>>>>>> all
>>>>     >>>>>>>>>>>>> going to be very nice.
>>>>     >>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
>>>>     resume the
>>>>     >>>>>>>>>>>>> discussion here, right?
>>>>     >>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>>>     >>>>> starting
>>>>     >>>>>>>> point
>>>>     >>>>>>>>> if
>>>>     >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
>>>>     starting
>>>>     >>>>>>> point.
>>>>     >>>>>>>>> But
>>>>     >>>>>>>>>>>>> as a novice in this project I might miss some important
>>>>     >>>>> details.
>>>>     >>>>>>>>>>>>> Regards,
>>>>     >>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>> Ivan
>>>>     >>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>     >>>>>>>>>>>>>> Ivan,
>>>>     >>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>     >>>>> stream.branch()
>>>>     >>>>>>>>> solution
>>>>     >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>>>     >>>>> invoked
>>>>     >>>>>> as
>>>>     >>>>>>>>> they’re
>>>>     >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
>>>>     still
>>>>     >>>>>> ought
>>>>     >>>>>>> to
>>>>     >>>>>>>>> be
>>>>     >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
>>>>     depend on
>>>>     >>>>> the
>>>>     >>>>>>>>> branched
>>>>     >>>>>>>>>>>>> streams having been set.
>>>>     >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
>>>>     access
>>>>     >>>>> the
>>>>     >>>>>>>>> branched
>>>>     >>>>>>>>>>>>> streams in the same scope as the original stream (that
>>>>     is, not
>>>>     >>>>>>>> inside
>>>>     >>>>>>>>> the
>>>>     >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>>>     >>>>> solutions.
>>>>     >>>>>> It
>>>>     >>>>>>>>> can be
>>>>     >>>>>>>>>>>>> worked around though.
>>>>     >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
>>>>     excited
>>>>     >>>>> to
>>>>     >>>>>>>> hear
>>>>     >>>>>>>>>>>>> your thoughts!]
>>>>     >>>>>>>>>>>>>> Paul
>>>>     >>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>     >>>>>>>>> wrote:
>>>>     >>>>>>>>>>>>>>> Hi Paul!
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>     >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at
>>> first
>>>>     >>>>> glance,
>>>>     >>>>>>> but
>>>>     >>>>>>>>> ---
>>>>     >>>>>>>>>>>>>>>> the newly branched streams are not available in the
>>>>     same
>>>>     >>>>>> scope
>>>>     >>>>>>> as
>>>>     >>>>>>>>> each
>>>>     >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
>>> together
>>>>     >>>>> again
>>>>     >>>>>> I
>>>>     >>>>>>>>> don't see
>>>>     >>>>>>>>>>>>> a way to do that.
>>>>     >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
>>>>     just
>>>>     >>>>>> going
>>>>     >>>>>>> to
>>>>     >>>>>>>>>>>>> write in details about this issue.
>>>>     >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
>>>>     we need
>>>>     >>>>> to
>>>>     >>>>>>>>> identify
>>>>     >>>>>>>>>>>>> customers who have bought coffee and made a purchase
>>>>     in the
>>>>     >>>>>>>>> electronics
>>>>     >>>>>>>>>>>>> store to give them coupons.
>>>>     >>>>>>>>>>>>>>> This is the code I usually write under these
>>>>     circumstances
>>>>     >>>>>> using
>>>>     >>>>>>>> my
>>>>     >>>>>>>>>>>>> 'brancher' class:
>>>>     >>>>>>>>>>>>>>> @Setter
>>>>     >>>>>>>>>>>>>>> class CouponIssuer{
>>>>     >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>     >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>     >>>>>>>>>>>>>>>       return
>>>>     >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>     >>>>>>>>>>>>>>>       /*In the real world the code here can be
>>>>     complex, so
>>>>     >>>>>>>>> creation of
>>>>     >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
>>>>     order to
>>>>     >>>>>>>> separate
>>>>     >>>>>>>>>>>>> classes' responsibilities.*/
>>>>     >>>>>>>>>>>>>>>  }
>>>>     >>>>>>>>>>>>>>> }
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>     >>>>>>>>>>>>>>>     .branch(predicate1,
>>> couponIssuer::setCoffePurchases)
>>>>     >>>>>>>>>>>>>>>     .branch(predicate2,
>>>>     >>>>>> couponIssuer::setElectronicsPurchases)
>>>>     >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
>>>>     everything
>>>>     >>>>>>>> later,
>>>>     >>>>>>>>>>>>> without the terminal operation!!!*/
>>>>     >>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>>>>     initialize the
>>>>     >>>>>>>>> CouponIssuer
>>>>     >>>>>>>>>>>>> we need the terminal operation to be called before
>>>>     >>>>>>>>> streamsBuilder.build()
>>>>     >>>>>>>>>>>>> is called.
>>>>     >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
>>>>     essentially
>>>>     >>>>>> the
>>>>     >>>>>>>>> next
>>>>     >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
>>>>     based on
>>>>     >>>>> my
>>>>     >>>>>>>>> experience,
>>>>     >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>     >>>>>>>>>>>>>>> Regards,
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> Ivan
>>>>     >>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>     >>>>>>>>>>>>>>>> Ivan,
>>>>     >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
>>>>     fluent
>>>>     >>>>> API
>>>>     >>>>>>>> based
>>>>     >>>>>>>>>>>>> off of
>>>>     >>>>>>>>>>>>>>>> KStream here
>>>>     (https://github.com/apache/kafka/pull/6512),
>>>>     >>>>>> and
>>>>     >>>>>>> I
>>>>     >>>>>>>>> think
>>>>     >>>>>>>>>>>>> I
>>>>     >>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>     >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>>     >>>>>>> compatibility
>>>>     >>>>>>>>>>>>> issues,
>>>>     >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
>>>>     that Java
>>>>     >>>>> is
>>>>     >>>>>>>> smart
>>>>     >>>>>>>>>>>>> enough to
>>>>     >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>>>>     returning one
>>>>     >>>>>>> thing
>>>>     >>>>>>>>> and
>>>>     >>>>>>>>>>>>> branch()
>>>>     >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>     >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
>>>>     need
>>>>     >>>>> it.
>>>>     >>>>>>> We
>>>>     >>>>>>>>> can
>>>>     >>>>>>>>>>>>> just
>>>>     >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
>>>>     shares
>>>>     >>>>>> its
>>>>     >>>>>>>>> state
>>>>     >>>>>>>>>>>>> with the
>>>>     >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
>>>>     branching.
>>>>     >>>>>>> It's
>>>>     >>>>>>>>> not
>>>>     >>>>>>>>>>>>> terribly
>>>>     >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>>>>     demonstrates
>>>>     >>>>>> its
>>>>     >>>>>>>>>>>>> feasibility.
>>>>     >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should
>>> be
>>>>     >>>>> final
>>>>     >>>>>> or
>>>>     >>>>>>>>> even a
>>>>     >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
>>>>     wanted to
>>>>     >>>>>> see
>>>>     >>>>>>>> how
>>>>     >>>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>     >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
>>>>     solution
>>>>     >>>>>>> could
>>>>     >>>>>>>> be
>>>>     >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>>>     >>>>> suggested
>>>>     >>>>>>>> was a
>>>>     >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
>>>>     streams
>>>>     >>>>>> are
>>>>     >>>>>>>> not
>>>>     >>>>>>>>>>>>>>>> available in the same scope as each other.  That
>>>>     is, if we
>>>>     >>>>>>> wanted
>>>>     >>>>>>>>> to
>>>>     >>>>>>>>>>>>> merge
>>>>     >>>>>>>>>>>>>>>> them back together again I don't see a way to do
>>>>     that.  The
>>>>     >>>>>> KIP
>>>>     >>>>>>>>>>>>> proposal
>>>>     >>>>>>>>>>>>>>>> has the same issue, though - all this means is that
>>> for
>>>>     >>>>>> either
>>>>     >>>>>>>>>>>>> solution,
>>>>     >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
>>>>     table.
>>>>     >>>>>>>>>>>>>>>> Thanks,
>>>>     >>>>>>>>>>>>>>>> Paul
>>>>     >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>>>>     >>>>>>>>>>>>> wrote:
>>>>     >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to
>>> this
>>>>     >>>>>> point.
>>>>     >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
>>>>     branch API
>>>>     >>>>>>> needs
>>>>     >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>     >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
>>>>     returns
>>>>     >>>>>> its
>>>>     >>>>>>>>> argument
>>>>     >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code
>>> won't
>>>>     >>>>> make
>>>>     >>>>>>>> sense
>>>>     >>>>>>>>>>>>> until
>>>>     >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
>>>>     instance
>>>>     >>>>>>>>> contrasts the
>>>>     >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> stream
>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>     >>>>>>>>> defaultBranch(..)
>>>>     >>>>>>>>>>>>> and
>>>>     >>>>>>>>>>>>>>>>> noDefault() return void
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface
>>> is
>>>>     >>>>>> defined.
>>>>     >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>     >>>>>>>> (defaultBranch(ks->)
>>>>     >>>>>>>>> and
>>>>     >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
>>>>     miss the
>>>>     >>>>>> fact
>>>>     >>>>>>>>> that one
>>>>     >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
>>>>     methods
>>>>     >>>>>> are
>>>>     >>>>>>>> not
>>>>     >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do
>>> better?
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> Regards,
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> Ivan
>>>>     >>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>     >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>     >>>>>>>>>>>>>>>>>>> Paul,
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>     >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>>>>     implemented the
>>>>     >>>>>>> easy
>>>>     >>>>>>>>> way.
>>>>     >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
>>> assumes
>>>>     >>>>> nothing
>>>>     >>>>>>>> will
>>>>     >>>>>>>>>>>>> reach
>>>>     >>>>>>>>>>>>>>>>>>>> the default branch,
>>>>     >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only
>>> option
>>>>     >>>>>> besides
>>>>     >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
>>>>     want to
>>>>     >>>>>> just
>>>>     >>>>>>>>> silently
>>>>     >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>>>>     predicate. 2)
>>>>     >>>>>>> Throwing
>>>>     >>>>>>>>> an
>>>>     >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
>>>>     looks
>>>>     >>>>>> like a
>>>>     >>>>>>>> bad
>>>>     >>>>>>>>>>>>> idea.
>>>>     >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
>>>>     emit a
>>>>     >>>>>>>> special
>>>>     >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly
>>> where
>>>>     >>>>>>> `default`
>>>>     >>>>>>>>> can
>>>>     >>>>>>>>>>>>> be
>>>>     >>>>>>>>>>>>>>>>>>> used.
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>     >>>>> InternalTopologyBuilder
>>>>     >>>>>>> to
>>>>     >>>>>>>>> track
>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>     >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>     a clear
>>>>     >>>>>>> error
>>>>     >>>>>>>>>>>>> before it
>>>>     >>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>>>     >>>>> compiled
>>>>     >>>>>>> and
>>>>     >>>>>>>>> run?
>>>>     >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>>>>     compile if
>>>>     >>>>> used
>>>>     >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>>>>     method chain
>>>>     >>>>>>>> starting
>>>>     >>>>>>>>>>>>> from
>>>>     >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
>>>>     between
>>>>     >>>>>>>> runtime
>>>>     >>>>>>>>> and
>>>>     >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>>>     >>>>> instantly
>>>>     >>>>>> on
>>>>     >>>>>>>>> unit
>>>>     >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
>>>>     compilation
>>>>     >>>>>>>> failure.
>>>>     >>>>>>>>>>>>>>>>>>> Regards,
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>> Ivan
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>     >>>>>>>>>>>>>>>>>>>> Ivan,
>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
>>>>     required.
>>>>     >>>>>>> But
>>>>     >>>>>>>> is
>>>>     >>>>>>>>>>>>> that
>>>>     >>>>>>>>>>>>>>>>>>>> really
>>>>     >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>>     >>>>>> defaultBranch
>>>>     >>>>>>>>> they
>>>>     >>>>>>>>>>>>> can
>>>>     >>>>>>>>>>>>>>>>>>>> call
>>>>     >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
>>>>     just as
>>>>     >>>>>>>>> easily.  In
>>>>     >>>>>>>>>>>>>>>>>>>> fact I
>>>>     >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API
>>> - a
>>>>     >>>>> user
>>>>     >>>>>>>> could
>>>>     >>>>>>>>>>>>> specify
>>>>     >>>>>>>>>>>>>>>>> a
>>>>     >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach
>>> the
>>>>     >>>>>> default
>>>>     >>>>>>>>> branch,
>>>>     >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>> That
>>>>     >>>>> seems
>>>>     >>>>>>> like
>>>>     >>>>>>>>> an
>>>>     >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
>>>>     which allows
>>>>     >>>>>> for
>>>>     >>>>>>>> the
>>>>     >>>>>>>>>>>>> more
>>>>     >>>>>>>>>>>>>>>>>>>> subtle
>>>>     >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
>>> dropped.
>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
>>>>     to be
>>>>     >>>>>> well
>>>>     >>>>>>>>>>>>>>>>>>>> documented, but
>>>>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>     >>>>> InternalTopologyBuilder
>>>>     >>>>>>> to
>>>>     >>>>>>>>> track
>>>>     >>>>>>>>>>>>>>>>>>>> dangling
>>>>     >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>>>>     a clear
>>>>     >>>>>>> error
>>>>     >>>>>>>>>>>>> before it
>>>>     >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is
>>> a
>>>>     >>>>> "build
>>>>     >>>>>>>> step"
>>>>     >>>>>>>>>>>>> where
>>>>     >>>>>>>>>>>>>>>>> the
>>>>     >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>     >>>>>> StreamsBuilder.build()
>>>>     >>>>>>> is
>>>>     >>>>>>>>>>>>> called.
>>>>     >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I
>>> agree
>>>>     >>>>> that
>>>>     >>>>>>> it's
>>>>     >>>>>>>>>>>>>>>>>>>> critical to
>>>>     >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
>>>>     stream.
>>>>     >>>>>>> With
>>>>     >>>>>>>>> the
>>>>     >>>>>>>>>>>>>>>>> fluent
>>>>     >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all
>>> other
>>>>     >>>>>>> operations
>>>>     >>>>>>>>> do -
>>>>     >>>>>>>>>>>>> if
>>>>     >>>>>>>>>>>>>>>>> you
>>>>     >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
>>> multiple
>>>>     >>>>> times,
>>>>     >>>>>>> you
>>>>     >>>>>>>>> just
>>>>     >>>>>>>>>>>>>>>>>>>> need the
>>>>     >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
>>>>     operations
>>>>     >>>>>> on
>>>>     >>>>>>> it
>>>>     >>>>>>>>> as
>>>>     >>>>>>>>>>>>> you
>>>>     >>>>>>>>>>>>>>>>>>>> desire.
>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>> Best,
>>>>     >>>>>>>>>>>>>>>>>>>> Paul
>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>>>>     >>>>>>>>>>>>>>>>>>>> wrote:
>>>>     >>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
>>>>     always need
>>>>     >>>>>> the
>>>>     >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
>>> operation we
>>>>     >>>>> don't
>>>>     >>>>>>>> know
>>>>     >>>>>>>>>>>>> when to
>>>>     >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
>>>>     so we
>>>>     >>>>> can
>>>>     >>>>>> do
>>>>     >>>>>>>>>>>>> something
>>>>     >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
>>> special
>>>>     >>>>> object
>>>>     >>>>>>>>>>>>> construction
>>>>     >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods.
>>> But
>>>>     >>>>> here
>>>>     >>>>>> we
>>>>     >>>>>>>>> have a
>>>>     >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
>>>>     flow,
>>>>     >>>>> so
>>>>     >>>>>> I
>>>>     >>>>>>>>> think
>>>>     >>>>>>>>>>>>> this
>>>>     >>>>>>>>>>>>>>>>> is
>>>>     >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>> Regards,
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>> Ivan
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>     >>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
>>>>     API, but I
>>>>     >>>>>> find
>>>>     >>>>>>>> the
>>>>     >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>     >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>>>>     contrasts the
>>>>     >>>>>>> fluency
>>>>     >>>>>>>>> of
>>>>     >>>>>>>>>>>>> other
>>>>     >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
>>>>     just call
>>>>     >>>>> a
>>>>     >>>>>>>>> method on
>>>>     >>>>>>>>>>>>> the
>>>>     >>>>>>>>>>>>>>>>>>>>> stream
>>>>     >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
>>>>     cases
>>>>     >>>>> are
>>>>     >>>>>>>>> defined
>>>>     >>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>     >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
>>>>     is very
>>>>     >>>>>> nice
>>>>     >>>>>>>>> and the
>>>>     >>>>>>>>>>>>>>>>>>>>>> right
>>>>     >>>>>>>>>>>>>>>>>>>>> way
>>>>     >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
>>>>     how we
>>>>     >>>>>>> specify
>>>>     >>>>>>>>> the
>>>>     >>>>>>>>>>>>> source
>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>> Like:
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
>>> this::handle1)
>>>>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
>>> this::handle2)
>>>>     >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>     >>>>>>>> KStreamBrancher
>>>>     >>>>>>>>> or
>>>>     >>>>>>>>>>>>>>>>>>>>> something,
>>>>     >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>>>>     terminated by
>>>>     >>>>>>>>>>>>> defaultBranch()
>>>>     >>>>>>>>>>>>>>>>>>>>>> (which
>>>>     >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>>>>     incompatible with
>>>>     >>>>> the
>>>>     >>>>>>>>> current
>>>>     >>>>>>>>>>>>>>>>>>>>>> API, so
>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>     >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
>>>>     different
>>>>     >>>>>> name,
>>>>     >>>>>>>> but
>>>>     >>>>>>>>> that
>>>>     >>>>>>>>>>>>>>>>>>>>>> seems
>>>>     >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>>     >>>>>> something
>>>>     >>>>>>>> like
>>>>     >>>>>>>>>>>>>>>>>>>>>> branched()
>>>>     >>>>>>>>>>>>>>>>>>>>> or
>>>>     >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
>>>>     KIP?  It
>>>>     >>>>>> seems
>>>>     >>>>>>>>> like it
>>>>     >>>>>>>>>>>>>>>>>>>>>> does to
>>>>     >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
>>>>     while also
>>>>     >>>>>>>> allowing
>>>>     >>>>>>>>> you
>>>>     >>>>>>>>>>>>> to
>>>>     >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>>>>     KBranchedStreams
>>>>     >>>>>> if
>>>>     >>>>>>>>> desired.
>>>>     >>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>     >>>>>>>>>>>>>>>>>>>>>> Paul
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>     >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>     >>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>     >>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
>>>>     ks){
>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>>>>     String> ks){
>>>>     >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>     >>>>>>>>>>>>>>>>>>>>>>> }
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> ......
>>>>     >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>>>>     this::handleFirstCase)
>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>>>>     this::handleSecondCase)
>>>>     >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>     >>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
>>> KafkaStreamsBrancher
>>>>     >>>>> takes a
>>>>     >>>>>>>>> Consumer
>>>>     >>>>>>>>>>>>> as a
>>>>     >>>>>>>>>>>>>>>>>>>>>>> second
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
>>>>     example in
>>>>     >>>>>> the
>>>>     >>>>>>>> KIP
>>>>     >>>>>>>>>>>>> shows
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> each
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>     >>>>>>>>> (KafkaStreams#to()
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would
>>> we
>>>>     >>>>> handle
>>>>     >>>>>>> the
>>>>     >>>>>>>>> case
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to
>>> continue
>>>>     >>>>>>>> processing
>>>>     >>>>>>>>> and
>>>>     >>>>>>>>>>>>> not
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the
>>> branched
>>>>     >>>>>> stream
>>>>     >>>>>>>>>>>>> immediately?
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
>>>>     we had
>>>>     >>>>>>>> something
>>>>     >>>>>>>>> like
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck
>>> <
>>>>     >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for
>>> KIP-
>>>>     >>>>> 418.
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
>>> KIP-418.
>>>>     >>>>> Please
>>>>     >>>>>>>> take
>>>>     >>>>>>>>> a
>>>>     >>>>>>>>>>>>> look
>>>>     >>>>>>>>>>>>>>>>> at
>>>>     >>>>>>>>>>>>>>>>>>>>> the
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
>>>>     feedback :)
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>     >>>>>
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>     >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>     >>>>> https://github.com/apache/kafka/pull/6164
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>>>>>>>>>>>>>>>>>
>>>>     >>>>>>>>>
>>>>     >
>>>>
>>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <jo...@confluent.io>.
Thanks for the idea, Matthias, it does seem like this would satisfy
everyone. Returning the map from the terminal operations also solves
the problem of merging/joining the branched streams, if we want to add
support for the compliment later on.

Under your suggestion, it seems that the name is required. Otherwise,
we wouldn't have keys for the map to return. I this this is actually
not too bad, since experience has taught us that, although names for
operations are not required to define stream processing logic, it does
significantly improve the operational experience when you can map the
topology, logs, metrics, etc. back to the source code. Since you
wouldn't (have to) reference the name to chain extra processing onto
the branch (thanks to the second argument), you can avoid the
"unchecked name" problem that Ivan pointed out.

In the current implementation of Branch, you can name the branch
operator itself, and then all the branches get index-suffixed names
built from the branch operator name. I guess under this proposal, we
could naturally append the branch name to the branching operator name,
like this:

   stream.split(Named.withName("mysplit")) //creates node "mysplit"
              .branch(..., ..., "abranch") // creates node "mysplit-abranch"
              .defaultBranch(...) // creates node "mysplit-default"

It does make me wonder about the DSL syntax itself, though.

We don't have a defined grammar, so there's plenty of room to debate
the "best" syntax in the context of each operation, but in general,
the KStream DSL operators follow this pattern:

    operator(function, config_object?) OR operator(config_object)

where config_object is often just Named in the "function" variant.
Even when the config_object isn't a Named, but some other config
class, that config class _always_ implements NamedOperation.

Here, we're introducing a totally different pattern:

  operator(function, function, string)

where the string is the name.
My first question is whether the name should instead be specified with
the NamedOperation interface.

My second question is whether we should just roll all these arguments
up into a config object like:

   KBranchedStream#branch(BranchConfig)

   interface BranchConfig extends NamedOperation {
    withPredicate(...);
    withChain(...);
    withName(...);
  }

Although I guess we'd like to call BranchConfig something more like
"Branched", even if I don't particularly like that pattern.

This makes the source code a little noisier, but it also makes us more
future-proof, as we can deal with a wide range of alternatives purely
in the config interface, and never have to deal with adding overloads
to the KBranchedStream if/when we decide we want the name to be
optional, or the KStream->KStream to be optional.

WDYT?

Thanks,
-John

On Fri, May 24, 2019 at 5:25 PM Michael Drogalis
<mi...@confluent.io> wrote:
>
> Matthias: I think that's pretty reasonable from my point of view. Good
> suggestion.
>
> On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > Interesting discussion.
> >
> > I am wondering, if we cannot unify the advantage of both approaches:
> >
> >
> >
> > KStream#split() -> KBranchedStream
> >
> > // branch is not easily accessible in current scope
> > KBranchedStream#branch(Predicate, Consumer<KStream>)
> >   -> KBranchedStream
> >
> > // assign a name to the branch and
> > // return the sub-stream to the current scope later
> > //
> > // can be simple as `#branch(p, s->s, "name")`
> > // or also complex as `#branch(p, s->s.filter(...), "name")`
> > KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
> >   -> KBranchedStream
> >
> > // default branch is not easily accessible
> > // return map of all named sub-stream into current scope
> > KBranchedStream#default(Cosumer<KStream>)
> >   -> Map<String,KStream>
> >
> > // assign custom name to default-branch
> > // return map of all named sub-stream into current scope
> > KBranchedStream#default(Function<KStream,KStream>, String)
> >   -> Map<String,KStream>
> >
> > // assign a default name for default
> > // return map of all named sub-stream into current scope
> > KBranchedStream#defaultBranch(Function<KStream,KStream>)
> >   -> Map<String,KStream>
> >
> > // return map of all names sub-stream into current scope
> > KBranchedStream#noDefaultBranch()
> >   -> Map<String,KStream>
> >
> >
> >
> > Hence, for each sub-stream, the user can pick to add a name and return
> > the branch "result" to the calling scope or not. The implementation can
> > also check at runtime that all returned names are unique. The returned
> > Map can be empty and it's also optional to use the Map.
> >
> > To me, it seems like a good way to get best of both worlds.
> >
> > Thoughts?
> >
> >
> >
> > -Matthias
> >
> >
> >
> >
> > On 5/6/19 5:15 PM, John Roesler wrote:
> > > Ivan,
> > >
> > > That's a very good point about the "start" operator in the dynamic case.
> > > I had no problem with "split()"; I was just questioning the necessity.
> > > Since you've provided a proof of necessity, I'm in favor of the
> > > "split()" start operator. Thanks!
> > >
> > > Separately, I'm interested to see where the present discussion leads.
> > > I've written enough Javascript code in my life to be suspicious of
> > > nested closures. You have a good point about using method references (or
> > > indeed function literals also work). It should be validating that this
> > > was also the JS community's first approach to flattening the logic when
> > > their nested closure situation got out of hand. Unfortunately, it's
> > > replacing nesting with redirection, both of which disrupt code
> > > readability (but in different ways for different reasons). In other
> > > words, I agree that function references is *the* first-order solution if
> > > the nested code does indeed become a problem.
> > >
> > > However, the history of JS also tells us that function references aren't
> > > the end of the story either, and you can see that by observing that
> > > there have been two follow-on eras, as they continue trying to cope with
> > > the consequences of living in such a callback-heavy language. First, you
> > > have Futures/Promises, which essentially let you convert nested code to
> > > method-chained code (Observables/FP is a popular variation on this).
> > > Most lately, you have async/await, which is an effort to apply language
> > > (not just API) syntax to the problem, and offer the "flattest" possible
> > > programming style to solve the problem (because you get back to just one
> > > code block per functional unit).
> > >
> > > Stream-processing is a different domain, and Java+KStreams is nowhere
> > > near as callback heavy as JS, so I don't think we have to take the JS
> > > story for granted, but then again, I think we can derive some valuable
> > > lessons by looking sideways to adjacent domains. I'm just bringing this
> > > up to inspire further/deeper discussion. At the same time, just like JS,
> > > we can afford to take an iterative approach to the problem.
> > >
> > > Separately again, I'm interested in the post-branch merge (and I'd also
> > > add join) problem that Paul brought up. We can clearly punt on it, by
> > > terminating the nested branches with sink operators. But is there a DSL
> > > way to do it?
> > >
> > > Thanks again for your driving this,
> > > -John
> > >
> > > On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
> > > <ma...@gmail.com>> wrote:
> > >
> > >     Ivan, I’ll definitely forfeit my point on the clumsiness of the
> > >     branch(predicate, consumer) solution, I don’t see any real drawbacks
> > >     for the dynamic case.
> > >
> > >     IMO the one trade off to consider at this point is the scope
> > >     question. I don’t know if I totally agree that “we rarely need them
> > >     in the same scope” since merging the branches back together later
> > >     seems like a perfectly plausible use case that can be a lot nicer
> > >     when the branched streams are in the same scope. That being said,
> > >     for the reasons Ivan listed, I think it is overall the better
> > >     solution - working around the scope thing is easy enough if you need
> > >     to.
> > >
> > >     > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> > >     <ip...@mail.ru.invalid> wrote:
> > >     >
> > >     > Hello everyone, thank you all for joining the discussion!
> > >     >
> > >     > Well, I don't think the idea of named branches, be it a
> > >     LinkedHashMap (no other Map will do, because order of definition
> > >     matters) or `branch` method  taking name and Consumer has more
> > >     advantages than drawbacks.
> > >     >
> > >     > In my opinion, the only real positive outcome from Michael's
> > >     proposal is that all the returned branches are in the same scope.
> > >     But 1) we rarely need them in the same scope 2) there is a
> > >     workaround for the scope problem, described in the KIP.
> > >     >
> > >     > 'Inlining the complex logic' is not a problem, because we can use
> > >     method references instead of lambdas. In real world scenarios you
> > >     tend to split the complex logic to methods anyway, so the code is
> > >     going to be clean.
> > >     >
> > >     > The drawbacks are strong. The cohesion between predicates and
> > >     handlers is lost. We have to define predicates in one place, and
> > >     handlers in another. This opens the door for bugs:
> > >     >
> > >     > - what if we forget to define a handler for a name? or a name for
> > >     a handler?
> > >     > - what if we misspell a name?
> > >     > - what if we copy-paste and duplicate a name?
> > >     >
> > >     > What Michael propose would have been totally OK if we had been
> > >     writing the API in Lua, Ruby or Python. In those languages the
> > >     "dynamic naming" approach would have looked most concise and
> > >     beautiful. But in Java we expect all the problems related to
> > >     identifiers to be eliminated in compile time.
> > >     >
> > >     > Do we have to invent duck-typing for the Java API?
> > >     >
> > >     > And if we do, what advantage are we supposed to get besides having
> > >     all the branches in the same scope? Michael, maybe I'm missing your
> > >     point?
> > >     >
> > >     > ---
> > >     >
> > >     > Earlier in this discussion John Roesler also proposed to do
> > >     without "start branching" operator, and later Paul mentioned that in
> > >     the case when we have to add a dynamic number of branches, the
> > >     current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
> > >     me address both comments here.
> > >     >
> > >     > 1) "Start branching" operator (I think that *split* is a good name
> > >     for it indeed) is critical when we need to do a dynamic branching,
> > >     see example below.
> > >     >
> > >     > 2) No, dynamic branching in current KIP is not clumsy at all.
> > >     Imagine a real-world scenario when you need one branch per enum
> > >     value (say, RecordType). You can have something like this:
> > >     >
> > >     > /*John:if we had to start with stream.branch(...) here, it would
> > >     have been much messier.*/
> > >     > KBranchedStream branched = stream.split();
> > >     >
> > >     > /*Not clumsy at all :-)*/
> > >     > for (RecordType recordType : RecordType.values())
> > >     >             branched = branched.branch((k, v) -> v.getRecType() ==
> > >     recordType,
> > >     >                     recordType::processRecords);
> > >     >
> > >     > Regards,
> > >     >
> > >     > Ivan
> > >     >
> > >     >
> > >     > 02.05.2019 14:40, Matthias J. Sax пишет:
> > >     >> I also agree with Michael's observation about the core problem of
> > >     >> current `branch()` implementation.
> > >     >>
> > >     >> However, I also don't like to pass in a clumsy Map object. My
> > >     thinking
> > >     >> was more aligned with Paul's proposal to just add a name to each
> > >     >> `branch()` statement and return a `Map<String,KStream>`.
> > >     >>
> > >     >> It makes the code easier to read, and also make the order of
> > >     >> `Predicates` (that is essential) easier to grasp.
> > >     >>
> > >     >>>>>> Map<String, KStream<K, V>> branches = stream.split()
> > >     >>>>>>    .branch("branchOne", Predicate<K, V>)
> > >     >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> > >     >>>>>>    .defaultBranch("defaultBranch");
> > >     >> An open question is the case for which no defaultBranch() should
> > be
> > >     >> specified. Atm, `split()` and `branch()` would return
> > >     `BranchedKStream`
> > >     >> and the call to `defaultBranch()` that returns the `Map` is
> > mandatory
> > >     >> (what is not the case atm). Or is this actually not a real
> > problem,
> > >     >> because users can just ignore the branch returned by
> > >     `defaultBranch()`
> > >     >> in the result `Map` ?
> > >     >>
> > >     >>
> > >     >> About "inlining": So far, it seems to be a matter of personal
> > >     >> preference. I can see arguments for both, but no "killer
> > >     argument" yet
> > >     >> that clearly make the case for one or the other.
> > >     >>
> > >     >>
> > >     >> -Matthias
> > >     >>
> > >     >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> > >     >>> Perhaps inlining is the wrong terminology. It doesn’t require
> > >     that a lambda with the full downstream topology be defined inline -
> > >     it can be a method reference as with Ivan’s original suggestion.
> > >     The advantage of putting the predicate and its downstream logic
> > >     (Consumer) together in branch() is that they are required to be near
> > >     to each other.
> > >     >>>
> > >     >>> Ultimately the downstream code has to live somewhere, and deep
> > >     branch trees will be hard to read regardless.
> > >     >>>
> > >     >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> > >     <michael.drogalis@confluent.io
> > >     <ma...@confluent.io>> wrote:
> > >     >>>>
> > >     >>>> I'm less enthusiastic about inlining the branch logic with its
> > >     downstream
> > >     >>>> functionality. Programs that have deep branch trees will
> > >     quickly become
> > >     >>>> harder to read as a single unit.
> > >     >>>>
> > >     >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> > >     <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
> > >     >>>>>
> > >     >>>>> Also +1 on the issues/goals as Michael outlined them, I think
> > >     that sets a
> > >     >>>>> great framework for the discussion.
> > >     >>>>>
> > >     >>>>> Regarding the SortedMap solution, my understanding is that the
> > >     current
> > >     >>>>> proposal in the KIP is what is in my PR which (pending naming
> > >     decisions) is
> > >     >>>>> roughly this:
> > >     >>>>>
> > >     >>>>> stream.split()
> > >     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> > >     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> > >     >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> > >     >>>>>
> > >     >>>>> Obviously some ordering is necessary, since branching as a
> > >     construct
> > >     >>>>> doesn't work without it, but this solution seems like it
> > >     provides as much
> > >     >>>>> associativity as the SortedMap solution, because each branch()
> > >     call
> > >     >>>>> directly associates the "conditional" with the "code block."
> > >     The value it
> > >     >>>>> provides over the KIP solution is the accessing of streams in
> > >     the same
> > >     >>>>> scope.
> > >     >>>>>
> > >     >>>>> The KIP solution is less "dynamic" than the SortedMap solution
> > >     in the sense
> > >     >>>>> that it is slightly clumsier to add a dynamic number of
> > >     branches, but it is
> > >     >>>>> certainly possible.  It seems to me like the API should favor
> > >     the "static"
> > >     >>>>> case anyway, and should make it simple and readable to
> > >     fluently declare and
> > >     >>>>> access your branches in-line.  It also makes it impossible to
> > >     ignore a
> > >     >>>>> branch, and it is possible to build an (almost) identical
> > >     SortedMap
> > >     >>>>> solution on top of it.
> > >     >>>>>
> > >     >>>>> I could also see a middle ground where instead of a raw
> > >     SortedMap being
> > >     >>>>> taken in, branch() takes a name and not a Consumer.  Something
> > >     like this:
> > >     >>>>>
> > >     >>>>> Map<String, KStream<K, V>> branches = stream.split()
> > >     >>>>>    .branch("branchOne", Predicate<K, V>)
> > >     >>>>>    .branch( "branchTwo", Predicate<K, V>)
> > >     >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
> > >     >>>>>
> > >     >>>>> Pros for that solution:
> > >     >>>>> - accessing branched KStreams in same scope
> > >     >>>>> - no double brace initialization, hopefully slightly more
> > >     readable than
> > >     >>>>> SortedMap
> > >     >>>>>
> > >     >>>>> Cons
> > >     >>>>> - downstream branch logic cannot be specified inline which
> > >     makes it harder
> > >     >>>>> to read top to bottom (like existing API and SortedMap, but
> > >     unlike the KIP)
> > >     >>>>> - you can forget to "handle" one of the branched streams (like
> > >     existing
> > >     >>>>> API and SortedMap, but unlike the KIP)
> > >     >>>>>
> > >     >>>>> (KBranchedStreams could even work *both* ways but perhaps
> > >     that's overdoing
> > >     >>>>> it).
> > >     >>>>>
> > >     >>>>> Overall I'm curious how important it is to be able to easily
> > >     access the
> > >     >>>>> branched KStream in the same scope as the original.  It's
> > >     possible that it
> > >     >>>>> doesn't need to be handled directly by the API, but instead
> > >     left up to the
> > >     >>>>> user.  I'm sort of in the middle on it.
> > >     >>>>>
> > >     >>>>> Paul
> > >     >>>>>
> > >     >>>>>
> > >     >>>>>
> > >     >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
> > >     <sophie@confluent.io <ma...@confluent.io>>
> > >     >>>>> wrote:
> > >     >>>>>
> > >     >>>>>> I'd like to +1 what Michael said about the issues with the
> > >     existing
> > >     >>>>> branch
> > >     >>>>>> method, I agree with what he's outlined and I think we should
> > >     proceed by
> > >     >>>>>> trying to alleviate these problems. Specifically it seems
> > >     important to be
> > >     >>>>>> able to cleanly access the individual branches (eg by mapping
> > >     >>>>>> name->stream), which I thought was the original intention of
> > >     this KIP.
> > >     >>>>>>
> > >     >>>>>> That said, I don't think we should so easily give in to the
> > >     double brace
> > >     >>>>>> anti-pattern or force ours users into it if at all possible to
> > >     >>>>> avoid...just
> > >     >>>>>> my two cents.
> > >     >>>>>>
> > >     >>>>>> Cheers,
> > >     >>>>>> Sophie
> > >     >>>>>>
> > >     >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
> > >     >>>>>> michael.drogalis@confluent.io
> > >     <ma...@confluent.io>> wrote:
> > >     >>>>>>
> > >     >>>>>>> I’d like to propose a different way of thinking about this.
> > >     To me,
> > >     >>>>> there
> > >     >>>>>>> are three problems with the existing branch signature:
> > >     >>>>>>>
> > >     >>>>>>> 1. If you use it the way most people do, Java raises unsafe
> > type
> > >     >>>>>> warnings.
> > >     >>>>>>> 2. The way in which you use the stream branches is
> > >     positionally coupled
> > >     >>>>>> to
> > >     >>>>>>> the ordering of the conditionals.
> > >     >>>>>>> 3. It is brittle to extend existing branch calls with
> > >     additional code
> > >     >>>>>>> paths.
> > >     >>>>>>>
> > >     >>>>>>> Using associative constructs instead of relying on ordered
> > >     constructs
> > >     >>>>>> would
> > >     >>>>>>> be a stronger approach. Consider a signature that instead
> > >     looks like
> > >     >>>>>> this:
> > >     >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
> > >     Predicate<?
> > >     >>>>>>> super K,? super V>>);
> > >     >>>>>>>
> > >     >>>>>>> Branches are given names in a map, and as a result, the API
> > >     returns a
> > >     >>>>>>> mapping of names to streams. The ordering of the
> > conditionals is
> > >     >>>>>> maintained
> > >     >>>>>>> because it’s a sorted map. Insert order determines the order
> > of
> > >     >>>>>> evaluation.
> > >     >>>>>>> This solves problem 1 because there are no more varargs. It
> > >     solves
> > >     >>>>>> problem
> > >     >>>>>>> 2 because you no longer lean on ordering to access the
> > >     branch you’re
> > >     >>>>>>> interested in. It solves problem 3 because you can introduce
> > >     another
> > >     >>>>>>> conditional by simply attaching another name to the
> > >     structure, rather
> > >     >>>>>> than
> > >     >>>>>>> messing with the existing indices.
> > >     >>>>>>>
> > >     >>>>>>> One of the drawbacks is that creating the map inline is
> > >     historically
> > >     >>>>>>> awkward in Java. I know it’s an anti-pattern to use
> > >     voluminously, but
> > >     >>>>>>> double brace initialization would clean up the aesthetics.
> > >     >>>>>>>
> > >     >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> > >     <john@confluent.io <ma...@confluent.io>>
> > >     >>>>> wrote:
> > >     >>>>>>>> Hi Ivan,
> > >     >>>>>>>>
> > >     >>>>>>>> Thanks for the update.
> > >     >>>>>>>>
> > >     >>>>>>>> FWIW, I agree with Matthias that the current "start
> > branching"
> > >     >>>>> operator
> > >     >>>>>>> is
> > >     >>>>>>>> confusing when named the same way as the actual branches.
> > >     "Split"
> > >     >>>>> seems
> > >     >>>>>>>> like a good name. Alternatively, we can do without a "start
> > >     >>>>> branching"
> > >     >>>>>>>> operator at all, and just do:
> > >     >>>>>>>>
> > >     >>>>>>>> stream
> > >     >>>>>>>>      .branch(Predicate)
> > >     >>>>>>>>      .branch(Predicate)
> > >     >>>>>>>>      .defaultBranch();
> > >     >>>>>>>>
> > >     >>>>>>>> Tentatively, I think that this branching operation should be
> > >     >>>>> terminal.
> > >     >>>>>>> That
> > >     >>>>>>>> way, we don't create ambiguity about how to use it. That
> > >     is, `branch`
> > >     >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
> > >     `void`, to
> > >     >>>>>>>> enforce that it comes last, and that there is only one
> > >     definition of
> > >     >>>>>> the
> > >     >>>>>>>> default branch. Potentially, we should log a warning if
> > >     there's no
> > >     >>>>>>> default,
> > >     >>>>>>>> and additionally log a warning (or throw an exception) if a
> > >     record
> > >     >>>>>> falls
> > >     >>>>>>>> though with no default.
> > >     >>>>>>>>
> > >     >>>>>>>> Thoughts?
> > >     >>>>>>>>
> > >     >>>>>>>> Thanks,
> > >     >>>>>>>> -John
> > >     >>>>>>>>
> > >     >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
> > >     >>>>> matthias@confluent.io <ma...@confluent.io>
> > >     >>>>>>>> wrote:
> > >     >>>>>>>>
> > >     >>>>>>>>> Thanks for updating the KIP and your answers.
> > >     >>>>>>>>>
> > >     >>>>>>>>>
> > >     >>>>>>>>>> this is to make the name similar to String#split
> > >     >>>>>>>>>>> that also returns an array, right?
> > >     >>>>>>>>> The intend was to avoid name duplication. The return type
> > >     should
> > >     >>>>>> _not_
> > >     >>>>>>>>> be an array.
> > >     >>>>>>>>>
> > >     >>>>>>>>> The current proposal is
> > >     >>>>>>>>>
> > >     >>>>>>>>> stream.branch()
> > >     >>>>>>>>>      .branch(Predicate)
> > >     >>>>>>>>>      .branch(Predicate)
> > >     >>>>>>>>>      .defaultBranch();
> > >     >>>>>>>>>
> > >     >>>>>>>>> IMHO, this reads a little odd, because the first
> > >     `branch()` does
> > >     >>>>> not
> > >     >>>>>>>>> take any parameters and has different semantics than the
> > later
> > >     >>>>>>>>> `branch()` calls. Note, that from the code snippet above,
> > it's
> > >     >>>>> hidden
> > >     >>>>>>>>> that the first call is `KStream#branch()` while the others
> > are
> > >     >>>>>>>>> `KBranchedStream#branch()` what makes reading the code
> > harder.
> > >     >>>>>>>>>
> > >     >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
> > >     I though
> > >     >>>>>> it
> > >     >>>>>>>>> might be better to also rename `KStream#branch()` to avoid
> > the
> > >     >>>>> naming
> > >     >>>>>>>>> overlap that seems to be confusing. The following reads
> > much
> > >     >>>>> cleaner
> > >     >>>>>> to
> > >     >>>>>>>> me:
> > >     >>>>>>>>> stream.split()
> > >     >>>>>>>>>      .branch(Predicate)
> > >     >>>>>>>>>      .branch(Predicate)
> > >     >>>>>>>>>      .defaultBranch();
> > >     >>>>>>>>>
> > >     >>>>>>>>> Maybe there is a better alternative to `split()` though to
> > >     avoid
> > >     >>>>> the
> > >     >>>>>>>>> naming overlap.
> > >     >>>>>>>>>
> > >     >>>>>>>>>
> > >     >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately
> > we
> > >     >>>>> cannot
> > >     >>>>>>> have
> > >     >>>>>>>>> a method with such name :-)
> > >     >>>>>>>>>
> > >     >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
> > >     with a
> > >     >>>>> short
> > >     >>>>>>>> name?
> > >     >>>>>>>>>
> > >     >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
> > >     with all
> > >     >>>>> it's
> > >     >>>>>>>>> methods? It will be part of public API and should be
> > >     contained in
> > >     >>>>> the
> > >     >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
> > >     >>>>>>>>> `defaultBranch()` is.
> > >     >>>>>>>>>
> > >     >>>>>>>>>
> > >     >>>>>>>>> You did not comment on the idea to add a
> > >     `KBranchedStream#get(int
> > >     >>>>>>> index)
> > >     >>>>>>>>> -> KStream` method to get the individually
> > >     branched-KStreams. Would
> > >     >>>>>> be
> > >     >>>>>>>>> nice to get your feedback about it. It seems you suggest
> > >     that users
> > >     >>>>>>>>> would need to write custom utility code otherwise, to
> > >     access them.
> > >     >>>>> We
> > >     >>>>>>>>> should discuss the pros and cons of both approaches. It
> > feels
> > >     >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
> > >     to get
> > >     >>>>> the
> > >     >>>>>>>>> branched-KStreams directly.
> > >     >>>>>>>>>
> > >     >>>>>>>>>
> > >     >>>>>>>>>
> > >     >>>>>>>>> -Matthias
> > >     >>>>>>>>>
> > >     >>>>>>>>>
> > >     >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> > >     >>>>>>>>>> Hi all!
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> I have updated the KIP-418 according to the new vision.
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> Matthias, thanks for your comment!
> > >     >>>>>>>>>>
> > >     >>>>>>>>>>> Renaming KStream#branch() -> #split()
> > >     >>>>>>>>>> I can see your point: this is to make the name similar to
> > >     >>>>>>> String#split
> > >     >>>>>>>>>> that also returns an array, right? But is it worth the
> > >     loss of
> > >     >>>>>>>> backwards
> > >     >>>>>>>>>> compatibility? We can have overloaded branch() as well
> > >     without
> > >     >>>>>>>> affecting
> > >     >>>>>>>>>> the existing code. Maybe the old array-based `branch`
> > method
> > >     >>>>> should
> > >     >>>>>>> be
> > >     >>>>>>>>>> deprecated, but this is a subject for discussion.
> > >     >>>>>>>>>>
> > >     >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> > >     >>>>> BranchingKStream#branch(),
> > >     >>>>>>>>>> KBranchedStream#defaultBranch() ->
> > BranchingKStream#default()
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default'
> > is,
> > >     >>>>>>> however, a
> > >     >>>>>>>>>> reserved word, so unfortunately we cannot have a method
> > >     with such
> > >     >>>>>>> name
> > >     >>>>>>>>> :-)
> > >     >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument,
> > but I
> > >     >>>>> think
> > >     >>>>>>> that
> > >     >>>>>>>>>> is not required?
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> Absolutely! I think that was just copy-paste error or
> > >     something.
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> Dear colleagues,
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> please revise the new version of the KIP and Paul's PR
> > >     >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> Any new suggestions/objections?
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> Regards,
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> Ivan
> > >     >>>>>>>>>>
> > >     >>>>>>>>>>
> > >     >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> > >     >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems
> > that
> > >     >>>>>>> everybody
> > >     >>>>>>>>>>> agrees that the current branch() method using arrays is
> > not
> > >     >>>>>> optimal.
> > >     >>>>>>>>>>> I had a quick look into the PR and I like the overall
> > >     proposal.
> > >     >>>>>>> There
> > >     >>>>>>>>>>> are some minor things we need to consider. I would
> > >     recommend the
> > >     >>>>>>>>>>> following renaming:
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>> KStream#branch() -> #split()
> > >     >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
> > >     >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> > >     BranchingKStream#default()
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>> It's just a suggestion to get slightly shorter method
> > names.
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>> In the current PR, defaultBranch() does take an
> > >     `Predicate` as
> > >     >>>>>>>> argument,
> > >     >>>>>>>>>>> but I think that is not required?
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>> Also, we should consider KIP-307, that was recently
> > >     accepted and
> > >     >>>>>> is
> > >     >>>>>>>>>>> currently implemented:
> > >     >>>>>>>>>>>
> > >     >>>>>
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> > >     >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
> > >     parameter.
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>> For the issue that the created `KStream` object are in
> > >     different
> > >     >>>>>>>> scopes:
> > >     >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
> > >     index)` method
> > >     >>>>>>> that
> > >     >>>>>>>>>>> returns the corresponding "branched" result `KStream`
> > >     object?
> > >     >>>>>> Maybe,
> > >     >>>>>>>> the
> > >     >>>>>>>>>>> second argument of `addBranch()` should not be a
> > >     >>>>>> `Consumer<KStream>`
> > >     >>>>>>>> but
> > >     >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
> > >     whatever
> > >     >>>>>> the
> > >     >>>>>>>>>>> `Function` returns?
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
> > >     current
> > >     >>>>>>>>>>> proposal. That makes it easier to review.
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>> -Matthias
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>>
> > >     >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> > >     >>>>>>>>>>>> Ivan,
> > >     >>>>>>>>>>>>
> > >     >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
> > >     makes sense
> > >     >>>>>> for
> > >     >>>>>>>> you
> > >     >>>>>>>>> to
> > >     >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
> > >     we'll
> > >     >>>>> need
> > >     >>>>>>>> some
> > >     >>>>>>>>>>>> buy-in from committers that have actual binding votes on
> > >     >>>>> whether
> > >     >>>>>>> the
> > >     >>>>>>>>> KIP
> > >     >>>>>>>>>>>> could be adopted.  It would be great to hear if they
> > >     think this
> > >     >>>>>> is
> > >     >>>>>>> a
> > >     >>>>>>>>> good
> > >     >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
> > >     starting a
> > >     >>>>>>> vote,
> > >     >>>>>>>>> or if
> > >     >>>>>>>>>>>> there is generally some indication of interest
> > beforehand.
> > >     >>>>>>>>>>>>
> > >     >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
> > >     assuming
> > >     >>>>> we
> > >     >>>>>> do
> > >     >>>>>>>>> move
> > >     >>>>>>>>>>>> forward the solution of "stream.branch() returns
> > >     >>>>>> KBranchedStream",
> > >     >>>>>>> do
> > >     >>>>>>>>> we
> > >     >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I
> > would
> > >     >>>>> favor
> > >     >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs
> > that
> > >     >>>>>>> accomplish
> > >     >>>>>>>>> the
> > >     >>>>>>>>>>>> same thing is confusing, especially when they're fairly
> > >     similar
> > >     >>>>>>>>> anyway.  We
> > >     >>>>>>>>>>>> just need to be sure we're not making something
> > >     >>>>>>> impossible/difficult
> > >     >>>>>>>>> that
> > >     >>>>>>>>>>>> is currently possible/easy.
> > >     >>>>>>>>>>>>
> > >     >>>>>>>>>>>> Regarding my PR - I think the general structure would
> > work,
> > >     >>>>> it's
> > >     >>>>>>>> just a
> > >     >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
> > >     >>>>>>> particular,
> > >     >>>>>>>>>>>> passing in the "predicates" and "children" lists which
> > get
> > >     >>>>>> modified
> > >     >>>>>>>> in
> > >     >>>>>>>>>>>> KBranchedStream but read from all the way
> > >     KStreamLazyBranch is
> > >     >>>>> a
> > >     >>>>>>> bit
> > >     >>>>>>>>>>>> complicated to follow.
> > >     >>>>>>>>>>>>
> > >     >>>>>>>>>>>> Thanks,
> > >     >>>>>>>>>>>> Paul
> > >     >>>>>>>>>>>>
> > >     >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
> > >     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> > >     >>>>>>>>> wrote:
> > >     >>>>>>>>>>>>> Hi Paul!
> > >     >>>>>>>>>>>>>
> > >     >>>>>>>>>>>>> I read your code carefully and now I am fully
> > >     convinced: your
> > >     >>>>>>>> proposal
> > >     >>>>>>>>>>>>> looks better and should work. We just have to document
> > the
> > >     >>>>>> crucial
> > >     >>>>>>>>> fact
> > >     >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
> > >     And then
> > >     >>>>>> it's
> > >     >>>>>>>> all
> > >     >>>>>>>>>>>>> going to be very nice.
> > >     >>>>>>>>>>>>>
> > >     >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
> > >     resume the
> > >     >>>>>>>>>>>>> discussion here, right?
> > >     >>>>>>>>>>>>>
> > >     >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
> > >     >>>>> starting
> > >     >>>>>>>> point
> > >     >>>>>>>>> if
> > >     >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
> > >     starting
> > >     >>>>>>> point.
> > >     >>>>>>>>> But
> > >     >>>>>>>>>>>>> as a novice in this project I might miss some important
> > >     >>>>> details.
> > >     >>>>>>>>>>>>> Regards,
> > >     >>>>>>>>>>>>>
> > >     >>>>>>>>>>>>> Ivan
> > >     >>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>
> > >     >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> > >     >>>>>>>>>>>>>> Ivan,
> > >     >>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
> > >     >>>>> stream.branch()
> > >     >>>>>>>>> solution
> > >     >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
> > >     >>>>> invoked
> > >     >>>>>> as
> > >     >>>>>>>>> they’re
> > >     >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
> > >     still
> > >     >>>>>> ought
> > >     >>>>>>> to
> > >     >>>>>>>>> be
> > >     >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
> > >     depend on
> > >     >>>>> the
> > >     >>>>>>>>> branched
> > >     >>>>>>>>>>>>> streams having been set.
> > >     >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
> > >     access
> > >     >>>>> the
> > >     >>>>>>>>> branched
> > >     >>>>>>>>>>>>> streams in the same scope as the original stream (that
> > >     is, not
> > >     >>>>>>>> inside
> > >     >>>>>>>>> the
> > >     >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
> > >     >>>>> solutions.
> > >     >>>>>> It
> > >     >>>>>>>>> can be
> > >     >>>>>>>>>>>>> worked around though.
> > >     >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
> > >     excited
> > >     >>>>> to
> > >     >>>>>>>> hear
> > >     >>>>>>>>>>>>> your thoughts!]
> > >     >>>>>>>>>>>>>> Paul
> > >     >>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
> > >     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> > >     >>>>>>>>> wrote:
> > >     >>>>>>>>>>>>>>> Hi Paul!
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
> > >     >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at
> > first
> > >     >>>>> glance,
> > >     >>>>>>> but
> > >     >>>>>>>>> ---
> > >     >>>>>>>>>>>>>>>> the newly branched streams are not available in the
> > >     same
> > >     >>>>>> scope
> > >     >>>>>>> as
> > >     >>>>>>>>> each
> > >     >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
> > together
> > >     >>>>> again
> > >     >>>>>> I
> > >     >>>>>>>>> don't see
> > >     >>>>>>>>>>>>> a way to do that.
> > >     >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
> > >     just
> > >     >>>>>> going
> > >     >>>>>>> to
> > >     >>>>>>>>>>>>> write in details about this issue.
> > >     >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
> > >     we need
> > >     >>>>> to
> > >     >>>>>>>>> identify
> > >     >>>>>>>>>>>>> customers who have bought coffee and made a purchase
> > >     in the
> > >     >>>>>>>>> electronics
> > >     >>>>>>>>>>>>> store to give them coupons.
> > >     >>>>>>>>>>>>>>> This is the code I usually write under these
> > >     circumstances
> > >     >>>>>> using
> > >     >>>>>>>> my
> > >     >>>>>>>>>>>>> 'brancher' class:
> > >     >>>>>>>>>>>>>>> @Setter
> > >     >>>>>>>>>>>>>>> class CouponIssuer{
> > >     >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
> > >     >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>   KStream<...> coupons(){
> > >     >>>>>>>>>>>>>>>       return
> > >     >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> > >     >>>>>>>>>>>>>>>       /*In the real world the code here can be
> > >     complex, so
> > >     >>>>>>>>> creation of
> > >     >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
> > >     order to
> > >     >>>>>>>> separate
> > >     >>>>>>>>>>>>> classes' responsibilities.*/
> > >     >>>>>>>>>>>>>>>  }
> > >     >>>>>>>>>>>>>>> }
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> > >     >>>>>>>>>>>>>>>     .branch(predicate1,
> > couponIssuer::setCoffePurchases)
> > >     >>>>>>>>>>>>>>>     .branch(predicate2,
> > >     >>>>>> couponIssuer::setElectronicsPurchases)
> > >     >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
> > >     everything
> > >     >>>>>>>> later,
> > >     >>>>>>>>>>>>> without the terminal operation!!!*/
> > >     >>>>>>>>>>>>>>> couponIssuer.coupons()...
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> Does this make sense?  In order to properly
> > >     initialize the
> > >     >>>>>>>>> CouponIssuer
> > >     >>>>>>>>>>>>> we need the terminal operation to be called before
> > >     >>>>>>>>> streamsBuilder.build()
> > >     >>>>>>>>>>>>> is called.
> > >     >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
> > >     essentially
> > >     >>>>>> the
> > >     >>>>>>>>> next
> > >     >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
> > >     based on
> > >     >>>>> my
> > >     >>>>>>>>> experience,
> > >     >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
> > >     >>>>>>>>>>>>>>> Regards,
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> Ivan
> > >     >>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> > >     >>>>>>>>>>>>>>>> Ivan,
> > >     >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
> > >     fluent
> > >     >>>>> API
> > >     >>>>>>>> based
> > >     >>>>>>>>>>>>> off of
> > >     >>>>>>>>>>>>>>>> KStream here
> > >     (https://github.com/apache/kafka/pull/6512),
> > >     >>>>>> and
> > >     >>>>>>> I
> > >     >>>>>>>>> think
> > >     >>>>>>>>>>>>> I
> > >     >>>>>>>>>>>>>>>> succeeded at removing both cons.
> > >     >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
> > >     >>>>>>> compatibility
> > >     >>>>>>>>>>>>> issues,
> > >     >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
> > >     that Java
> > >     >>>>> is
> > >     >>>>>>>> smart
> > >     >>>>>>>>>>>>> enough to
> > >     >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
> > >     returning one
> > >     >>>>>>> thing
> > >     >>>>>>>>> and
> > >     >>>>>>>>>>>>> branch()
> > >     >>>>>>>>>>>>>>>>    with no arguments returning another thing.
> > >     >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
> > >     need
> > >     >>>>> it.
> > >     >>>>>>> We
> > >     >>>>>>>>> can
> > >     >>>>>>>>>>>>> just
> > >     >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
> > >     shares
> > >     >>>>>> its
> > >     >>>>>>>>> state
> > >     >>>>>>>>>>>>> with the
> > >     >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
> > >     branching.
> > >     >>>>>>> It's
> > >     >>>>>>>>> not
> > >     >>>>>>>>>>>>> terribly
> > >     >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
> > >     demonstrates
> > >     >>>>>> its
> > >     >>>>>>>>>>>>> feasibility.
> > >     >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should
> > be
> > >     >>>>> final
> > >     >>>>>> or
> > >     >>>>>>>>> even a
> > >     >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
> > >     wanted to
> > >     >>>>>> see
> > >     >>>>>>>> how
> > >     >>>>>>>>>>>>>>>> challenging it would be to get the API working.
> > >     >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
> > >     solution
> > >     >>>>>>> could
> > >     >>>>>>>> be
> > >     >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
> > >     >>>>> suggested
> > >     >>>>>>>> was a
> > >     >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
> > >     streams
> > >     >>>>>> are
> > >     >>>>>>>> not
> > >     >>>>>>>>>>>>>>>> available in the same scope as each other.  That
> > >     is, if we
> > >     >>>>>>> wanted
> > >     >>>>>>>>> to
> > >     >>>>>>>>>>>>> merge
> > >     >>>>>>>>>>>>>>>> them back together again I don't see a way to do
> > >     that.  The
> > >     >>>>>> KIP
> > >     >>>>>>>>>>>>> proposal
> > >     >>>>>>>>>>>>>>>> has the same issue, though - all this means is that
> > for
> > >     >>>>>> either
> > >     >>>>>>>>>>>>> solution,
> > >     >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
> > >     table.
> > >     >>>>>>>>>>>>>>>> Thanks,
> > >     >>>>>>>>>>>>>>>> Paul
> > >     >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> > >     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
> > >     >>>>>>>>>>>>> wrote:
> > >     >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to
> > this
> > >     >>>>>> point.
> > >     >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
> > >     branch API
> > >     >>>>>>> needs
> > >     >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> There are two potential ways to do it:
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> > >     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> > >     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> > >     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> > >     >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
> > >     returns
> > >     >>>>>> its
> > >     >>>>>>>>> argument
> > >     >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code
> > won't
> > >     >>>>> make
> > >     >>>>>>>> sense
> > >     >>>>>>>>>>>>> until
> > >     >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
> > >     instance
> > >     >>>>>>>>> contrasts the
> > >     >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> stream
> > >     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> > >     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> > >     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
> > >     >>>>>>>>> defaultBranch(..)
> > >     >>>>>>>>>>>>> and
> > >     >>>>>>>>>>>>>>>>> noDefault() return void
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface
> > is
> > >     >>>>>> defined.
> > >     >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
> > >     >>>>>>>> (defaultBranch(ks->)
> > >     >>>>>>>>> and
> > >     >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
> > >     miss the
> > >     >>>>>> fact
> > >     >>>>>>>>> that one
> > >     >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
> > >     methods
> > >     >>>>>> are
> > >     >>>>>>>> not
> > >     >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do
> > better?
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> Regards,
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> Ivan
> > >     >>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> > >     >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> > >     >>>>>>>>>>>>>>>>>>> Paul,
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
> > >     >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
> > >     implemented the
> > >     >>>>>>> easy
> > >     >>>>>>>>> way.
> > >     >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
> > assumes
> > >     >>>>> nothing
> > >     >>>>>>>> will
> > >     >>>>>>>>>>>>> reach
> > >     >>>>>>>>>>>>>>>>>>>> the default branch,
> > >     >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only
> > option
> > >     >>>>>> besides
> > >     >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
> > >     want to
> > >     >>>>>> just
> > >     >>>>>>>>> silently
> > >     >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
> > >     predicate. 2)
> > >     >>>>>>> Throwing
> > >     >>>>>>>>> an
> > >     >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
> > >     looks
> > >     >>>>>> like a
> > >     >>>>>>>> bad
> > >     >>>>>>>>>>>>> idea.
> > >     >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
> > >     emit a
> > >     >>>>>>>> special
> > >     >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly
> > where
> > >     >>>>>>> `default`
> > >     >>>>>>>>> can
> > >     >>>>>>>>>>>>> be
> > >     >>>>>>>>>>>>>>>>>>> used.
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> > >     >>>>> InternalTopologyBuilder
> > >     >>>>>>> to
> > >     >>>>>>>>> track
> > >     >>>>>>>>>>>>>>>>>>>> dangling
> > >     >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
> > >     a clear
> > >     >>>>>>> error
> > >     >>>>>>>>>>>>> before it
> > >     >>>>>>>>>>>>>>>>>>> becomes an issue.
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
> > >     >>>>> compiled
> > >     >>>>>>> and
> > >     >>>>>>>>> run?
> > >     >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
> > >     compile if
> > >     >>>>> used
> > >     >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
> > >     method chain
> > >     >>>>>>>> starting
> > >     >>>>>>>>>>>>> from
> > >     >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
> > >     between
> > >     >>>>>>>> runtime
> > >     >>>>>>>>> and
> > >     >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
> > >     >>>>> instantly
> > >     >>>>>> on
> > >     >>>>>>>>> unit
> > >     >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
> > >     compilation
> > >     >>>>>>>> failure.
> > >     >>>>>>>>>>>>>>>>>>> Regards,
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>> Ivan
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> > >     >>>>>>>>>>>>>>>>>>>> Ivan,
> > >     >>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
> > >     required.
> > >     >>>>>>> But
> > >     >>>>>>>> is
> > >     >>>>>>>>>>>>> that
> > >     >>>>>>>>>>>>>>>>>>>> really
> > >     >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
> > >     >>>>>> defaultBranch
> > >     >>>>>>>>> they
> > >     >>>>>>>>>>>>> can
> > >     >>>>>>>>>>>>>>>>>>>> call
> > >     >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
> > >     just as
> > >     >>>>>>>>> easily.  In
> > >     >>>>>>>>>>>>>>>>>>>> fact I
> > >     >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API
> > - a
> > >     >>>>> user
> > >     >>>>>>>> could
> > >     >>>>>>>>>>>>> specify
> > >     >>>>>>>>>>>>>>>>> a
> > >     >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach
> > the
> > >     >>>>>> default
> > >     >>>>>>>>> branch,
> > >     >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
> > That
> > >     >>>>> seems
> > >     >>>>>>> like
> > >     >>>>>>>>> an
> > >     >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
> > >     which allows
> > >     >>>>>> for
> > >     >>>>>>>> the
> > >     >>>>>>>>>>>>> more
> > >     >>>>>>>>>>>>>>>>>>>> subtle
> > >     >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
> > dropped.
> > >     >>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
> > >     to be
> > >     >>>>>> well
> > >     >>>>>>>>>>>>>>>>>>>> documented, but
> > >     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> > >     >>>>> InternalTopologyBuilder
> > >     >>>>>>> to
> > >     >>>>>>>>> track
> > >     >>>>>>>>>>>>>>>>>>>> dangling
> > >     >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
> > >     a clear
> > >     >>>>>>> error
> > >     >>>>>>>>>>>>> before it
> > >     >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is
> > a
> > >     >>>>> "build
> > >     >>>>>>>> step"
> > >     >>>>>>>>>>>>> where
> > >     >>>>>>>>>>>>>>>>> the
> > >     >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
> > >     >>>>>> StreamsBuilder.build()
> > >     >>>>>>> is
> > >     >>>>>>>>>>>>> called.
> > >     >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I
> > agree
> > >     >>>>> that
> > >     >>>>>>> it's
> > >     >>>>>>>>>>>>>>>>>>>> critical to
> > >     >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
> > >     stream.
> > >     >>>>>>> With
> > >     >>>>>>>>> the
> > >     >>>>>>>>>>>>>>>>> fluent
> > >     >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all
> > other
> > >     >>>>>>> operations
> > >     >>>>>>>>> do -
> > >     >>>>>>>>>>>>> if
> > >     >>>>>>>>>>>>>>>>> you
> > >     >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
> > multiple
> > >     >>>>> times,
> > >     >>>>>>> you
> > >     >>>>>>>>> just
> > >     >>>>>>>>>>>>>>>>>>>> need the
> > >     >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
> > >     operations
> > >     >>>>>> on
> > >     >>>>>>> it
> > >     >>>>>>>>> as
> > >     >>>>>>>>>>>>> you
> > >     >>>>>>>>>>>>>>>>>>>> desire.
> > >     >>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>> Thoughts?
> > >     >>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>> Best,
> > >     >>>>>>>>>>>>>>>>>>>> Paul
> > >     >>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> > >     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
> > >     >>>>>>>>>>>>>>>>>>>> wrote:
> > >     >>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
> > >     always need
> > >     >>>>>> the
> > >     >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
> > operation we
> > >     >>>>> don't
> > >     >>>>>>>> know
> > >     >>>>>>>>>>>>> when to
> > >     >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
> > >     so we
> > >     >>>>> can
> > >     >>>>>> do
> > >     >>>>>>>>>>>>> something
> > >     >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
> > special
> > >     >>>>> object
> > >     >>>>>>>>>>>>> construction
> > >     >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods.
> > But
> > >     >>>>> here
> > >     >>>>>> we
> > >     >>>>>>>>> have a
> > >     >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
> > >     flow,
> > >     >>>>> so
> > >     >>>>>> I
> > >     >>>>>>>>> think
> > >     >>>>>>>>>>>>> this
> > >     >>>>>>>>>>>>>>>>> is
> > >     >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>> Regards,
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>> Ivan
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> > >     >>>>>>>>>>>>>>>>>>>>>> Ivan,
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
> > >     API, but I
> > >     >>>>>> find
> > >     >>>>>>>> the
> > >     >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> > >     >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
> > >     contrasts the
> > >     >>>>>>> fluency
> > >     >>>>>>>>> of
> > >     >>>>>>>>>>>>> other
> > >     >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
> > >     just call
> > >     >>>>> a
> > >     >>>>>>>>> method on
> > >     >>>>>>>>>>>>> the
> > >     >>>>>>>>>>>>>>>>>>>>> stream
> > >     >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
> > >     cases
> > >     >>>>> are
> > >     >>>>>>>>> defined
> > >     >>>>>>>>>>>>>>>>>>>>>> fluently.
> > >     >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
> > >     is very
> > >     >>>>>> nice
> > >     >>>>>>>>> and the
> > >     >>>>>>>>>>>>>>>>>>>>>> right
> > >     >>>>>>>>>>>>>>>>>>>>> way
> > >     >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
> > >     how we
> > >     >>>>>>> specify
> > >     >>>>>>>>> the
> > >     >>>>>>>>>>>>> source
> > >     >>>>>>>>>>>>>>>>>>>>>> stream.
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>> Like:
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> > >     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
> > this::handle1)
> > >     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
> > this::handle2)
> > >     >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
> > >     >>>>>>>> KStreamBrancher
> > >     >>>>>>>>> or
> > >     >>>>>>>>>>>>>>>>>>>>> something,
> > >     >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
> > >     terminated by
> > >     >>>>>>>>>>>>> defaultBranch()
> > >     >>>>>>>>>>>>>>>>>>>>>> (which
> > >     >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
> > >     incompatible with
> > >     >>>>> the
> > >     >>>>>>>>> current
> > >     >>>>>>>>>>>>>>>>>>>>>> API, so
> > >     >>>>>>>>>>>>>>>>>>>>> the
> > >     >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
> > >     different
> > >     >>>>>> name,
> > >     >>>>>>>> but
> > >     >>>>>>>>> that
> > >     >>>>>>>>>>>>>>>>>>>>>> seems
> > >     >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
> > >     >>>>>> something
> > >     >>>>>>>> like
> > >     >>>>>>>>>>>>>>>>>>>>>> branched()
> > >     >>>>>>>>>>>>>>>>>>>>> or
> > >     >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
> > >     KIP?  It
> > >     >>>>>> seems
> > >     >>>>>>>>> like it
> > >     >>>>>>>>>>>>>>>>>>>>>> does to
> > >     >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
> > >     while also
> > >     >>>>>>>> allowing
> > >     >>>>>>>>> you
> > >     >>>>>>>>>>>>> to
> > >     >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
> > >     KBranchedStreams
> > >     >>>>>> if
> > >     >>>>>>>>> desired.
> > >     >>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >     >>>>>>>>>>>>>>>>>>>>>> Paul
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> > >     >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> > >     >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >     >>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
> > >     ks){
> > >     >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
> > >     >>>>>>>>>>>>>>>>>>>>>>> }
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
> > >     String> ks){
> > >     >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
> > >     >>>>>>>>>>>>>>>>>>>>>>> }
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> ......
> > >     >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> > >     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> > >     this::handleFirstCase)
> > >     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> > >     this::handleSecondCase)
> > >     >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> Ivan
> > >     >>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> > >     >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> > KafkaStreamsBrancher
> > >     >>>>> takes a
> > >     >>>>>>>>> Consumer
> > >     >>>>>>>>>>>>> as a
> > >     >>>>>>>>>>>>>>>>>>>>>>> second
> > >     >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
> > >     example in
> > >     >>>>>> the
> > >     >>>>>>>> KIP
> > >     >>>>>>>>>>>>> shows
> > >     >>>>>>>>>>>>>>>>>>>>>>>> each
> > >     >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
> > >     >>>>>>>>> (KafkaStreams#to()
> > >     >>>>>>>>>>>>>>>>>>>>>>>> in this
> > >     >>>>>>>>>>>>>>>>>>>>>>>> case).
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would
> > we
> > >     >>>>> handle
> > >     >>>>>>> the
> > >     >>>>>>>>> case
> > >     >>>>>>>>>>>>>>>>>>>>>>>> where the
> > >     >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to
> > continue
> > >     >>>>>>>> processing
> > >     >>>>>>>>> and
> > >     >>>>>>>>>>>>> not
> > >     >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the
> > branched
> > >     >>>>>> stream
> > >     >>>>>>>>>>>>> immediately?
> > >     >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
> > >     we had
> > >     >>>>>>>> something
> > >     >>>>>>>>> like
> > >     >>>>>>>>>>>>>>>>>>>>>>>> this:
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
> > >     >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> > >     >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> > >     >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> > >     >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> > >     >>>>>>>>>>>>>>>>>>>>>>>> Bill
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck
> > <
> > >     >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> > >     >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >     >>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> All,
> > >     >>>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for
> > KIP-
> > >     >>>>> 418.
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> > >     >>>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> > >     >>>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
> > KIP-418.
> > >     >>>>> Please
> > >     >>>>>>>> take
> > >     >>>>>>>>> a
> > >     >>>>>>>>>>>>> look
> > >     >>>>>>>>>>>>>>>>> at
> > >     >>>>>>>>>>>>>>>>>>>>> the
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
> > >     feedback :)
> > >     >>>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> > >     >>>>>
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> > >     >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> > >     >>>>> https://github.com/apache/kafka/pull/6164
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >     >>>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> > >     >>>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>>>>>>>>>>>>>>>>>
> > >     >>>>>>>>>
> > >     >
> > >
> >

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Michael Drogalis <mi...@confluent.io>.
Matthias: I think that's pretty reasonable from my point of view. Good
suggestion.

On Thu, May 23, 2019 at 9:50 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> Interesting discussion.
>
> I am wondering, if we cannot unify the advantage of both approaches:
>
>
>
> KStream#split() -> KBranchedStream
>
> // branch is not easily accessible in current scope
> KBranchedStream#branch(Predicate, Consumer<KStream>)
>   -> KBranchedStream
>
> // assign a name to the branch and
> // return the sub-stream to the current scope later
> //
> // can be simple as `#branch(p, s->s, "name")`
> // or also complex as `#branch(p, s->s.filter(...), "name")`
> KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
>   -> KBranchedStream
>
> // default branch is not easily accessible
> // return map of all named sub-stream into current scope
> KBranchedStream#default(Cosumer<KStream>)
>   -> Map<String,KStream>
>
> // assign custom name to default-branch
> // return map of all named sub-stream into current scope
> KBranchedStream#default(Function<KStream,KStream>, String)
>   -> Map<String,KStream>
>
> // assign a default name for default
> // return map of all named sub-stream into current scope
> KBranchedStream#defaultBranch(Function<KStream,KStream>)
>   -> Map<String,KStream>
>
> // return map of all names sub-stream into current scope
> KBranchedStream#noDefaultBranch()
>   -> Map<String,KStream>
>
>
>
> Hence, for each sub-stream, the user can pick to add a name and return
> the branch "result" to the calling scope or not. The implementation can
> also check at runtime that all returned names are unique. The returned
> Map can be empty and it's also optional to use the Map.
>
> To me, it seems like a good way to get best of both worlds.
>
> Thoughts?
>
>
>
> -Matthias
>
>
>
>
> On 5/6/19 5:15 PM, John Roesler wrote:
> > Ivan,
> >
> > That's a very good point about the "start" operator in the dynamic case.
> > I had no problem with "split()"; I was just questioning the necessity.
> > Since you've provided a proof of necessity, I'm in favor of the
> > "split()" start operator. Thanks!
> >
> > Separately, I'm interested to see where the present discussion leads.
> > I've written enough Javascript code in my life to be suspicious of
> > nested closures. You have a good point about using method references (or
> > indeed function literals also work). It should be validating that this
> > was also the JS community's first approach to flattening the logic when
> > their nested closure situation got out of hand. Unfortunately, it's
> > replacing nesting with redirection, both of which disrupt code
> > readability (but in different ways for different reasons). In other
> > words, I agree that function references is *the* first-order solution if
> > the nested code does indeed become a problem.
> >
> > However, the history of JS also tells us that function references aren't
> > the end of the story either, and you can see that by observing that
> > there have been two follow-on eras, as they continue trying to cope with
> > the consequences of living in such a callback-heavy language. First, you
> > have Futures/Promises, which essentially let you convert nested code to
> > method-chained code (Observables/FP is a popular variation on this).
> > Most lately, you have async/await, which is an effort to apply language
> > (not just API) syntax to the problem, and offer the "flattest" possible
> > programming style to solve the problem (because you get back to just one
> > code block per functional unit).
> >
> > Stream-processing is a different domain, and Java+KStreams is nowhere
> > near as callback heavy as JS, so I don't think we have to take the JS
> > story for granted, but then again, I think we can derive some valuable
> > lessons by looking sideways to adjacent domains. I'm just bringing this
> > up to inspire further/deeper discussion. At the same time, just like JS,
> > we can afford to take an iterative approach to the problem.
> >
> > Separately again, I'm interested in the post-branch merge (and I'd also
> > add join) problem that Paul brought up. We can clearly punt on it, by
> > terminating the nested branches with sink operators. But is there a DSL
> > way to do it?
> >
> > Thanks again for your driving this,
> > -John
> >
> > On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
> > <ma...@gmail.com>> wrote:
> >
> >     Ivan, I’ll definitely forfeit my point on the clumsiness of the
> >     branch(predicate, consumer) solution, I don’t see any real drawbacks
> >     for the dynamic case.
> >
> >     IMO the one trade off to consider at this point is the scope
> >     question. I don’t know if I totally agree that “we rarely need them
> >     in the same scope” since merging the branches back together later
> >     seems like a perfectly plausible use case that can be a lot nicer
> >     when the branched streams are in the same scope. That being said,
> >     for the reasons Ivan listed, I think it is overall the better
> >     solution - working around the scope thing is easy enough if you need
> >     to.
> >
> >     > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
> >     <ip...@mail.ru.invalid> wrote:
> >     >
> >     > Hello everyone, thank you all for joining the discussion!
> >     >
> >     > Well, I don't think the idea of named branches, be it a
> >     LinkedHashMap (no other Map will do, because order of definition
> >     matters) or `branch` method  taking name and Consumer has more
> >     advantages than drawbacks.
> >     >
> >     > In my opinion, the only real positive outcome from Michael's
> >     proposal is that all the returned branches are in the same scope.
> >     But 1) we rarely need them in the same scope 2) there is a
> >     workaround for the scope problem, described in the KIP.
> >     >
> >     > 'Inlining the complex logic' is not a problem, because we can use
> >     method references instead of lambdas. In real world scenarios you
> >     tend to split the complex logic to methods anyway, so the code is
> >     going to be clean.
> >     >
> >     > The drawbacks are strong. The cohesion between predicates and
> >     handlers is lost. We have to define predicates in one place, and
> >     handlers in another. This opens the door for bugs:
> >     >
> >     > - what if we forget to define a handler for a name? or a name for
> >     a handler?
> >     > - what if we misspell a name?
> >     > - what if we copy-paste and duplicate a name?
> >     >
> >     > What Michael propose would have been totally OK if we had been
> >     writing the API in Lua, Ruby or Python. In those languages the
> >     "dynamic naming" approach would have looked most concise and
> >     beautiful. But in Java we expect all the problems related to
> >     identifiers to be eliminated in compile time.
> >     >
> >     > Do we have to invent duck-typing for the Java API?
> >     >
> >     > And if we do, what advantage are we supposed to get besides having
> >     all the branches in the same scope? Michael, maybe I'm missing your
> >     point?
> >     >
> >     > ---
> >     >
> >     > Earlier in this discussion John Roesler also proposed to do
> >     without "start branching" operator, and later Paul mentioned that in
> >     the case when we have to add a dynamic number of branches, the
> >     current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
> >     me address both comments here.
> >     >
> >     > 1) "Start branching" operator (I think that *split* is a good name
> >     for it indeed) is critical when we need to do a dynamic branching,
> >     see example below.
> >     >
> >     > 2) No, dynamic branching in current KIP is not clumsy at all.
> >     Imagine a real-world scenario when you need one branch per enum
> >     value (say, RecordType). You can have something like this:
> >     >
> >     > /*John:if we had to start with stream.branch(...) here, it would
> >     have been much messier.*/
> >     > KBranchedStream branched = stream.split();
> >     >
> >     > /*Not clumsy at all :-)*/
> >     > for (RecordType recordType : RecordType.values())
> >     >             branched = branched.branch((k, v) -> v.getRecType() ==
> >     recordType,
> >     >                     recordType::processRecords);
> >     >
> >     > Regards,
> >     >
> >     > Ivan
> >     >
> >     >
> >     > 02.05.2019 14:40, Matthias J. Sax пишет:
> >     >> I also agree with Michael's observation about the core problem of
> >     >> current `branch()` implementation.
> >     >>
> >     >> However, I also don't like to pass in a clumsy Map object. My
> >     thinking
> >     >> was more aligned with Paul's proposal to just add a name to each
> >     >> `branch()` statement and return a `Map<String,KStream>`.
> >     >>
> >     >> It makes the code easier to read, and also make the order of
> >     >> `Predicates` (that is essential) easier to grasp.
> >     >>
> >     >>>>>> Map<String, KStream<K, V>> branches = stream.split()
> >     >>>>>>    .branch("branchOne", Predicate<K, V>)
> >     >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >     >>>>>>    .defaultBranch("defaultBranch");
> >     >> An open question is the case for which no defaultBranch() should
> be
> >     >> specified. Atm, `split()` and `branch()` would return
> >     `BranchedKStream`
> >     >> and the call to `defaultBranch()` that returns the `Map` is
> mandatory
> >     >> (what is not the case atm). Or is this actually not a real
> problem,
> >     >> because users can just ignore the branch returned by
> >     `defaultBranch()`
> >     >> in the result `Map` ?
> >     >>
> >     >>
> >     >> About "inlining": So far, it seems to be a matter of personal
> >     >> preference. I can see arguments for both, but no "killer
> >     argument" yet
> >     >> that clearly make the case for one or the other.
> >     >>
> >     >>
> >     >> -Matthias
> >     >>
> >     >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >     >>> Perhaps inlining is the wrong terminology. It doesn’t require
> >     that a lambda with the full downstream topology be defined inline -
> >     it can be a method reference as with Ivan’s original suggestion.
> >     The advantage of putting the predicate and its downstream logic
> >     (Consumer) together in branch() is that they are required to be near
> >     to each other.
> >     >>>
> >     >>> Ultimately the downstream code has to live somewhere, and deep
> >     branch trees will be hard to read regardless.
> >     >>>
> >     >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
> >     <michael.drogalis@confluent.io
> >     <ma...@confluent.io>> wrote:
> >     >>>>
> >     >>>> I'm less enthusiastic about inlining the branch logic with its
> >     downstream
> >     >>>> functionality. Programs that have deep branch trees will
> >     quickly become
> >     >>>> harder to read as a single unit.
> >     >>>>
> >     >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
> >     <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
> >     >>>>>
> >     >>>>> Also +1 on the issues/goals as Michael outlined them, I think
> >     that sets a
> >     >>>>> great framework for the discussion.
> >     >>>>>
> >     >>>>> Regarding the SortedMap solution, my understanding is that the
> >     current
> >     >>>>> proposal in the KIP is what is in my PR which (pending naming
> >     decisions) is
> >     >>>>> roughly this:
> >     >>>>>
> >     >>>>> stream.split()
> >     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >     >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >     >>>>>
> >     >>>>> Obviously some ordering is necessary, since branching as a
> >     construct
> >     >>>>> doesn't work without it, but this solution seems like it
> >     provides as much
> >     >>>>> associativity as the SortedMap solution, because each branch()
> >     call
> >     >>>>> directly associates the "conditional" with the "code block."
> >     The value it
> >     >>>>> provides over the KIP solution is the accessing of streams in
> >     the same
> >     >>>>> scope.
> >     >>>>>
> >     >>>>> The KIP solution is less "dynamic" than the SortedMap solution
> >     in the sense
> >     >>>>> that it is slightly clumsier to add a dynamic number of
> >     branches, but it is
> >     >>>>> certainly possible.  It seems to me like the API should favor
> >     the "static"
> >     >>>>> case anyway, and should make it simple and readable to
> >     fluently declare and
> >     >>>>> access your branches in-line.  It also makes it impossible to
> >     ignore a
> >     >>>>> branch, and it is possible to build an (almost) identical
> >     SortedMap
> >     >>>>> solution on top of it.
> >     >>>>>
> >     >>>>> I could also see a middle ground where instead of a raw
> >     SortedMap being
> >     >>>>> taken in, branch() takes a name and not a Consumer.  Something
> >     like this:
> >     >>>>>
> >     >>>>> Map<String, KStream<K, V>> branches = stream.split()
> >     >>>>>    .branch("branchOne", Predicate<K, V>)
> >     >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >     >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
> >     >>>>>
> >     >>>>> Pros for that solution:
> >     >>>>> - accessing branched KStreams in same scope
> >     >>>>> - no double brace initialization, hopefully slightly more
> >     readable than
> >     >>>>> SortedMap
> >     >>>>>
> >     >>>>> Cons
> >     >>>>> - downstream branch logic cannot be specified inline which
> >     makes it harder
> >     >>>>> to read top to bottom (like existing API and SortedMap, but
> >     unlike the KIP)
> >     >>>>> - you can forget to "handle" one of the branched streams (like
> >     existing
> >     >>>>> API and SortedMap, but unlike the KIP)
> >     >>>>>
> >     >>>>> (KBranchedStreams could even work *both* ways but perhaps
> >     that's overdoing
> >     >>>>> it).
> >     >>>>>
> >     >>>>> Overall I'm curious how important it is to be able to easily
> >     access the
> >     >>>>> branched KStream in the same scope as the original.  It's
> >     possible that it
> >     >>>>> doesn't need to be handled directly by the API, but instead
> >     left up to the
> >     >>>>> user.  I'm sort of in the middle on it.
> >     >>>>>
> >     >>>>> Paul
> >     >>>>>
> >     >>>>>
> >     >>>>>
> >     >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
> >     <sophie@confluent.io <ma...@confluent.io>>
> >     >>>>> wrote:
> >     >>>>>
> >     >>>>>> I'd like to +1 what Michael said about the issues with the
> >     existing
> >     >>>>> branch
> >     >>>>>> method, I agree with what he's outlined and I think we should
> >     proceed by
> >     >>>>>> trying to alleviate these problems. Specifically it seems
> >     important to be
> >     >>>>>> able to cleanly access the individual branches (eg by mapping
> >     >>>>>> name->stream), which I thought was the original intention of
> >     this KIP.
> >     >>>>>>
> >     >>>>>> That said, I don't think we should so easily give in to the
> >     double brace
> >     >>>>>> anti-pattern or force ours users into it if at all possible to
> >     >>>>> avoid...just
> >     >>>>>> my two cents.
> >     >>>>>>
> >     >>>>>> Cheers,
> >     >>>>>> Sophie
> >     >>>>>>
> >     >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
> >     >>>>>> michael.drogalis@confluent.io
> >     <ma...@confluent.io>> wrote:
> >     >>>>>>
> >     >>>>>>> I’d like to propose a different way of thinking about this.
> >     To me,
> >     >>>>> there
> >     >>>>>>> are three problems with the existing branch signature:
> >     >>>>>>>
> >     >>>>>>> 1. If you use it the way most people do, Java raises unsafe
> type
> >     >>>>>> warnings.
> >     >>>>>>> 2. The way in which you use the stream branches is
> >     positionally coupled
> >     >>>>>> to
> >     >>>>>>> the ordering of the conditionals.
> >     >>>>>>> 3. It is brittle to extend existing branch calls with
> >     additional code
> >     >>>>>>> paths.
> >     >>>>>>>
> >     >>>>>>> Using associative constructs instead of relying on ordered
> >     constructs
> >     >>>>>> would
> >     >>>>>>> be a stronger approach. Consider a signature that instead
> >     looks like
> >     >>>>>> this:
> >     >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
> >     Predicate<?
> >     >>>>>>> super K,? super V>>);
> >     >>>>>>>
> >     >>>>>>> Branches are given names in a map, and as a result, the API
> >     returns a
> >     >>>>>>> mapping of names to streams. The ordering of the
> conditionals is
> >     >>>>>> maintained
> >     >>>>>>> because it’s a sorted map. Insert order determines the order
> of
> >     >>>>>> evaluation.
> >     >>>>>>> This solves problem 1 because there are no more varargs. It
> >     solves
> >     >>>>>> problem
> >     >>>>>>> 2 because you no longer lean on ordering to access the
> >     branch you’re
> >     >>>>>>> interested in. It solves problem 3 because you can introduce
> >     another
> >     >>>>>>> conditional by simply attaching another name to the
> >     structure, rather
> >     >>>>>> than
> >     >>>>>>> messing with the existing indices.
> >     >>>>>>>
> >     >>>>>>> One of the drawbacks is that creating the map inline is
> >     historically
> >     >>>>>>> awkward in Java. I know it’s an anti-pattern to use
> >     voluminously, but
> >     >>>>>>> double brace initialization would clean up the aesthetics.
> >     >>>>>>>
> >     >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
> >     <john@confluent.io <ma...@confluent.io>>
> >     >>>>> wrote:
> >     >>>>>>>> Hi Ivan,
> >     >>>>>>>>
> >     >>>>>>>> Thanks for the update.
> >     >>>>>>>>
> >     >>>>>>>> FWIW, I agree with Matthias that the current "start
> branching"
> >     >>>>> operator
> >     >>>>>>> is
> >     >>>>>>>> confusing when named the same way as the actual branches.
> >     "Split"
> >     >>>>> seems
> >     >>>>>>>> like a good name. Alternatively, we can do without a "start
> >     >>>>> branching"
> >     >>>>>>>> operator at all, and just do:
> >     >>>>>>>>
> >     >>>>>>>> stream
> >     >>>>>>>>      .branch(Predicate)
> >     >>>>>>>>      .branch(Predicate)
> >     >>>>>>>>      .defaultBranch();
> >     >>>>>>>>
> >     >>>>>>>> Tentatively, I think that this branching operation should be
> >     >>>>> terminal.
> >     >>>>>>> That
> >     >>>>>>>> way, we don't create ambiguity about how to use it. That
> >     is, `branch`
> >     >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
> >     `void`, to
> >     >>>>>>>> enforce that it comes last, and that there is only one
> >     definition of
> >     >>>>>> the
> >     >>>>>>>> default branch. Potentially, we should log a warning if
> >     there's no
> >     >>>>>>> default,
> >     >>>>>>>> and additionally log a warning (or throw an exception) if a
> >     record
> >     >>>>>> falls
> >     >>>>>>>> though with no default.
> >     >>>>>>>>
> >     >>>>>>>> Thoughts?
> >     >>>>>>>>
> >     >>>>>>>> Thanks,
> >     >>>>>>>> -John
> >     >>>>>>>>
> >     >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
> >     >>>>> matthias@confluent.io <ma...@confluent.io>
> >     >>>>>>>> wrote:
> >     >>>>>>>>
> >     >>>>>>>>> Thanks for updating the KIP and your answers.
> >     >>>>>>>>>
> >     >>>>>>>>>
> >     >>>>>>>>>> this is to make the name similar to String#split
> >     >>>>>>>>>>> that also returns an array, right?
> >     >>>>>>>>> The intend was to avoid name duplication. The return type
> >     should
> >     >>>>>> _not_
> >     >>>>>>>>> be an array.
> >     >>>>>>>>>
> >     >>>>>>>>> The current proposal is
> >     >>>>>>>>>
> >     >>>>>>>>> stream.branch()
> >     >>>>>>>>>      .branch(Predicate)
> >     >>>>>>>>>      .branch(Predicate)
> >     >>>>>>>>>      .defaultBranch();
> >     >>>>>>>>>
> >     >>>>>>>>> IMHO, this reads a little odd, because the first
> >     `branch()` does
> >     >>>>> not
> >     >>>>>>>>> take any parameters and has different semantics than the
> later
> >     >>>>>>>>> `branch()` calls. Note, that from the code snippet above,
> it's
> >     >>>>> hidden
> >     >>>>>>>>> that the first call is `KStream#branch()` while the others
> are
> >     >>>>>>>>> `KBranchedStream#branch()` what makes reading the code
> harder.
> >     >>>>>>>>>
> >     >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
> >     I though
> >     >>>>>> it
> >     >>>>>>>>> might be better to also rename `KStream#branch()` to avoid
> the
> >     >>>>> naming
> >     >>>>>>>>> overlap that seems to be confusing. The following reads
> much
> >     >>>>> cleaner
> >     >>>>>> to
> >     >>>>>>>> me:
> >     >>>>>>>>> stream.split()
> >     >>>>>>>>>      .branch(Predicate)
> >     >>>>>>>>>      .branch(Predicate)
> >     >>>>>>>>>      .defaultBranch();
> >     >>>>>>>>>
> >     >>>>>>>>> Maybe there is a better alternative to `split()` though to
> >     avoid
> >     >>>>> the
> >     >>>>>>>>> naming overlap.
> >     >>>>>>>>>
> >     >>>>>>>>>
> >     >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately
> we
> >     >>>>> cannot
> >     >>>>>>> have
> >     >>>>>>>>> a method with such name :-)
> >     >>>>>>>>>
> >     >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
> >     with a
> >     >>>>> short
> >     >>>>>>>> name?
> >     >>>>>>>>>
> >     >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
> >     with all
> >     >>>>> it's
> >     >>>>>>>>> methods? It will be part of public API and should be
> >     contained in
> >     >>>>> the
> >     >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
> >     >>>>>>>>> `defaultBranch()` is.
> >     >>>>>>>>>
> >     >>>>>>>>>
> >     >>>>>>>>> You did not comment on the idea to add a
> >     `KBranchedStream#get(int
> >     >>>>>>> index)
> >     >>>>>>>>> -> KStream` method to get the individually
> >     branched-KStreams. Would
> >     >>>>>> be
> >     >>>>>>>>> nice to get your feedback about it. It seems you suggest
> >     that users
> >     >>>>>>>>> would need to write custom utility code otherwise, to
> >     access them.
> >     >>>>> We
> >     >>>>>>>>> should discuss the pros and cons of both approaches. It
> feels
> >     >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
> >     to get
> >     >>>>> the
> >     >>>>>>>>> branched-KStreams directly.
> >     >>>>>>>>>
> >     >>>>>>>>>
> >     >>>>>>>>>
> >     >>>>>>>>> -Matthias
> >     >>>>>>>>>
> >     >>>>>>>>>
> >     >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >     >>>>>>>>>> Hi all!
> >     >>>>>>>>>>
> >     >>>>>>>>>> I have updated the KIP-418 according to the new vision.
> >     >>>>>>>>>>
> >     >>>>>>>>>> Matthias, thanks for your comment!
> >     >>>>>>>>>>
> >     >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >     >>>>>>>>>> I can see your point: this is to make the name similar to
> >     >>>>>>> String#split
> >     >>>>>>>>>> that also returns an array, right? But is it worth the
> >     loss of
> >     >>>>>>>> backwards
> >     >>>>>>>>>> compatibility? We can have overloaded branch() as well
> >     without
> >     >>>>>>>> affecting
> >     >>>>>>>>>> the existing code. Maybe the old array-based `branch`
> method
> >     >>>>> should
> >     >>>>>>> be
> >     >>>>>>>>>> deprecated, but this is a subject for discussion.
> >     >>>>>>>>>>
> >     >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >     >>>>> BranchingKStream#branch(),
> >     >>>>>>>>>> KBranchedStream#defaultBranch() ->
> BranchingKStream#default()
> >     >>>>>>>>>>
> >     >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default'
> is,
> >     >>>>>>> however, a
> >     >>>>>>>>>> reserved word, so unfortunately we cannot have a method
> >     with such
> >     >>>>>>> name
> >     >>>>>>>>> :-)
> >     >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument,
> but I
> >     >>>>> think
> >     >>>>>>> that
> >     >>>>>>>>>> is not required?
> >     >>>>>>>>>>
> >     >>>>>>>>>> Absolutely! I think that was just copy-paste error or
> >     something.
> >     >>>>>>>>>>
> >     >>>>>>>>>> Dear colleagues,
> >     >>>>>>>>>>
> >     >>>>>>>>>> please revise the new version of the KIP and Paul's PR
> >     >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >     >>>>>>>>>>
> >     >>>>>>>>>> Any new suggestions/objections?
> >     >>>>>>>>>>
> >     >>>>>>>>>> Regards,
> >     >>>>>>>>>>
> >     >>>>>>>>>> Ivan
> >     >>>>>>>>>>
> >     >>>>>>>>>>
> >     >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >     >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems
> that
> >     >>>>>>> everybody
> >     >>>>>>>>>>> agrees that the current branch() method using arrays is
> not
> >     >>>>>> optimal.
> >     >>>>>>>>>>> I had a quick look into the PR and I like the overall
> >     proposal.
> >     >>>>>>> There
> >     >>>>>>>>>>> are some minor things we need to consider. I would
> >     recommend the
> >     >>>>>>>>>>> following renaming:
> >     >>>>>>>>>>>
> >     >>>>>>>>>>> KStream#branch() -> #split()
> >     >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
> >     >>>>>>>>>>> KBranchedStream#defaultBranch() ->
> >     BranchingKStream#default()
> >     >>>>>>>>>>>
> >     >>>>>>>>>>> It's just a suggestion to get slightly shorter method
> names.
> >     >>>>>>>>>>>
> >     >>>>>>>>>>> In the current PR, defaultBranch() does take an
> >     `Predicate` as
> >     >>>>>>>> argument,
> >     >>>>>>>>>>> but I think that is not required?
> >     >>>>>>>>>>>
> >     >>>>>>>>>>> Also, we should consider KIP-307, that was recently
> >     accepted and
> >     >>>>>> is
> >     >>>>>>>>>>> currently implemented:
> >     >>>>>>>>>>>
> >     >>>>>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >     >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
> >     parameter.
> >     >>>>>>>>>>>
> >     >>>>>>>>>>>
> >     >>>>>>>>>>> For the issue that the created `KStream` object are in
> >     different
> >     >>>>>>>> scopes:
> >     >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
> >     index)` method
> >     >>>>>>> that
> >     >>>>>>>>>>> returns the corresponding "branched" result `KStream`
> >     object?
> >     >>>>>> Maybe,
> >     >>>>>>>> the
> >     >>>>>>>>>>> second argument of `addBranch()` should not be a
> >     >>>>>> `Consumer<KStream>`
> >     >>>>>>>> but
> >     >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
> >     whatever
> >     >>>>>> the
> >     >>>>>>>>>>> `Function` returns?
> >     >>>>>>>>>>>
> >     >>>>>>>>>>>
> >     >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
> >     current
> >     >>>>>>>>>>> proposal. That makes it easier to review.
> >     >>>>>>>>>>>
> >     >>>>>>>>>>>
> >     >>>>>>>>>>> -Matthias
> >     >>>>>>>>>>>
> >     >>>>>>>>>>>
> >     >>>>>>>>>>>
> >     >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >     >>>>>>>>>>>> Ivan,
> >     >>>>>>>>>>>>
> >     >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
> >     makes sense
> >     >>>>>> for
> >     >>>>>>>> you
> >     >>>>>>>>> to
> >     >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
> >     we'll
> >     >>>>> need
> >     >>>>>>>> some
> >     >>>>>>>>>>>> buy-in from committers that have actual binding votes on
> >     >>>>> whether
> >     >>>>>>> the
> >     >>>>>>>>> KIP
> >     >>>>>>>>>>>> could be adopted.  It would be great to hear if they
> >     think this
> >     >>>>>> is
> >     >>>>>>> a
> >     >>>>>>>>> good
> >     >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
> >     starting a
> >     >>>>>>> vote,
> >     >>>>>>>>> or if
> >     >>>>>>>>>>>> there is generally some indication of interest
> beforehand.
> >     >>>>>>>>>>>>
> >     >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
> >     assuming
> >     >>>>> we
> >     >>>>>> do
> >     >>>>>>>>> move
> >     >>>>>>>>>>>> forward the solution of "stream.branch() returns
> >     >>>>>> KBranchedStream",
> >     >>>>>>> do
> >     >>>>>>>>> we
> >     >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I
> would
> >     >>>>> favor
> >     >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs
> that
> >     >>>>>>> accomplish
> >     >>>>>>>>> the
> >     >>>>>>>>>>>> same thing is confusing, especially when they're fairly
> >     similar
> >     >>>>>>>>> anyway.  We
> >     >>>>>>>>>>>> just need to be sure we're not making something
> >     >>>>>>> impossible/difficult
> >     >>>>>>>>> that
> >     >>>>>>>>>>>> is currently possible/easy.
> >     >>>>>>>>>>>>
> >     >>>>>>>>>>>> Regarding my PR - I think the general structure would
> work,
> >     >>>>> it's
> >     >>>>>>>> just a
> >     >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
> >     >>>>>>> particular,
> >     >>>>>>>>>>>> passing in the "predicates" and "children" lists which
> get
> >     >>>>>> modified
> >     >>>>>>>> in
> >     >>>>>>>>>>>> KBranchedStream but read from all the way
> >     KStreamLazyBranch is
> >     >>>>> a
> >     >>>>>>> bit
> >     >>>>>>>>>>>> complicated to follow.
> >     >>>>>>>>>>>>
> >     >>>>>>>>>>>> Thanks,
> >     >>>>>>>>>>>> Paul
> >     >>>>>>>>>>>>
> >     >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
> >     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >     >>>>>>>>> wrote:
> >     >>>>>>>>>>>>> Hi Paul!
> >     >>>>>>>>>>>>>
> >     >>>>>>>>>>>>> I read your code carefully and now I am fully
> >     convinced: your
> >     >>>>>>>> proposal
> >     >>>>>>>>>>>>> looks better and should work. We just have to document
> the
> >     >>>>>> crucial
> >     >>>>>>>>> fact
> >     >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
> >     And then
> >     >>>>>> it's
> >     >>>>>>>> all
> >     >>>>>>>>>>>>> going to be very nice.
> >     >>>>>>>>>>>>>
> >     >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
> >     resume the
> >     >>>>>>>>>>>>> discussion here, right?
> >     >>>>>>>>>>>>>
> >     >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
> >     >>>>> starting
> >     >>>>>>>> point
> >     >>>>>>>>> if
> >     >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
> >     starting
> >     >>>>>>> point.
> >     >>>>>>>>> But
> >     >>>>>>>>>>>>> as a novice in this project I might miss some important
> >     >>>>> details.
> >     >>>>>>>>>>>>> Regards,
> >     >>>>>>>>>>>>>
> >     >>>>>>>>>>>>> Ivan
> >     >>>>>>>>>>>>>
> >     >>>>>>>>>>>>>
> >     >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >     >>>>>>>>>>>>>> Ivan,
> >     >>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
> >     >>>>> stream.branch()
> >     >>>>>>>>> solution
> >     >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
> >     >>>>> invoked
> >     >>>>>> as
> >     >>>>>>>>> they’re
> >     >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
> >     still
> >     >>>>>> ought
> >     >>>>>>> to
> >     >>>>>>>>> be
> >     >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
> >     depend on
> >     >>>>> the
> >     >>>>>>>>> branched
> >     >>>>>>>>>>>>> streams having been set.
> >     >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
> >     access
> >     >>>>> the
> >     >>>>>>>>> branched
> >     >>>>>>>>>>>>> streams in the same scope as the original stream (that
> >     is, not
> >     >>>>>>>> inside
> >     >>>>>>>>> the
> >     >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
> >     >>>>> solutions.
> >     >>>>>> It
> >     >>>>>>>>> can be
> >     >>>>>>>>>>>>> worked around though.
> >     >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
> >     excited
> >     >>>>> to
> >     >>>>>>>> hear
> >     >>>>>>>>>>>>> your thoughts!]
> >     >>>>>>>>>>>>>> Paul
> >     >>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
> >     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >     >>>>>>>>> wrote:
> >     >>>>>>>>>>>>>>> Hi Paul!
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
> >     >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at
> first
> >     >>>>> glance,
> >     >>>>>>> but
> >     >>>>>>>>> ---
> >     >>>>>>>>>>>>>>>> the newly branched streams are not available in the
> >     same
> >     >>>>>> scope
> >     >>>>>>> as
> >     >>>>>>>>> each
> >     >>>>>>>>>>>>> other.  That is, if we wanted to merge them back
> together
> >     >>>>> again
> >     >>>>>> I
> >     >>>>>>>>> don't see
> >     >>>>>>>>>>>>> a way to do that.
> >     >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
> >     just
> >     >>>>>> going
> >     >>>>>>> to
> >     >>>>>>>>>>>>> write in details about this issue.
> >     >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
> >     we need
> >     >>>>> to
> >     >>>>>>>>> identify
> >     >>>>>>>>>>>>> customers who have bought coffee and made a purchase
> >     in the
> >     >>>>>>>>> electronics
> >     >>>>>>>>>>>>> store to give them coupons.
> >     >>>>>>>>>>>>>>> This is the code I usually write under these
> >     circumstances
> >     >>>>>> using
> >     >>>>>>>> my
> >     >>>>>>>>>>>>> 'brancher' class:
> >     >>>>>>>>>>>>>>> @Setter
> >     >>>>>>>>>>>>>>> class CouponIssuer{
> >     >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
> >     >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >     >>>>>>>>>>>>>>>       return
> >     >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >     >>>>>>>>>>>>>>>       /*In the real world the code here can be
> >     complex, so
> >     >>>>>>>>> creation of
> >     >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
> >     order to
> >     >>>>>>>> separate
> >     >>>>>>>>>>>>> classes' responsibilities.*/
> >     >>>>>>>>>>>>>>>  }
> >     >>>>>>>>>>>>>>> }
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >     >>>>>>>>>>>>>>>     .branch(predicate1,
> couponIssuer::setCoffePurchases)
> >     >>>>>>>>>>>>>>>     .branch(predicate2,
> >     >>>>>> couponIssuer::setElectronicsPurchases)
> >     >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
> >     everything
> >     >>>>>>>> later,
> >     >>>>>>>>>>>>> without the terminal operation!!!*/
> >     >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> Does this make sense?  In order to properly
> >     initialize the
> >     >>>>>>>>> CouponIssuer
> >     >>>>>>>>>>>>> we need the terminal operation to be called before
> >     >>>>>>>>> streamsBuilder.build()
> >     >>>>>>>>>>>>> is called.
> >     >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
> >     essentially
> >     >>>>>> the
> >     >>>>>>>>> next
> >     >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
> >     based on
> >     >>>>> my
> >     >>>>>>>>> experience,
> >     >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
> >     >>>>>>>>>>>>>>> Regards,
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> Ivan
> >     >>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >     >>>>>>>>>>>>>>>> Ivan,
> >     >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
> >     fluent
> >     >>>>> API
> >     >>>>>>>> based
> >     >>>>>>>>>>>>> off of
> >     >>>>>>>>>>>>>>>> KStream here
> >     (https://github.com/apache/kafka/pull/6512),
> >     >>>>>> and
> >     >>>>>>> I
> >     >>>>>>>>> think
> >     >>>>>>>>>>>>> I
> >     >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >     >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
> >     >>>>>>> compatibility
> >     >>>>>>>>>>>>> issues,
> >     >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
> >     that Java
> >     >>>>> is
> >     >>>>>>>> smart
> >     >>>>>>>>>>>>> enough to
> >     >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
> >     returning one
> >     >>>>>>> thing
> >     >>>>>>>>> and
> >     >>>>>>>>>>>>> branch()
> >     >>>>>>>>>>>>>>>>    with no arguments returning another thing.
> >     >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
> >     need
> >     >>>>> it.
> >     >>>>>>> We
> >     >>>>>>>>> can
> >     >>>>>>>>>>>>> just
> >     >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
> >     shares
> >     >>>>>> its
> >     >>>>>>>>> state
> >     >>>>>>>>>>>>> with the
> >     >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
> >     branching.
> >     >>>>>>> It's
> >     >>>>>>>>> not
> >     >>>>>>>>>>>>> terribly
> >     >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
> >     demonstrates
> >     >>>>>> its
> >     >>>>>>>>>>>>> feasibility.
> >     >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should
> be
> >     >>>>> final
> >     >>>>>> or
> >     >>>>>>>>> even a
> >     >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
> >     wanted to
> >     >>>>>> see
> >     >>>>>>>> how
> >     >>>>>>>>>>>>>>>> challenging it would be to get the API working.
> >     >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
> >     solution
> >     >>>>>>> could
> >     >>>>>>>> be
> >     >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
> >     >>>>> suggested
> >     >>>>>>>> was a
> >     >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
> >     streams
> >     >>>>>> are
> >     >>>>>>>> not
> >     >>>>>>>>>>>>>>>> available in the same scope as each other.  That
> >     is, if we
> >     >>>>>>> wanted
> >     >>>>>>>>> to
> >     >>>>>>>>>>>>> merge
> >     >>>>>>>>>>>>>>>> them back together again I don't see a way to do
> >     that.  The
> >     >>>>>> KIP
> >     >>>>>>>>>>>>> proposal
> >     >>>>>>>>>>>>>>>> has the same issue, though - all this means is that
> for
> >     >>>>>> either
> >     >>>>>>>>>>>>> solution,
> >     >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
> >     table.
> >     >>>>>>>>>>>>>>>> Thanks,
> >     >>>>>>>>>>>>>>>> Paul
> >     >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> >     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
> >     >>>>>>>>>>>>> wrote:
> >     >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to
> this
> >     >>>>>> point.
> >     >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
> >     branch API
> >     >>>>>>> needs
> >     >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> There are two potential ways to do it:
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >     >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
> >     returns
> >     >>>>>> its
> >     >>>>>>>>> argument
> >     >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code
> won't
> >     >>>>> make
> >     >>>>>>>> sense
> >     >>>>>>>>>>>>> until
> >     >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
> >     instance
> >     >>>>>>>>> contrasts the
> >     >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> stream
> >     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
> >     >>>>>>>>> defaultBranch(..)
> >     >>>>>>>>>>>>> and
> >     >>>>>>>>>>>>>>>>> noDefault() return void
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface
> is
> >     >>>>>> defined.
> >     >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
> >     >>>>>>>> (defaultBranch(ks->)
> >     >>>>>>>>> and
> >     >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
> >     miss the
> >     >>>>>> fact
> >     >>>>>>>>> that one
> >     >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
> >     methods
> >     >>>>>> are
> >     >>>>>>>> not
> >     >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do
> better?
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> Regards,
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> Ivan
> >     >>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> >     >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> >     >>>>>>>>>>>>>>>>>>> Paul,
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
> >     >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
> >     implemented the
> >     >>>>>>> easy
> >     >>>>>>>>> way.
> >     >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that
> assumes
> >     >>>>> nothing
> >     >>>>>>>> will
> >     >>>>>>>>>>>>> reach
> >     >>>>>>>>>>>>>>>>>>>> the default branch,
> >     >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only
> option
> >     >>>>>> besides
> >     >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
> >     want to
> >     >>>>>> just
> >     >>>>>>>>> silently
> >     >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
> >     predicate. 2)
> >     >>>>>>> Throwing
> >     >>>>>>>>> an
> >     >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
> >     looks
> >     >>>>>> like a
> >     >>>>>>>> bad
> >     >>>>>>>>>>>>> idea.
> >     >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
> >     emit a
> >     >>>>>>>> special
> >     >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly
> where
> >     >>>>>>> `default`
> >     >>>>>>>>> can
> >     >>>>>>>>>>>>> be
> >     >>>>>>>>>>>>>>>>>>> used.
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >     >>>>> InternalTopologyBuilder
> >     >>>>>>> to
> >     >>>>>>>>> track
> >     >>>>>>>>>>>>>>>>>>>> dangling
> >     >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
> >     a clear
> >     >>>>>>> error
> >     >>>>>>>>>>>>> before it
> >     >>>>>>>>>>>>>>>>>>> becomes an issue.
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
> >     >>>>> compiled
> >     >>>>>>> and
> >     >>>>>>>>> run?
> >     >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
> >     compile if
> >     >>>>> used
> >     >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
> >     method chain
> >     >>>>>>>> starting
> >     >>>>>>>>>>>>> from
> >     >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
> >     between
> >     >>>>>>>> runtime
> >     >>>>>>>>> and
> >     >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
> >     >>>>> instantly
> >     >>>>>> on
> >     >>>>>>>>> unit
> >     >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
> >     compilation
> >     >>>>>>>> failure.
> >     >>>>>>>>>>>>>>>>>>> Regards,
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>> Ivan
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> >     >>>>>>>>>>>>>>>>>>>> Ivan,
> >     >>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
> >     required.
> >     >>>>>>> But
> >     >>>>>>>> is
> >     >>>>>>>>>>>>> that
> >     >>>>>>>>>>>>>>>>>>>> really
> >     >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
> >     >>>>>> defaultBranch
> >     >>>>>>>>> they
> >     >>>>>>>>>>>>> can
> >     >>>>>>>>>>>>>>>>>>>> call
> >     >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
> >     just as
> >     >>>>>>>>> easily.  In
> >     >>>>>>>>>>>>>>>>>>>> fact I
> >     >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API
> - a
> >     >>>>> user
> >     >>>>>>>> could
> >     >>>>>>>>>>>>> specify
> >     >>>>>>>>>>>>>>>>> a
> >     >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach
> the
> >     >>>>>> default
> >     >>>>>>>>> branch,
> >     >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
> That
> >     >>>>> seems
> >     >>>>>>> like
> >     >>>>>>>>> an
> >     >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
> >     which allows
> >     >>>>>> for
> >     >>>>>>>> the
> >     >>>>>>>>>>>>> more
> >     >>>>>>>>>>>>>>>>>>>> subtle
> >     >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting
> dropped.
> >     >>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
> >     to be
> >     >>>>>> well
> >     >>>>>>>>>>>>>>>>>>>> documented, but
> >     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >     >>>>> InternalTopologyBuilder
> >     >>>>>>> to
> >     >>>>>>>>> track
> >     >>>>>>>>>>>>>>>>>>>> dangling
> >     >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
> >     a clear
> >     >>>>>>> error
> >     >>>>>>>>>>>>> before it
> >     >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is
> a
> >     >>>>> "build
> >     >>>>>>>> step"
> >     >>>>>>>>>>>>> where
> >     >>>>>>>>>>>>>>>>> the
> >     >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
> >     >>>>>> StreamsBuilder.build()
> >     >>>>>>> is
> >     >>>>>>>>>>>>> called.
> >     >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I
> agree
> >     >>>>> that
> >     >>>>>>> it's
> >     >>>>>>>>>>>>>>>>>>>> critical to
> >     >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
> >     stream.
> >     >>>>>>> With
> >     >>>>>>>>> the
> >     >>>>>>>>>>>>>>>>> fluent
> >     >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all
> other
> >     >>>>>>> operations
> >     >>>>>>>>> do -
> >     >>>>>>>>>>>>> if
> >     >>>>>>>>>>>>>>>>> you
> >     >>>>>>>>>>>>>>>>>>>> want to process off the original KStream
> multiple
> >     >>>>> times,
> >     >>>>>>> you
> >     >>>>>>>>> just
> >     >>>>>>>>>>>>>>>>>>>> need the
> >     >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
> >     operations
> >     >>>>>> on
> >     >>>>>>> it
> >     >>>>>>>>> as
> >     >>>>>>>>>>>>> you
> >     >>>>>>>>>>>>>>>>>>>> desire.
> >     >>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>> Thoughts?
> >     >>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>> Best,
> >     >>>>>>>>>>>>>>>>>>>> Paul
> >     >>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> >     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
> >     >>>>>>>>>>>>>>>>>>>> wrote:
> >     >>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
> >     always need
> >     >>>>>> the
> >     >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal
> operation we
> >     >>>>> don't
> >     >>>>>>>> know
> >     >>>>>>>>>>>>> when to
> >     >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
> >     so we
> >     >>>>> can
> >     >>>>>> do
> >     >>>>>>>>>>>>> something
> >     >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of
> special
> >     >>>>> object
> >     >>>>>>>>>>>>> construction
> >     >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods.
> But
> >     >>>>> here
> >     >>>>>> we
> >     >>>>>>>>> have a
> >     >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
> >     flow,
> >     >>>>> so
> >     >>>>>> I
> >     >>>>>>>>> think
> >     >>>>>>>>>>>>> this
> >     >>>>>>>>>>>>>>>>> is
> >     >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>> Regards,
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>> Ivan
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> >     >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
> >     API, but I
> >     >>>>>> find
> >     >>>>>>>> the
> >     >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >     >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
> >     contrasts the
> >     >>>>>>> fluency
> >     >>>>>>>>> of
> >     >>>>>>>>>>>>> other
> >     >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
> >     just call
> >     >>>>> a
> >     >>>>>>>>> method on
> >     >>>>>>>>>>>>> the
> >     >>>>>>>>>>>>>>>>>>>>> stream
> >     >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
> >     cases
> >     >>>>> are
> >     >>>>>>>>> defined
> >     >>>>>>>>>>>>>>>>>>>>>> fluently.
> >     >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
> >     is very
> >     >>>>>> nice
> >     >>>>>>>>> and the
> >     >>>>>>>>>>>>>>>>>>>>>> right
> >     >>>>>>>>>>>>>>>>>>>>> way
> >     >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
> >     how we
> >     >>>>>>> specify
> >     >>>>>>>>> the
> >     >>>>>>>>>>>>> source
> >     >>>>>>>>>>>>>>>>>>>>>> stream.
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>> Like:
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1,
> this::handle1)
> >     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2,
> this::handle2)
> >     >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
> >     >>>>>>>> KStreamBrancher
> >     >>>>>>>>> or
> >     >>>>>>>>>>>>>>>>>>>>> something,
> >     >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
> >     terminated by
> >     >>>>>>>>>>>>> defaultBranch()
> >     >>>>>>>>>>>>>>>>>>>>>> (which
> >     >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
> >     incompatible with
> >     >>>>> the
> >     >>>>>>>>> current
> >     >>>>>>>>>>>>>>>>>>>>>> API, so
> >     >>>>>>>>>>>>>>>>>>>>> the
> >     >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
> >     different
> >     >>>>>> name,
> >     >>>>>>>> but
> >     >>>>>>>>> that
> >     >>>>>>>>>>>>>>>>>>>>>> seems
> >     >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
> >     >>>>>> something
> >     >>>>>>>> like
> >     >>>>>>>>>>>>>>>>>>>>>> branched()
> >     >>>>>>>>>>>>>>>>>>>>> or
> >     >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
> >     KIP?  It
> >     >>>>>> seems
> >     >>>>>>>>> like it
> >     >>>>>>>>>>>>>>>>>>>>>> does to
> >     >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
> >     while also
> >     >>>>>>>> allowing
> >     >>>>>>>>> you
> >     >>>>>>>>>>>>> to
> >     >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
> >     KBranchedStreams
> >     >>>>>> if
> >     >>>>>>>>> desired.
> >     >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >     >>>>>>>>>>>>>>>>>>>>>> Paul
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> >     >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >     >>>>>>>>>>>>>>>>>>>>>> wrote:
> >     >>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
> >     ks){
> >     >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
> >     >>>>>>>>>>>>>>>>>>>>>>> }
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
> >     String> ks){
> >     >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
> >     >>>>>>>>>>>>>>>>>>>>>>> }
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> ......
> >     >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> >     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
> >     this::handleFirstCase)
> >     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
> >     this::handleSecondCase)
> >     >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >     >>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> >     >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the
> KafkaStreamsBrancher
> >     >>>>> takes a
> >     >>>>>>>>> Consumer
> >     >>>>>>>>>>>>> as a
> >     >>>>>>>>>>>>>>>>>>>>>>> second
> >     >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
> >     example in
> >     >>>>>> the
> >     >>>>>>>> KIP
> >     >>>>>>>>>>>>> shows
> >     >>>>>>>>>>>>>>>>>>>>>>>> each
> >     >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
> >     >>>>>>>>> (KafkaStreams#to()
> >     >>>>>>>>>>>>>>>>>>>>>>>> in this
> >     >>>>>>>>>>>>>>>>>>>>>>>> case).
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would
> we
> >     >>>>> handle
> >     >>>>>>> the
> >     >>>>>>>>> case
> >     >>>>>>>>>>>>>>>>>>>>>>>> where the
> >     >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to
> continue
> >     >>>>>>>> processing
> >     >>>>>>>>> and
> >     >>>>>>>>>>>>> not
> >     >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the
> branched
> >     >>>>>> stream
> >     >>>>>>>>>>>>> immediately?
> >     >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
> >     we had
> >     >>>>>>>> something
> >     >>>>>>>>> like
> >     >>>>>>>>>>>>>>>>>>>>>>>> this:
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
> >     >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >     >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >     >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >     >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >     >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck
> <
> >     >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
> >     >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >     >>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >     >>>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for
> KIP-
> >     >>>>> 418.
> >     >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >     >>>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >     >>>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about
> KIP-418.
> >     >>>>> Please
> >     >>>>>>>> take
> >     >>>>>>>>> a
> >     >>>>>>>>>>>>> look
> >     >>>>>>>>>>>>>>>>> at
> >     >>>>>>>>>>>>>>>>>>>>> the
> >     >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
> >     feedback :)
> >     >>>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >     >>>>>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >     >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >     >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >     >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >     >>>>> https://github.com/apache/kafka/pull/6164
> >     >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >     >>>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >     >>>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>>>>>>>>>>>>>>>>>
> >     >>>>>>>>>
> >     >
> >
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Interesting discussion.

I am wondering, if we cannot unify the advantage of both approaches:



KStream#split() -> KBranchedStream

// branch is not easily accessible in current scope
KBranchedStream#branch(Predicate, Consumer<KStream>)
  -> KBranchedStream

// assign a name to the branch and
// return the sub-stream to the current scope later
//
// can be simple as `#branch(p, s->s, "name")`
// or also complex as `#branch(p, s->s.filter(...), "name")`
KBranchedStream#branch(Predicate, Function<KStream,KStream>, String)
  -> KBranchedStream

// default branch is not easily accessible
// return map of all named sub-stream into current scope
KBranchedStream#default(Cosumer<KStream>)
  -> Map<String,KStream>

// assign custom name to default-branch
// return map of all named sub-stream into current scope
KBranchedStream#default(Function<KStream,KStream>, String)
  -> Map<String,KStream>

// assign a default name for default
// return map of all named sub-stream into current scope
KBranchedStream#defaultBranch(Function<KStream,KStream>)
  -> Map<String,KStream>

// return map of all names sub-stream into current scope
KBranchedStream#noDefaultBranch()
  -> Map<String,KStream>



Hence, for each sub-stream, the user can pick to add a name and return
the branch "result" to the calling scope or not. The implementation can
also check at runtime that all returned names are unique. The returned
Map can be empty and it's also optional to use the Map.

To me, it seems like a good way to get best of both worlds.

Thoughts?



-Matthias




On 5/6/19 5:15 PM, John Roesler wrote:
> Ivan,
> 
> That's a very good point about the "start" operator in the dynamic case.
> I had no problem with "split()"; I was just questioning the necessity.
> Since you've provided a proof of necessity, I'm in favor of the
> "split()" start operator. Thanks!
> 
> Separately, I'm interested to see where the present discussion leads.
> I've written enough Javascript code in my life to be suspicious of
> nested closures. You have a good point about using method references (or
> indeed function literals also work). It should be validating that this
> was also the JS community's first approach to flattening the logic when
> their nested closure situation got out of hand. Unfortunately, it's
> replacing nesting with redirection, both of which disrupt code
> readability (but in different ways for different reasons). In other
> words, I agree that function references is *the* first-order solution if
> the nested code does indeed become a problem.
> 
> However, the history of JS also tells us that function references aren't
> the end of the story either, and you can see that by observing that
> there have been two follow-on eras, as they continue trying to cope with
> the consequences of living in such a callback-heavy language. First, you
> have Futures/Promises, which essentially let you convert nested code to
> method-chained code (Observables/FP is a popular variation on this).
> Most lately, you have async/await, which is an effort to apply language
> (not just API) syntax to the problem, and offer the "flattest" possible
> programming style to solve the problem (because you get back to just one
> code block per functional unit).
> 
> Stream-processing is a different domain, and Java+KStreams is nowhere
> near as callback heavy as JS, so I don't think we have to take the JS
> story for granted, but then again, I think we can derive some valuable
> lessons by looking sideways to adjacent domains. I'm just bringing this
> up to inspire further/deeper discussion. At the same time, just like JS,
> we can afford to take an iterative approach to the problem.
> 
> Separately again, I'm interested in the post-branch merge (and I'd also
> add join) problem that Paul brought up. We can clearly punt on it, by
> terminating the nested branches with sink operators. But is there a DSL
> way to do it?
> 
> Thanks again for your driving this,
> -John 
> 
> On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pgwhalen@gmail.com
> <ma...@gmail.com>> wrote:
> 
>     Ivan, I’ll definitely forfeit my point on the clumsiness of the
>     branch(predicate, consumer) solution, I don’t see any real drawbacks
>     for the dynamic case.
> 
>     IMO the one trade off to consider at this point is the scope
>     question. I don’t know if I totally agree that “we rarely need them
>     in the same scope” since merging the branches back together later
>     seems like a perfectly plausible use case that can be a lot nicer
>     when the branched streams are in the same scope. That being said,
>     for the reasons Ivan listed, I think it is overall the better
>     solution - working around the scope thing is easy enough if you need
>     to.
> 
>     > On May 2, 2019, at 7:00 PM, Ivan Ponomarev
>     <ip...@mail.ru.invalid> wrote:
>     >
>     > Hello everyone, thank you all for joining the discussion!
>     >
>     > Well, I don't think the idea of named branches, be it a
>     LinkedHashMap (no other Map will do, because order of definition
>     matters) or `branch` method  taking name and Consumer has more
>     advantages than drawbacks.
>     >
>     > In my opinion, the only real positive outcome from Michael's
>     proposal is that all the returned branches are in the same scope.
>     But 1) we rarely need them in the same scope 2) there is a
>     workaround for the scope problem, described in the KIP.
>     >
>     > 'Inlining the complex logic' is not a problem, because we can use
>     method references instead of lambdas. In real world scenarios you
>     tend to split the complex logic to methods anyway, so the code is
>     going to be clean.
>     >
>     > The drawbacks are strong. The cohesion between predicates and
>     handlers is lost. We have to define predicates in one place, and
>     handlers in another. This opens the door for bugs:
>     >
>     > - what if we forget to define a handler for a name? or a name for
>     a handler?
>     > - what if we misspell a name?
>     > - what if we copy-paste and duplicate a name?
>     >
>     > What Michael propose would have been totally OK if we had been
>     writing the API in Lua, Ruby or Python. In those languages the
>     "dynamic naming" approach would have looked most concise and
>     beautiful. But in Java we expect all the problems related to
>     identifiers to be eliminated in compile time.
>     >
>     > Do we have to invent duck-typing for the Java API?
>     >
>     > And if we do, what advantage are we supposed to get besides having
>     all the branches in the same scope? Michael, maybe I'm missing your
>     point?
>     >
>     > ---
>     >
>     > Earlier in this discussion John Roesler also proposed to do
>     without "start branching" operator, and later Paul mentioned that in
>     the case when we have to add a dynamic number of branches, the
>     current KIP is 'clumsier' compared to Michael's 'Map' solution. Let
>     me address both comments here.
>     >
>     > 1) "Start branching" operator (I think that *split* is a good name
>     for it indeed) is critical when we need to do a dynamic branching,
>     see example below.
>     >
>     > 2) No, dynamic branching in current KIP is not clumsy at all.
>     Imagine a real-world scenario when you need one branch per enum
>     value (say, RecordType). You can have something like this:
>     >
>     > /*John:if we had to start with stream.branch(...) here, it would
>     have been much messier.*/
>     > KBranchedStream branched = stream.split();
>     >
>     > /*Not clumsy at all :-)*/
>     > for (RecordType recordType : RecordType.values())
>     >             branched = branched.branch((k, v) -> v.getRecType() ==
>     recordType,
>     >                     recordType::processRecords);
>     >
>     > Regards,
>     >
>     > Ivan
>     >
>     >
>     > 02.05.2019 14:40, Matthias J. Sax пишет:
>     >> I also agree with Michael's observation about the core problem of
>     >> current `branch()` implementation.
>     >>
>     >> However, I also don't like to pass in a clumsy Map object. My
>     thinking
>     >> was more aligned with Paul's proposal to just add a name to each
>     >> `branch()` statement and return a `Map<String,KStream>`.
>     >>
>     >> It makes the code easier to read, and also make the order of
>     >> `Predicates` (that is essential) easier to grasp.
>     >>
>     >>>>>> Map<String, KStream<K, V>> branches = stream.split()
>     >>>>>>    .branch("branchOne", Predicate<K, V>)
>     >>>>>>    .branch( "branchTwo", Predicate<K, V>)
>     >>>>>>    .defaultBranch("defaultBranch");
>     >> An open question is the case for which no defaultBranch() should be
>     >> specified. Atm, `split()` and `branch()` would return
>     `BranchedKStream`
>     >> and the call to `defaultBranch()` that returns the `Map` is mandatory
>     >> (what is not the case atm). Or is this actually not a real problem,
>     >> because users can just ignore the branch returned by
>     `defaultBranch()`
>     >> in the result `Map` ?
>     >>
>     >>
>     >> About "inlining": So far, it seems to be a matter of personal
>     >> preference. I can see arguments for both, but no "killer
>     argument" yet
>     >> that clearly make the case for one or the other.
>     >>
>     >>
>     >> -Matthias
>     >>
>     >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>     >>> Perhaps inlining is the wrong terminology. It doesn’t require
>     that a lambda with the full downstream topology be defined inline -
>     it can be a method reference as with Ivan’s original suggestion. 
>     The advantage of putting the predicate and its downstream logic
>     (Consumer) together in branch() is that they are required to be near
>     to each other.
>     >>>
>     >>> Ultimately the downstream code has to live somewhere, and deep
>     branch trees will be hard to read regardless.
>     >>>
>     >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis
>     <michael.drogalis@confluent.io
>     <ma...@confluent.io>> wrote:
>     >>>>
>     >>>> I'm less enthusiastic about inlining the branch logic with its
>     downstream
>     >>>> functionality. Programs that have deep branch trees will
>     quickly become
>     >>>> harder to read as a single unit.
>     >>>>
>     >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen
>     <pgwhalen@gmail.com <ma...@gmail.com>> wrote:
>     >>>>>
>     >>>>> Also +1 on the issues/goals as Michael outlined them, I think
>     that sets a
>     >>>>> great framework for the discussion.
>     >>>>>
>     >>>>> Regarding the SortedMap solution, my understanding is that the
>     current
>     >>>>> proposal in the KIP is what is in my PR which (pending naming
>     decisions) is
>     >>>>> roughly this:
>     >>>>>
>     >>>>> stream.split()
>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>     >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>     >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>     >>>>>
>     >>>>> Obviously some ordering is necessary, since branching as a
>     construct
>     >>>>> doesn't work without it, but this solution seems like it
>     provides as much
>     >>>>> associativity as the SortedMap solution, because each branch()
>     call
>     >>>>> directly associates the "conditional" with the "code block." 
>     The value it
>     >>>>> provides over the KIP solution is the accessing of streams in
>     the same
>     >>>>> scope.
>     >>>>>
>     >>>>> The KIP solution is less "dynamic" than the SortedMap solution
>     in the sense
>     >>>>> that it is slightly clumsier to add a dynamic number of
>     branches, but it is
>     >>>>> certainly possible.  It seems to me like the API should favor
>     the "static"
>     >>>>> case anyway, and should make it simple and readable to
>     fluently declare and
>     >>>>> access your branches in-line.  It also makes it impossible to
>     ignore a
>     >>>>> branch, and it is possible to build an (almost) identical
>     SortedMap
>     >>>>> solution on top of it.
>     >>>>>
>     >>>>> I could also see a middle ground where instead of a raw
>     SortedMap being
>     >>>>> taken in, branch() takes a name and not a Consumer.  Something
>     like this:
>     >>>>>
>     >>>>> Map<String, KStream<K, V>> branches = stream.split()
>     >>>>>    .branch("branchOne", Predicate<K, V>)
>     >>>>>    .branch( "branchTwo", Predicate<K, V>)
>     >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>     >>>>>
>     >>>>> Pros for that solution:
>     >>>>> - accessing branched KStreams in same scope
>     >>>>> - no double brace initialization, hopefully slightly more
>     readable than
>     >>>>> SortedMap
>     >>>>>
>     >>>>> Cons
>     >>>>> - downstream branch logic cannot be specified inline which
>     makes it harder
>     >>>>> to read top to bottom (like existing API and SortedMap, but
>     unlike the KIP)
>     >>>>> - you can forget to "handle" one of the branched streams (like
>     existing
>     >>>>> API and SortedMap, but unlike the KIP)
>     >>>>>
>     >>>>> (KBranchedStreams could even work *both* ways but perhaps
>     that's overdoing
>     >>>>> it).
>     >>>>>
>     >>>>> Overall I'm curious how important it is to be able to easily
>     access the
>     >>>>> branched KStream in the same scope as the original.  It's
>     possible that it
>     >>>>> doesn't need to be handled directly by the API, but instead
>     left up to the
>     >>>>> user.  I'm sort of in the middle on it.
>     >>>>>
>     >>>>> Paul
>     >>>>>
>     >>>>>
>     >>>>>
>     >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman
>     <sophie@confluent.io <ma...@confluent.io>>
>     >>>>> wrote:
>     >>>>>
>     >>>>>> I'd like to +1 what Michael said about the issues with the
>     existing
>     >>>>> branch
>     >>>>>> method, I agree with what he's outlined and I think we should
>     proceed by
>     >>>>>> trying to alleviate these problems. Specifically it seems
>     important to be
>     >>>>>> able to cleanly access the individual branches (eg by mapping
>     >>>>>> name->stream), which I thought was the original intention of
>     this KIP.
>     >>>>>>
>     >>>>>> That said, I don't think we should so easily give in to the
>     double brace
>     >>>>>> anti-pattern or force ours users into it if at all possible to
>     >>>>> avoid...just
>     >>>>>> my two cents.
>     >>>>>>
>     >>>>>> Cheers,
>     >>>>>> Sophie
>     >>>>>>
>     >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>     >>>>>> michael.drogalis@confluent.io
>     <ma...@confluent.io>> wrote:
>     >>>>>>
>     >>>>>>> I’d like to propose a different way of thinking about this.
>     To me,
>     >>>>> there
>     >>>>>>> are three problems with the existing branch signature:
>     >>>>>>>
>     >>>>>>> 1. If you use it the way most people do, Java raises unsafe type
>     >>>>>> warnings.
>     >>>>>>> 2. The way in which you use the stream branches is
>     positionally coupled
>     >>>>>> to
>     >>>>>>> the ordering of the conditionals.
>     >>>>>>> 3. It is brittle to extend existing branch calls with
>     additional code
>     >>>>>>> paths.
>     >>>>>>>
>     >>>>>>> Using associative constructs instead of relying on ordered
>     constructs
>     >>>>>> would
>     >>>>>>> be a stronger approach. Consider a signature that instead
>     looks like
>     >>>>>> this:
>     >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
>     Predicate<?
>     >>>>>>> super K,? super V>>);
>     >>>>>>>
>     >>>>>>> Branches are given names in a map, and as a result, the API
>     returns a
>     >>>>>>> mapping of names to streams. The ordering of the conditionals is
>     >>>>>> maintained
>     >>>>>>> because it’s a sorted map. Insert order determines the order of
>     >>>>>> evaluation.
>     >>>>>>> This solves problem 1 because there are no more varargs. It
>     solves
>     >>>>>> problem
>     >>>>>>> 2 because you no longer lean on ordering to access the
>     branch you’re
>     >>>>>>> interested in. It solves problem 3 because you can introduce
>     another
>     >>>>>>> conditional by simply attaching another name to the
>     structure, rather
>     >>>>>> than
>     >>>>>>> messing with the existing indices.
>     >>>>>>>
>     >>>>>>> One of the drawbacks is that creating the map inline is
>     historically
>     >>>>>>> awkward in Java. I know it’s an anti-pattern to use
>     voluminously, but
>     >>>>>>> double brace initialization would clean up the aesthetics.
>     >>>>>>>
>     >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler
>     <john@confluent.io <ma...@confluent.io>>
>     >>>>> wrote:
>     >>>>>>>> Hi Ivan,
>     >>>>>>>>
>     >>>>>>>> Thanks for the update.
>     >>>>>>>>
>     >>>>>>>> FWIW, I agree with Matthias that the current "start branching"
>     >>>>> operator
>     >>>>>>> is
>     >>>>>>>> confusing when named the same way as the actual branches.
>     "Split"
>     >>>>> seems
>     >>>>>>>> like a good name. Alternatively, we can do without a "start
>     >>>>> branching"
>     >>>>>>>> operator at all, and just do:
>     >>>>>>>>
>     >>>>>>>> stream
>     >>>>>>>>      .branch(Predicate)
>     >>>>>>>>      .branch(Predicate)
>     >>>>>>>>      .defaultBranch();
>     >>>>>>>>
>     >>>>>>>> Tentatively, I think that this branching operation should be
>     >>>>> terminal.
>     >>>>>>> That
>     >>>>>>>> way, we don't create ambiguity about how to use it. That
>     is, `branch`
>     >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is
>     `void`, to
>     >>>>>>>> enforce that it comes last, and that there is only one
>     definition of
>     >>>>>> the
>     >>>>>>>> default branch. Potentially, we should log a warning if
>     there's no
>     >>>>>>> default,
>     >>>>>>>> and additionally log a warning (or throw an exception) if a
>     record
>     >>>>>> falls
>     >>>>>>>> though with no default.
>     >>>>>>>>
>     >>>>>>>> Thoughts?
>     >>>>>>>>
>     >>>>>>>> Thanks,
>     >>>>>>>> -John
>     >>>>>>>>
>     >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>     >>>>> matthias@confluent.io <ma...@confluent.io>
>     >>>>>>>> wrote:
>     >>>>>>>>
>     >>>>>>>>> Thanks for updating the KIP and your answers.
>     >>>>>>>>>
>     >>>>>>>>>
>     >>>>>>>>>> this is to make the name similar to String#split
>     >>>>>>>>>>> that also returns an array, right?
>     >>>>>>>>> The intend was to avoid name duplication. The return type
>     should
>     >>>>>> _not_
>     >>>>>>>>> be an array.
>     >>>>>>>>>
>     >>>>>>>>> The current proposal is
>     >>>>>>>>>
>     >>>>>>>>> stream.branch()
>     >>>>>>>>>      .branch(Predicate)
>     >>>>>>>>>      .branch(Predicate)
>     >>>>>>>>>      .defaultBranch();
>     >>>>>>>>>
>     >>>>>>>>> IMHO, this reads a little odd, because the first
>     `branch()` does
>     >>>>> not
>     >>>>>>>>> take any parameters and has different semantics than the later
>     >>>>>>>>> `branch()` calls. Note, that from the code snippet above, it's
>     >>>>> hidden
>     >>>>>>>>> that the first call is `KStream#branch()` while the others are
>     >>>>>>>>> `KBranchedStream#branch()` what makes reading the code harder.
>     >>>>>>>>>
>     >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`,
>     I though
>     >>>>>> it
>     >>>>>>>>> might be better to also rename `KStream#branch()` to avoid the
>     >>>>> naming
>     >>>>>>>>> overlap that seems to be confusing. The following reads much
>     >>>>> cleaner
>     >>>>>> to
>     >>>>>>>> me:
>     >>>>>>>>> stream.split()
>     >>>>>>>>>      .branch(Predicate)
>     >>>>>>>>>      .branch(Predicate)
>     >>>>>>>>>      .defaultBranch();
>     >>>>>>>>>
>     >>>>>>>>> Maybe there is a better alternative to `split()` though to
>     avoid
>     >>>>> the
>     >>>>>>>>> naming overlap.
>     >>>>>>>>>
>     >>>>>>>>>
>     >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately we
>     >>>>> cannot
>     >>>>>>> have
>     >>>>>>>>> a method with such name :-)
>     >>>>>>>>>
>     >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up
>     with a
>     >>>>> short
>     >>>>>>>> name?
>     >>>>>>>>>
>     >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP
>     with all
>     >>>>> it's
>     >>>>>>>>> methods? It will be part of public API and should be
>     contained in
>     >>>>> the
>     >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>     >>>>>>>>> `defaultBranch()` is.
>     >>>>>>>>>
>     >>>>>>>>>
>     >>>>>>>>> You did not comment on the idea to add a
>     `KBranchedStream#get(int
>     >>>>>>> index)
>     >>>>>>>>> -> KStream` method to get the individually
>     branched-KStreams. Would
>     >>>>>> be
>     >>>>>>>>> nice to get your feedback about it. It seems you suggest
>     that users
>     >>>>>>>>> would need to write custom utility code otherwise, to
>     access them.
>     >>>>> We
>     >>>>>>>>> should discuss the pros and cons of both approaches. It feels
>     >>>>>>>>> "incomplete" to me atm, if the API has no built-in support
>     to get
>     >>>>> the
>     >>>>>>>>> branched-KStreams directly.
>     >>>>>>>>>
>     >>>>>>>>>
>     >>>>>>>>>
>     >>>>>>>>> -Matthias
>     >>>>>>>>>
>     >>>>>>>>>
>     >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>     >>>>>>>>>> Hi all!
>     >>>>>>>>>>
>     >>>>>>>>>> I have updated the KIP-418 according to the new vision.
>     >>>>>>>>>>
>     >>>>>>>>>> Matthias, thanks for your comment!
>     >>>>>>>>>>
>     >>>>>>>>>>> Renaming KStream#branch() -> #split()
>     >>>>>>>>>> I can see your point: this is to make the name similar to
>     >>>>>>> String#split
>     >>>>>>>>>> that also returns an array, right? But is it worth the
>     loss of
>     >>>>>>>> backwards
>     >>>>>>>>>> compatibility? We can have overloaded branch() as well
>     without
>     >>>>>>>> affecting
>     >>>>>>>>>> the existing code. Maybe the old array-based `branch` method
>     >>>>> should
>     >>>>>>> be
>     >>>>>>>>>> deprecated, but this is a subject for discussion.
>     >>>>>>>>>>
>     >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>     >>>>> BranchingKStream#branch(),
>     >>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>     >>>>>>>>>>
>     >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default' is,
>     >>>>>>> however, a
>     >>>>>>>>>> reserved word, so unfortunately we cannot have a method
>     with such
>     >>>>>>> name
>     >>>>>>>>> :-)
>     >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument, but I
>     >>>>> think
>     >>>>>>> that
>     >>>>>>>>>> is not required?
>     >>>>>>>>>>
>     >>>>>>>>>> Absolutely! I think that was just copy-paste error or
>     something.
>     >>>>>>>>>>
>     >>>>>>>>>> Dear colleagues,
>     >>>>>>>>>>
>     >>>>>>>>>> please revise the new version of the KIP and Paul's PR
>     >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>     >>>>>>>>>>
>     >>>>>>>>>> Any new suggestions/objections?
>     >>>>>>>>>>
>     >>>>>>>>>> Regards,
>     >>>>>>>>>>
>     >>>>>>>>>> Ivan
>     >>>>>>>>>>
>     >>>>>>>>>>
>     >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>     >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems that
>     >>>>>>> everybody
>     >>>>>>>>>>> agrees that the current branch() method using arrays is not
>     >>>>>> optimal.
>     >>>>>>>>>>> I had a quick look into the PR and I like the overall
>     proposal.
>     >>>>>>> There
>     >>>>>>>>>>> are some minor things we need to consider. I would
>     recommend the
>     >>>>>>>>>>> following renaming:
>     >>>>>>>>>>>
>     >>>>>>>>>>> KStream#branch() -> #split()
>     >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>     >>>>>>>>>>> KBranchedStream#defaultBranch() ->
>     BranchingKStream#default()
>     >>>>>>>>>>>
>     >>>>>>>>>>> It's just a suggestion to get slightly shorter method names.
>     >>>>>>>>>>>
>     >>>>>>>>>>> In the current PR, defaultBranch() does take an
>     `Predicate` as
>     >>>>>>>> argument,
>     >>>>>>>>>>> but I think that is not required?
>     >>>>>>>>>>>
>     >>>>>>>>>>> Also, we should consider KIP-307, that was recently
>     accepted and
>     >>>>>> is
>     >>>>>>>>>>> currently implemented:
>     >>>>>>>>>>>
>     >>>>>
>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>     >>>>>>>>>>> Ie, we should add overloads that accepted a `Named`
>     parameter.
>     >>>>>>>>>>>
>     >>>>>>>>>>>
>     >>>>>>>>>>> For the issue that the created `KStream` object are in
>     different
>     >>>>>>>> scopes:
>     >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int
>     index)` method
>     >>>>>>> that
>     >>>>>>>>>>> returns the corresponding "branched" result `KStream`
>     object?
>     >>>>>> Maybe,
>     >>>>>>>> the
>     >>>>>>>>>>> second argument of `addBranch()` should not be a
>     >>>>>> `Consumer<KStream>`
>     >>>>>>>> but
>     >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return
>     whatever
>     >>>>>> the
>     >>>>>>>>>>> `Function` returns?
>     >>>>>>>>>>>
>     >>>>>>>>>>>
>     >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
>     current
>     >>>>>>>>>>> proposal. That makes it easier to review.
>     >>>>>>>>>>>
>     >>>>>>>>>>>
>     >>>>>>>>>>> -Matthias
>     >>>>>>>>>>>
>     >>>>>>>>>>>
>     >>>>>>>>>>>
>     >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>     >>>>>>>>>>>> Ivan,
>     >>>>>>>>>>>>
>     >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it
>     makes sense
>     >>>>>> for
>     >>>>>>>> you
>     >>>>>>>>> to
>     >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously
>     we'll
>     >>>>> need
>     >>>>>>>> some
>     >>>>>>>>>>>> buy-in from committers that have actual binding votes on
>     >>>>> whether
>     >>>>>>> the
>     >>>>>>>>> KIP
>     >>>>>>>>>>>> could be adopted.  It would be great to hear if they
>     think this
>     >>>>>> is
>     >>>>>>> a
>     >>>>>>>>> good
>     >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by
>     starting a
>     >>>>>>> vote,
>     >>>>>>>>> or if
>     >>>>>>>>>>>> there is generally some indication of interest beforehand.
>     >>>>>>>>>>>>
>     >>>>>>>>>>>> That being said, I'll continue the discussion a bit:
>     assuming
>     >>>>> we
>     >>>>>> do
>     >>>>>>>>> move
>     >>>>>>>>>>>> forward the solution of "stream.branch() returns
>     >>>>>> KBranchedStream",
>     >>>>>>> do
>     >>>>>>>>> we
>     >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I would
>     >>>>> favor
>     >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs that
>     >>>>>>> accomplish
>     >>>>>>>>> the
>     >>>>>>>>>>>> same thing is confusing, especially when they're fairly
>     similar
>     >>>>>>>>> anyway.  We
>     >>>>>>>>>>>> just need to be sure we're not making something
>     >>>>>>> impossible/difficult
>     >>>>>>>>> that
>     >>>>>>>>>>>> is currently possible/easy.
>     >>>>>>>>>>>>
>     >>>>>>>>>>>> Regarding my PR - I think the general structure would work,
>     >>>>> it's
>     >>>>>>>> just a
>     >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>     >>>>>>> particular,
>     >>>>>>>>>>>> passing in the "predicates" and "children" lists which get
>     >>>>>> modified
>     >>>>>>>> in
>     >>>>>>>>>>>> KBranchedStream but read from all the way
>     KStreamLazyBranch is
>     >>>>> a
>     >>>>>>> bit
>     >>>>>>>>>>>> complicated to follow.
>     >>>>>>>>>>>>
>     >>>>>>>>>>>> Thanks,
>     >>>>>>>>>>>> Paul
>     >>>>>>>>>>>>
>     >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>     >>>>>>>>> wrote:
>     >>>>>>>>>>>>> Hi Paul!
>     >>>>>>>>>>>>>
>     >>>>>>>>>>>>> I read your code carefully and now I am fully
>     convinced: your
>     >>>>>>>> proposal
>     >>>>>>>>>>>>> looks better and should work. We just have to document the
>     >>>>>> crucial
>     >>>>>>>>> fact
>     >>>>>>>>>>>>> that KStream consumers are invoked as they're added.
>     And then
>     >>>>>> it's
>     >>>>>>>> all
>     >>>>>>>>>>>>> going to be very nice.
>     >>>>>>>>>>>>>
>     >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and
>     resume the
>     >>>>>>>>>>>>> discussion here, right?
>     >>>>>>>>>>>>>
>     >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>     >>>>> starting
>     >>>>>>>> point
>     >>>>>>>>> if
>     >>>>>>>>>>>>> we go in this direction'? To me it looks like a good
>     starting
>     >>>>>>> point.
>     >>>>>>>>> But
>     >>>>>>>>>>>>> as a novice in this project I might miss some important
>     >>>>> details.
>     >>>>>>>>>>>>> Regards,
>     >>>>>>>>>>>>>
>     >>>>>>>>>>>>> Ivan
>     >>>>>>>>>>>>>
>     >>>>>>>>>>>>>
>     >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>     >>>>>>>>>>>>>> Ivan,
>     >>>>>>>>>>>>>>
>     >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>     >>>>> stream.branch()
>     >>>>>>>>> solution
>     >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>     >>>>> invoked
>     >>>>>> as
>     >>>>>>>>> they’re
>     >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user
>     still
>     >>>>>> ought
>     >>>>>>> to
>     >>>>>>>>> be
>     >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and
>     depend on
>     >>>>> the
>     >>>>>>>>> branched
>     >>>>>>>>>>>>> streams having been set.
>     >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to
>     access
>     >>>>> the
>     >>>>>>>>> branched
>     >>>>>>>>>>>>> streams in the same scope as the original stream (that
>     is, not
>     >>>>>>>> inside
>     >>>>>>>>> the
>     >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>     >>>>> solutions.
>     >>>>>> It
>     >>>>>>>>> can be
>     >>>>>>>>>>>>> worked around though.
>     >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m
>     excited
>     >>>>> to
>     >>>>>>>> hear
>     >>>>>>>>>>>>> your thoughts!]
>     >>>>>>>>>>>>>> Paul
>     >>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>     >>>>>> iponomarev@mail.ru <ma...@mail.ru>
>     >>>>>>>>> wrote:
>     >>>>>>>>>>>>>>> Hi Paul!
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>     >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at first
>     >>>>> glance,
>     >>>>>>> but
>     >>>>>>>>> ---
>     >>>>>>>>>>>>>>>> the newly branched streams are not available in the
>     same
>     >>>>>> scope
>     >>>>>>> as
>     >>>>>>>>> each
>     >>>>>>>>>>>>> other.  That is, if we wanted to merge them back together
>     >>>>> again
>     >>>>>> I
>     >>>>>>>>> don't see
>     >>>>>>>>>>>>> a way to do that.
>     >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was
>     just
>     >>>>>> going
>     >>>>>>> to
>     >>>>>>>>>>>>> write in details about this issue.
>     >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say
>     we need
>     >>>>> to
>     >>>>>>>>> identify
>     >>>>>>>>>>>>> customers who have bought coffee and made a purchase
>     in the
>     >>>>>>>>> electronics
>     >>>>>>>>>>>>> store to give them coupons.
>     >>>>>>>>>>>>>>> This is the code I usually write under these
>     circumstances
>     >>>>>> using
>     >>>>>>>> my
>     >>>>>>>>>>>>> 'brancher' class:
>     >>>>>>>>>>>>>>> @Setter
>     >>>>>>>>>>>>>>> class CouponIssuer{
>     >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>     >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>   KStream<...> coupons(){
>     >>>>>>>>>>>>>>>       return
>     >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>     >>>>>>>>>>>>>>>       /*In the real world the code here can be
>     complex, so
>     >>>>>>>>> creation of
>     >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in
>     order to
>     >>>>>>>> separate
>     >>>>>>>>>>>>> classes' responsibilities.*/
>     >>>>>>>>>>>>>>>  }
>     >>>>>>>>>>>>>>> }
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>     >>>>>>>>>>>>>>>     .branch(predicate1, couponIssuer::setCoffePurchases)
>     >>>>>>>>>>>>>>>     .branch(predicate2,
>     >>>>>> couponIssuer::setElectronicsPurchases)
>     >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
>     everything
>     >>>>>>>> later,
>     >>>>>>>>>>>>> without the terminal operation!!!*/
>     >>>>>>>>>>>>>>> couponIssuer.coupons()...
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> Does this make sense?  In order to properly
>     initialize the
>     >>>>>>>>> CouponIssuer
>     >>>>>>>>>>>>> we need the terminal operation to be called before
>     >>>>>>>>> streamsBuilder.build()
>     >>>>>>>>>>>>> is called.
>     >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
>     essentially
>     >>>>>> the
>     >>>>>>>>> next
>     >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts
>     based on
>     >>>>> my
>     >>>>>>>>> experience,
>     >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>     >>>>>>>>>>>>>>> Regards,
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> Ivan
>     >>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>     >>>>>>>>>>>>>>>> Ivan,
>     >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a
>     fluent
>     >>>>> API
>     >>>>>>>> based
>     >>>>>>>>>>>>> off of
>     >>>>>>>>>>>>>>>> KStream here
>     (https://github.com/apache/kafka/pull/6512),
>     >>>>>> and
>     >>>>>>> I
>     >>>>>>>>> think
>     >>>>>>>>>>>>> I
>     >>>>>>>>>>>>>>>> succeeded at removing both cons.
>     >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>     >>>>>>> compatibility
>     >>>>>>>>>>>>> issues,
>     >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware
>     that Java
>     >>>>> is
>     >>>>>>>> smart
>     >>>>>>>>>>>>> enough to
>     >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...)
>     returning one
>     >>>>>>> thing
>     >>>>>>>>> and
>     >>>>>>>>>>>>> branch()
>     >>>>>>>>>>>>>>>>    with no arguments returning another thing.
>     >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually
>     need
>     >>>>> it.
>     >>>>>>> We
>     >>>>>>>>> can
>     >>>>>>>>>>>>> just
>     >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who
>     shares
>     >>>>>> its
>     >>>>>>>>> state
>     >>>>>>>>>>>>> with the
>     >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the
>     branching.
>     >>>>>>> It's
>     >>>>>>>>> not
>     >>>>>>>>>>>>> terribly
>     >>>>>>>>>>>>>>>>    pretty in its current form, but I think it
>     demonstrates
>     >>>>>> its
>     >>>>>>>>>>>>> feasibility.
>     >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should be
>     >>>>> final
>     >>>>>> or
>     >>>>>>>>> even a
>     >>>>>>>>>>>>>>>> starting point if we go in this direction, I just
>     wanted to
>     >>>>>> see
>     >>>>>>>> how
>     >>>>>>>>>>>>>>>> challenging it would be to get the API working.
>     >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing
>     solution
>     >>>>>>> could
>     >>>>>>>> be
>     >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>     >>>>> suggested
>     >>>>>>>> was a
>     >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
>     streams
>     >>>>>> are
>     >>>>>>>> not
>     >>>>>>>>>>>>>>>> available in the same scope as each other.  That
>     is, if we
>     >>>>>>> wanted
>     >>>>>>>>> to
>     >>>>>>>>>>>>> merge
>     >>>>>>>>>>>>>>>> them back together again I don't see a way to do
>     that.  The
>     >>>>>> KIP
>     >>>>>>>>>>>>> proposal
>     >>>>>>>>>>>>>>>> has the same issue, though - all this means is that for
>     >>>>>> either
>     >>>>>>>>>>>>> solution,
>     >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the
>     table.
>     >>>>>>>>>>>>>>>> Thanks,
>     >>>>>>>>>>>>>>>> Paul
>     >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>>
>     >>>>>>>>>>>>> wrote:
>     >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to this
>     >>>>>> point.
>     >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that
>     branch API
>     >>>>>>> needs
>     >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> There are two potential ways to do it:
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>     >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf
>     returns
>     >>>>>> its
>     >>>>>>>>> argument
>     >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't
>     >>>>> make
>     >>>>>>>> sense
>     >>>>>>>>>>>>> until
>     >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher
>     instance
>     >>>>>>>>> contrasts the
>     >>>>>>>>>>>>>>>>> fluency of other KStream methods.
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> stream
>     >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>     >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>     >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>     >>>>>>>>> defaultBranch(..)
>     >>>>>>>>>>>>> and
>     >>>>>>>>>>>>>>>>> noDefault() return void
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface is
>     >>>>>> defined.
>     >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>     >>>>>>>> (defaultBranch(ks->)
>     >>>>>>>>> and
>     >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to
>     miss the
>     >>>>>> fact
>     >>>>>>>>> that one
>     >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
>     methods
>     >>>>>> are
>     >>>>>>>> not
>     >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do better?
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> Regards,
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> Ivan
>     >>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>     >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>     >>>>>>>>>>>>>>>>>>> Paul,
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>     >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be
>     implemented the
>     >>>>>>> easy
>     >>>>>>>>> way.
>     >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that assumes
>     >>>>> nothing
>     >>>>>>>> will
>     >>>>>>>>>>>>> reach
>     >>>>>>>>>>>>>>>>>>>> the default branch,
>     >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only option
>     >>>>>> besides
>     >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we
>     want to
>     >>>>>> just
>     >>>>>>>>> silently
>     >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any
>     predicate. 2)
>     >>>>>>> Throwing
>     >>>>>>>>> an
>     >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing
>     looks
>     >>>>>> like a
>     >>>>>>>> bad
>     >>>>>>>>>>>>> idea.
>     >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to
>     emit a
>     >>>>>>>> special
>     >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly where
>     >>>>>>> `default`
>     >>>>>>>>> can
>     >>>>>>>>>>>>> be
>     >>>>>>>>>>>>>>>>>>> used.
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>     >>>>> InternalTopologyBuilder
>     >>>>>>> to
>     >>>>>>>>> track
>     >>>>>>>>>>>>>>>>>>>> dangling
>     >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>     a clear
>     >>>>>>> error
>     >>>>>>>>>>>>> before it
>     >>>>>>>>>>>>>>>>>>> becomes an issue.
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>     >>>>> compiled
>     >>>>>>> and
>     >>>>>>>>> run?
>     >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't
>     compile if
>     >>>>> used
>     >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a
>     method chain
>     >>>>>>>> starting
>     >>>>>>>>>>>>> from
>     >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference
>     between
>     >>>>>>>> runtime
>     >>>>>>>>> and
>     >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>     >>>>> instantly
>     >>>>>> on
>     >>>>>>>>> unit
>     >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a
>     compilation
>     >>>>>>>> failure.
>     >>>>>>>>>>>>>>>>>>> Regards,
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>> Ivan
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>     >>>>>>>>>>>>>>>>>>>> Ivan,
>     >>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
>     required.
>     >>>>>>> But
>     >>>>>>>> is
>     >>>>>>>>>>>>> that
>     >>>>>>>>>>>>>>>>>>>> really
>     >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>     >>>>>> defaultBranch
>     >>>>>>>>> they
>     >>>>>>>>>>>>> can
>     >>>>>>>>>>>>>>>>>>>> call
>     >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?)
>     just as
>     >>>>>>>>> easily.  In
>     >>>>>>>>>>>>>>>>>>>> fact I
>     >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API - a
>     >>>>> user
>     >>>>>>>> could
>     >>>>>>>>>>>>> specify
>     >>>>>>>>>>>>>>>>> a
>     >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach the
>     >>>>>> default
>     >>>>>>>>> branch,
>     >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.  That
>     >>>>> seems
>     >>>>>>> like
>     >>>>>>>>> an
>     >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API,
>     which allows
>     >>>>>> for
>     >>>>>>>> the
>     >>>>>>>>>>>>> more
>     >>>>>>>>>>>>>>>>>>>> subtle
>     >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting dropped.
>     >>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has
>     to be
>     >>>>>> well
>     >>>>>>>>>>>>>>>>>>>> documented, but
>     >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>     >>>>> InternalTopologyBuilder
>     >>>>>>> to
>     >>>>>>>>> track
>     >>>>>>>>>>>>>>>>>>>> dangling
>     >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise
>     a clear
>     >>>>>>> error
>     >>>>>>>>>>>>> before it
>     >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is a
>     >>>>> "build
>     >>>>>>>> step"
>     >>>>>>>>>>>>> where
>     >>>>>>>>>>>>>>>>> the
>     >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>     >>>>>> StreamsBuilder.build()
>     >>>>>>> is
>     >>>>>>>>>>>>> called.
>     >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I agree
>     >>>>> that
>     >>>>>>> it's
>     >>>>>>>>>>>>>>>>>>>> critical to
>     >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
>     stream.
>     >>>>>>> With
>     >>>>>>>>> the
>     >>>>>>>>>>>>>>>>> fluent
>     >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all other
>     >>>>>>> operations
>     >>>>>>>>> do -
>     >>>>>>>>>>>>> if
>     >>>>>>>>>>>>>>>>> you
>     >>>>>>>>>>>>>>>>>>>> want to process off the original KStream multiple
>     >>>>> times,
>     >>>>>>> you
>     >>>>>>>>> just
>     >>>>>>>>>>>>>>>>>>>> need the
>     >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
>     operations
>     >>>>>> on
>     >>>>>>> it
>     >>>>>>>>> as
>     >>>>>>>>>>>>> you
>     >>>>>>>>>>>>>>>>>>>> desire.
>     >>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>> Thoughts?
>     >>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>> Best,
>     >>>>>>>>>>>>>>>>>>>> Paul
>     >>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>     >>>>>>>>> iponomarev@mail.ru <ma...@mail.ru>
>     >>>>>>>>>>>>>>>>>>>> wrote:
>     >>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>> Hello Paul,
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not
>     always need
>     >>>>>> the
>     >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal operation we
>     >>>>> don't
>     >>>>>>>> know
>     >>>>>>>>>>>>> when to
>     >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument,
>     so we
>     >>>>> can
>     >>>>>> do
>     >>>>>>>>>>>>> something
>     >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of special
>     >>>>> object
>     >>>>>>>>>>>>> construction
>     >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods. But
>     >>>>> here
>     >>>>>> we
>     >>>>>>>>> have a
>     >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the
>     flow,
>     >>>>> so
>     >>>>>> I
>     >>>>>>>>> think
>     >>>>>>>>>>>>> this
>     >>>>>>>>>>>>>>>>> is
>     >>>>>>>>>>>>>>>>>>>>> still idiomatic.
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>> Regards,
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>> Ivan
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>     >>>>>>>>>>>>>>>>>>>>>> Ivan,
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this
>     API, but I
>     >>>>>> find
>     >>>>>>>> the
>     >>>>>>>>>>>>>>>>>>>>>> onTopOff()
>     >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it
>     contrasts the
>     >>>>>>> fluency
>     >>>>>>>>> of
>     >>>>>>>>>>>>> other
>     >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to
>     just call
>     >>>>> a
>     >>>>>>>>> method on
>     >>>>>>>>>>>>> the
>     >>>>>>>>>>>>>>>>>>>>> stream
>     >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch
>     cases
>     >>>>> are
>     >>>>>>>>> defined
>     >>>>>>>>>>>>>>>>>>>>>> fluently.
>     >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase)
>     is very
>     >>>>>> nice
>     >>>>>>>>> and the
>     >>>>>>>>>>>>>>>>>>>>>> right
>     >>>>>>>>>>>>>>>>>>>>> way
>     >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around
>     how we
>     >>>>>>> specify
>     >>>>>>>>> the
>     >>>>>>>>>>>>> source
>     >>>>>>>>>>>>>>>>>>>>>> stream.
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>> Like:
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>> stream.branch()
>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1, this::handle1)
>     >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2, this::handle2)
>     >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>     >>>>>>>> KStreamBrancher
>     >>>>>>>>> or
>     >>>>>>>>>>>>>>>>>>>>> something,
>     >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and
>     terminated by
>     >>>>>>>>>>>>> defaultBranch()
>     >>>>>>>>>>>>>>>>>>>>>> (which
>     >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously
>     incompatible with
>     >>>>> the
>     >>>>>>>>> current
>     >>>>>>>>>>>>>>>>>>>>>> API, so
>     >>>>>>>>>>>>>>>>>>>>> the
>     >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a
>     different
>     >>>>>> name,
>     >>>>>>>> but
>     >>>>>>>>> that
>     >>>>>>>>>>>>>>>>>>>>>> seems
>     >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>     >>>>>> something
>     >>>>>>>> like
>     >>>>>>>>>>>>>>>>>>>>>> branched()
>     >>>>>>>>>>>>>>>>>>>>> or
>     >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your
>     KIP?  It
>     >>>>>> seems
>     >>>>>>>>> like it
>     >>>>>>>>>>>>>>>>>>>>>> does to
>     >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching
>     while also
>     >>>>>>>> allowing
>     >>>>>>>>> you
>     >>>>>>>>>>>>> to
>     >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
>     KBranchedStreams
>     >>>>>> if
>     >>>>>>>>> desired.
>     >>>>>>>>>>>>>>>>>>>>>> Thanks,
>     >>>>>>>>>>>>>>>>>>>>>> Paul
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>     >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>     >>>>>>>>>>>>>>>>>>>>>> wrote:
>     >>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String>
>     ks){
>     >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>     >>>>>>>>>>>>>>>>>>>>>>> }
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String,
>     String> ks){
>     >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>     >>>>>>>>>>>>>>>>>>>>>>> }
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> ......
>     >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1,
>     this::handleFirstCase)
>     >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2,
>     this::handleSecondCase)
>     >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> Regards,
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> Ivan
>     >>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>     >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher
>     >>>>> takes a
>     >>>>>>>>> Consumer
>     >>>>>>>>>>>>> as a
>     >>>>>>>>>>>>>>>>>>>>>>> second
>     >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the
>     example in
>     >>>>>> the
>     >>>>>>>> KIP
>     >>>>>>>>>>>>> shows
>     >>>>>>>>>>>>>>>>>>>>>>>> each
>     >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>     >>>>>>>>> (KafkaStreams#to()
>     >>>>>>>>>>>>>>>>>>>>>>>> in this
>     >>>>>>>>>>>>>>>>>>>>>>>> case).
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would we
>     >>>>> handle
>     >>>>>>> the
>     >>>>>>>>> case
>     >>>>>>>>>>>>>>>>>>>>>>>> where the
>     >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to continue
>     >>>>>>>> processing
>     >>>>>>>>> and
>     >>>>>>>>>>>>> not
>     >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the branched
>     >>>>>> stream
>     >>>>>>>>>>>>> immediately?
>     >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if
>     we had
>     >>>>>>>> something
>     >>>>>>>>> like
>     >>>>>>>>>>>>>>>>>>>>>>>> this:
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>     >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>     >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>     >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>     >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>     >>>>>>>>>>>>>>>>>>>>>>>> Bill
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
>     >>>>>>>>> bbejeck@gmail.com <ma...@gmail.com>
>     >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>     >>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>> All,
>     >>>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP-
>     >>>>> 418.
>     >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>     >>>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>     >>>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418.
>     >>>>> Please
>     >>>>>>>> take
>     >>>>>>>>> a
>     >>>>>>>>>>>>> look
>     >>>>>>>>>>>>>>>>> at
>     >>>>>>>>>>>>>>>>>>>>> the
>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any
>     feedback :)
>     >>>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>     >>>>>
>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>     >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>     >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>     >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>     >>>>> https://github.com/apache/kafka/pull/6164
>     >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>     >>>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>     >>>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>>>>>>>>>>>>>>>>>
>     >>>>>>>>>
>     >
> 

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <jo...@confluent.io>.
Ivan,

That's a very good point about the "start" operator in the dynamic case. I
had no problem with "split()"; I was just questioning the necessity. Since
you've provided a proof of necessity, I'm in favor of the "split()" start
operator. Thanks!

Separately, I'm interested to see where the present discussion leads. I've
written enough Javascript code in my life to be suspicious of nested
closures. You have a good point about using method references (or indeed
function literals also work). It should be validating that this was also
the JS community's first approach to flattening the logic when their nested
closure situation got out of hand. Unfortunately, it's replacing nesting
with redirection, both of which disrupt code readability (but in different
ways for different reasons). In other words, I agree that function
references is *the* first-order solution if the nested code does indeed
become a problem.

However, the history of JS also tells us that function references aren't
the end of the story either, and you can see that by observing that there
have been two follow-on eras, as they continue trying to cope with the
consequences of living in such a callback-heavy language. First, you have
Futures/Promises, which essentially let you convert nested code to
method-chained code (Observables/FP is a popular variation on this). Most
lately, you have async/await, which is an effort to apply language (not
just API) syntax to the problem, and offer the "flattest" possible
programming style to solve the problem (because you get back to just one
code block per functional unit).

Stream-processing is a different domain, and Java+KStreams is nowhere near
as callback heavy as JS, so I don't think we have to take the JS story for
granted, but then again, I think we can derive some valuable lessons by
looking sideways to adjacent domains. I'm just bringing this up to inspire
further/deeper discussion. At the same time, just like JS, we can afford to
take an iterative approach to the problem.

Separately again, I'm interested in the post-branch merge (and I'd also add
join) problem that Paul brought up. We can clearly punt on it, by
terminating the nested branches with sink operators. But is there a DSL way
to do it?

Thanks again for your driving this,
-John

On Thu, May 2, 2019 at 7:39 PM Paul Whalen <pg...@gmail.com> wrote:

> Ivan, I’ll definitely forfeit my point on the clumsiness of the
> branch(predicate, consumer) solution, I don’t see any real drawbacks for
> the dynamic case.
>
> IMO the one trade off to consider at this point is the scope question. I
> don’t know if I totally agree that “we rarely need them in the same scope”
> since merging the branches back together later seems like a perfectly
> plausible use case that can be a lot nicer when the branched streams are in
> the same scope. That being said, for the reasons Ivan listed, I think it is
> overall the better solution - working around the scope thing is easy enough
> if you need to.
>
> > On May 2, 2019, at 7:00 PM, Ivan Ponomarev <ip...@mail.ru.invalid>
> wrote:
> >
> > Hello everyone, thank you all for joining the discussion!
> >
> > Well, I don't think the idea of named branches, be it a LinkedHashMap
> (no other Map will do, because order of definition matters) or `branch`
> method  taking name and Consumer has more advantages than drawbacks.
> >
> > In my opinion, the only real positive outcome from Michael's proposal is
> that all the returned branches are in the same scope. But 1) we rarely need
> them in the same scope 2) there is a workaround for the scope problem,
> described in the KIP.
> >
> > 'Inlining the complex logic' is not a problem, because we can use method
> references instead of lambdas. In real world scenarios you tend to split
> the complex logic to methods anyway, so the code is going to be clean.
> >
> > The drawbacks are strong. The cohesion between predicates and handlers
> is lost. We have to define predicates in one place, and handlers in
> another. This opens the door for bugs:
> >
> > - what if we forget to define a handler for a name? or a name for a
> handler?
> > - what if we misspell a name?
> > - what if we copy-paste and duplicate a name?
> >
> > What Michael propose would have been totally OK if we had been writing
> the API in Lua, Ruby or Python. In those languages the "dynamic naming"
> approach would have looked most concise and beautiful. But in Java we
> expect all the problems related to identifiers to be eliminated in compile
> time.
> >
> > Do we have to invent duck-typing for the Java API?
> >
> > And if we do, what advantage are we supposed to get besides having all
> the branches in the same scope? Michael, maybe I'm missing your point?
> >
> > ---
> >
> > Earlier in this discussion John Roesler also proposed to do without
> "start branching" operator, and later Paul mentioned that in the case when
> we have to add a dynamic number of branches, the current KIP is 'clumsier'
> compared to Michael's 'Map' solution. Let me address both comments here.
> >
> > 1) "Start branching" operator (I think that *split* is a good name for
> it indeed) is critical when we need to do a dynamic branching, see example
> below.
> >
> > 2) No, dynamic branching in current KIP is not clumsy at all. Imagine a
> real-world scenario when you need one branch per enum value (say,
> RecordType). You can have something like this:
> >
> > /*John:if we had to start with stream.branch(...) here, it would have
> been much messier.*/
> > KBranchedStream branched = stream.split();
> >
> > /*Not clumsy at all :-)*/
> > for (RecordType recordType : RecordType.values())
> >             branched = branched.branch((k, v) -> v.getRecType() ==
> recordType,
> >                     recordType::processRecords);
> >
> > Regards,
> >
> > Ivan
> >
> >
> > 02.05.2019 14:40, Matthias J. Sax пишет:
> >> I also agree with Michael's observation about the core problem of
> >> current `branch()` implementation.
> >>
> >> However, I also don't like to pass in a clumsy Map object. My thinking
> >> was more aligned with Paul's proposal to just add a name to each
> >> `branch()` statement and return a `Map<String,KStream>`.
> >>
> >> It makes the code easier to read, and also make the order of
> >> `Predicates` (that is essential) easier to grasp.
> >>
> >>>>>> Map<String, KStream<K, V>> branches = stream.split()
> >>>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>>    .defaultBranch("defaultBranch");
> >> An open question is the case for which no defaultBranch() should be
> >> specified. Atm, `split()` and `branch()` would return `BranchedKStream`
> >> and the call to `defaultBranch()` that returns the `Map` is mandatory
> >> (what is not the case atm). Or is this actually not a real problem,
> >> because users can just ignore the branch returned by `defaultBranch()`
> >> in the result `Map` ?
> >>
> >>
> >> About "inlining": So far, it seems to be a matter of personal
> >> preference. I can see arguments for both, but no "killer argument" yet
> >> that clearly make the case for one or the other.
> >>
> >>
> >> -Matthias
> >>
> >>> On 5/1/19 6:26 PM, Paul Whalen wrote:
> >>> Perhaps inlining is the wrong terminology. It doesn’t require that a
> lambda with the full downstream topology be defined inline - it can be a
> method reference as with Ivan’s original suggestion.  The advantage of
> putting the predicate and its downstream logic (Consumer) together in
> branch() is that they are required to be near to each other.
> >>>
> >>> Ultimately the downstream code has to live somewhere, and deep branch
> trees will be hard to read regardless.
> >>>
> >>>> On May 1, 2019, at 1:07 PM, Michael Drogalis <
> michael.drogalis@confluent.io> wrote:
> >>>>
> >>>> I'm less enthusiastic about inlining the branch logic with its
> downstream
> >>>> functionality. Programs that have deep branch trees will quickly
> become
> >>>> harder to read as a single unit.
> >>>>
> >>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen <pg...@gmail.com>
> wrote:
> >>>>>
> >>>>> Also +1 on the issues/goals as Michael outlined them, I think that
> sets a
> >>>>> great framework for the discussion.
> >>>>>
> >>>>> Regarding the SortedMap solution, my understanding is that the
> current
> >>>>> proposal in the KIP is what is in my PR which (pending naming
> decisions) is
> >>>>> roughly this:
> >>>>>
> >>>>> stream.split()
> >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
> >>>>>    .defaultBranch(Consumer<KStream<K, V>>);
> >>>>>
> >>>>> Obviously some ordering is necessary, since branching as a construct
> >>>>> doesn't work without it, but this solution seems like it provides as
> much
> >>>>> associativity as the SortedMap solution, because each branch() call
> >>>>> directly associates the "conditional" with the "code block."  The
> value it
> >>>>> provides over the KIP solution is the accessing of streams in the
> same
> >>>>> scope.
> >>>>>
> >>>>> The KIP solution is less "dynamic" than the SortedMap solution in
> the sense
> >>>>> that it is slightly clumsier to add a dynamic number of branches,
> but it is
> >>>>> certainly possible.  It seems to me like the API should favor the
> "static"
> >>>>> case anyway, and should make it simple and readable to fluently
> declare and
> >>>>> access your branches in-line.  It also makes it impossible to ignore
> a
> >>>>> branch, and it is possible to build an (almost) identical SortedMap
> >>>>> solution on top of it.
> >>>>>
> >>>>> I could also see a middle ground where instead of a raw SortedMap
> being
> >>>>> taken in, branch() takes a name and not a Consumer.  Something like
> this:
> >>>>>
> >>>>> Map<String, KStream<K, V>> branches = stream.split()
> >>>>>    .branch("branchOne", Predicate<K, V>)
> >>>>>    .branch( "branchTwo", Predicate<K, V>)
> >>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
> >>>>>
> >>>>> Pros for that solution:
> >>>>> - accessing branched KStreams in same scope
> >>>>> - no double brace initialization, hopefully slightly more readable
> than
> >>>>> SortedMap
> >>>>>
> >>>>> Cons
> >>>>> - downstream branch logic cannot be specified inline which makes it
> harder
> >>>>> to read top to bottom (like existing API and SortedMap, but unlike
> the KIP)
> >>>>> - you can forget to "handle" one of the branched streams (like
> existing
> >>>>> API and SortedMap, but unlike the KIP)
> >>>>>
> >>>>> (KBranchedStreams could even work *both* ways but perhaps that's
> overdoing
> >>>>> it).
> >>>>>
> >>>>> Overall I'm curious how important it is to be able to easily access
> the
> >>>>> branched KStream in the same scope as the original.  It's possible
> that it
> >>>>> doesn't need to be handled directly by the API, but instead left up
> to the
> >>>>> user.  I'm sort of in the middle on it.
> >>>>>
> >>>>> Paul
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman <
> sophie@confluent.io>
> >>>>> wrote:
> >>>>>
> >>>>>> I'd like to +1 what Michael said about the issues with the existing
> >>>>> branch
> >>>>>> method, I agree with what he's outlined and I think we should
> proceed by
> >>>>>> trying to alleviate these problems. Specifically it seems important
> to be
> >>>>>> able to cleanly access the individual branches (eg by mapping
> >>>>>> name->stream), which I thought was the original intention of this
> KIP.
> >>>>>>
> >>>>>> That said, I don't think we should so easily give in to the double
> brace
> >>>>>> anti-pattern or force ours users into it if at all possible to
> >>>>> avoid...just
> >>>>>> my two cents.
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Sophie
> >>>>>>
> >>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
> >>>>>> michael.drogalis@confluent.io> wrote:
> >>>>>>
> >>>>>>> I’d like to propose a different way of thinking about this. To me,
> >>>>> there
> >>>>>>> are three problems with the existing branch signature:
> >>>>>>>
> >>>>>>> 1. If you use it the way most people do, Java raises unsafe type
> >>>>>> warnings.
> >>>>>>> 2. The way in which you use the stream branches is positionally
> coupled
> >>>>>> to
> >>>>>>> the ordering of the conditionals.
> >>>>>>> 3. It is brittle to extend existing branch calls with additional
> code
> >>>>>>> paths.
> >>>>>>>
> >>>>>>> Using associative constructs instead of relying on ordered
> constructs
> >>>>>> would
> >>>>>>> be a stronger approach. Consider a signature that instead looks
> like
> >>>>>> this:
> >>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String,
> Predicate<?
> >>>>>>> super K,? super V>>);
> >>>>>>>
> >>>>>>> Branches are given names in a map, and as a result, the API
> returns a
> >>>>>>> mapping of names to streams. The ordering of the conditionals is
> >>>>>> maintained
> >>>>>>> because it’s a sorted map. Insert order determines the order of
> >>>>>> evaluation.
> >>>>>>> This solves problem 1 because there are no more varargs. It solves
> >>>>>> problem
> >>>>>>> 2 because you no longer lean on ordering to access the branch
> you’re
> >>>>>>> interested in. It solves problem 3 because you can introduce
> another
> >>>>>>> conditional by simply attaching another name to the structure,
> rather
> >>>>>> than
> >>>>>>> messing with the existing indices.
> >>>>>>>
> >>>>>>> One of the drawbacks is that creating the map inline is
> historically
> >>>>>>> awkward in Java. I know it’s an anti-pattern to use voluminously,
> but
> >>>>>>> double brace initialization would clean up the aesthetics.
> >>>>>>>
> >>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io>
> >>>>> wrote:
> >>>>>>>> Hi Ivan,
> >>>>>>>>
> >>>>>>>> Thanks for the update.
> >>>>>>>>
> >>>>>>>> FWIW, I agree with Matthias that the current "start branching"
> >>>>> operator
> >>>>>>> is
> >>>>>>>> confusing when named the same way as the actual branches. "Split"
> >>>>> seems
> >>>>>>>> like a good name. Alternatively, we can do without a "start
> >>>>> branching"
> >>>>>>>> operator at all, and just do:
> >>>>>>>>
> >>>>>>>> stream
> >>>>>>>>      .branch(Predicate)
> >>>>>>>>      .branch(Predicate)
> >>>>>>>>      .defaultBranch();
> >>>>>>>>
> >>>>>>>> Tentatively, I think that this branching operation should be
> >>>>> terminal.
> >>>>>>> That
> >>>>>>>> way, we don't create ambiguity about how to use it. That is,
> `branch`
> >>>>>>>> should return `KBranchedStream`, while `defaultBranch` is `void`,
> to
> >>>>>>>> enforce that it comes last, and that there is only one definition
> of
> >>>>>> the
> >>>>>>>> default branch. Potentially, we should log a warning if there's no
> >>>>>>> default,
> >>>>>>>> and additionally log a warning (or throw an exception) if a record
> >>>>>> falls
> >>>>>>>> though with no default.
> >>>>>>>>
> >>>>>>>> Thoughts?
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>> -John
> >>>>>>>>
> >>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
> >>>>> matthias@confluent.io
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Thanks for updating the KIP and your answers.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> this is to make the name similar to String#split
> >>>>>>>>>>> that also returns an array, right?
> >>>>>>>>> The intend was to avoid name duplication. The return type should
> >>>>>> _not_
> >>>>>>>>> be an array.
> >>>>>>>>>
> >>>>>>>>> The current proposal is
> >>>>>>>>>
> >>>>>>>>> stream.branch()
> >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>      .defaultBranch();
> >>>>>>>>>
> >>>>>>>>> IMHO, this reads a little odd, because the first `branch()` does
> >>>>> not
> >>>>>>>>> take any parameters and has different semantics than the later
> >>>>>>>>> `branch()` calls. Note, that from the code snippet above, it's
> >>>>> hidden
> >>>>>>>>> that the first call is `KStream#branch()` while the others are
> >>>>>>>>> `KBranchedStream#branch()` what makes reading the code harder.
> >>>>>>>>>
> >>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`, I
> though
> >>>>>> it
> >>>>>>>>> might be better to also rename `KStream#branch()` to avoid the
> >>>>> naming
> >>>>>>>>> overlap that seems to be confusing. The following reads much
> >>>>> cleaner
> >>>>>> to
> >>>>>>>> me:
> >>>>>>>>> stream.split()
> >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>      .branch(Predicate)
> >>>>>>>>>      .defaultBranch();
> >>>>>>>>>
> >>>>>>>>> Maybe there is a better alternative to `split()` though to avoid
> >>>>> the
> >>>>>>>>> naming overlap.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> 'default' is, however, a reserved word, so unfortunately we
> >>>>> cannot
> >>>>>>> have
> >>>>>>>>> a method with such name :-)
> >>>>>>>>>
> >>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up with a
> >>>>> short
> >>>>>>>> name?
> >>>>>>>>>
> >>>>>>>>> Can you add the interface `KBranchedStream` to the KIP with all
> >>>>> it's
> >>>>>>>>> methods? It will be part of public API and should be contained in
> >>>>> the
> >>>>>>>>> KIP. For example, it's unclear atm, what the return type of
> >>>>>>>>> `defaultBranch()` is.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> You did not comment on the idea to add a `KBranchedStream#get(int
> >>>>>>> index)
> >>>>>>>>> -> KStream` method to get the individually branched-KStreams.
> Would
> >>>>>> be
> >>>>>>>>> nice to get your feedback about it. It seems you suggest that
> users
> >>>>>>>>> would need to write custom utility code otherwise, to access
> them.
> >>>>> We
> >>>>>>>>> should discuss the pros and cons of both approaches. It feels
> >>>>>>>>> "incomplete" to me atm, if the API has no built-in support to get
> >>>>> the
> >>>>>>>>> branched-KStreams directly.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> -Matthias
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> >>>>>>>>>> Hi all!
> >>>>>>>>>>
> >>>>>>>>>> I have updated the KIP-418 according to the new vision.
> >>>>>>>>>>
> >>>>>>>>>> Matthias, thanks for your comment!
> >>>>>>>>>>
> >>>>>>>>>>> Renaming KStream#branch() -> #split()
> >>>>>>>>>> I can see your point: this is to make the name similar to
> >>>>>>> String#split
> >>>>>>>>>> that also returns an array, right? But is it worth the loss of
> >>>>>>>> backwards
> >>>>>>>>>> compatibility? We can have overloaded branch() as well without
> >>>>>>>> affecting
> >>>>>>>>>> the existing code. Maybe the old array-based `branch` method
> >>>>> should
> >>>>>>> be
> >>>>>>>>>> deprecated, but this is a subject for discussion.
> >>>>>>>>>>
> >>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
> >>>>> BranchingKStream#branch(),
> >>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> >>>>>>>>>>
> >>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default' is,
> >>>>>>> however, a
> >>>>>>>>>> reserved word, so unfortunately we cannot have a method with
> such
> >>>>>>> name
> >>>>>>>>> :-)
> >>>>>>>>>>> defaultBranch() does take an `Predicate` as argument, but I
> >>>>> think
> >>>>>>> that
> >>>>>>>>>> is not required?
> >>>>>>>>>>
> >>>>>>>>>> Absolutely! I think that was just copy-paste error or something.
> >>>>>>>>>>
> >>>>>>>>>> Dear colleagues,
> >>>>>>>>>>
> >>>>>>>>>> please revise the new version of the KIP and Paul's PR
> >>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
> >>>>>>>>>>
> >>>>>>>>>> Any new suggestions/objections?
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>>
> >>>>>>>>>> Ivan
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
> >>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems that
> >>>>>>> everybody
> >>>>>>>>>>> agrees that the current branch() method using arrays is not
> >>>>>> optimal.
> >>>>>>>>>>> I had a quick look into the PR and I like the overall proposal.
> >>>>>>> There
> >>>>>>>>>>> are some minor things we need to consider. I would recommend
> the
> >>>>>>>>>>> following renaming:
> >>>>>>>>>>>
> >>>>>>>>>>> KStream#branch() -> #split()
> >>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
> >>>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> >>>>>>>>>>>
> >>>>>>>>>>> It's just a suggestion to get slightly shorter method names.
> >>>>>>>>>>>
> >>>>>>>>>>> In the current PR, defaultBranch() does take an `Predicate` as
> >>>>>>>> argument,
> >>>>>>>>>>> but I think that is not required?
> >>>>>>>>>>>
> >>>>>>>>>>> Also, we should consider KIP-307, that was recently accepted
> and
> >>>>>> is
> >>>>>>>>>>> currently implemented:
> >>>>>>>>>>>
> >>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>>>>>>>>>> Ie, we should add overloads that accepted a `Named` parameter.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> For the issue that the created `KStream` object are in
> different
> >>>>>>>> scopes:
> >>>>>>>>>>> could we extend `KBranchedStream` with a `get(int index)`
> method
> >>>>>>> that
> >>>>>>>>>>> returns the corresponding "branched" result `KStream` object?
> >>>>>> Maybe,
> >>>>>>>> the
> >>>>>>>>>>> second argument of `addBranch()` should not be a
> >>>>>> `Consumer<KStream>`
> >>>>>>>> but
> >>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return whatever
> >>>>>> the
> >>>>>>>>>>> `Function` returns?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Finally, I would also suggest to update the KIP with the
> current
> >>>>>>>>>>> proposal. That makes it easier to review.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> -Matthias
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>
> >>>>>>>>>>>> I'm a bit of a novice here as well, but I think it makes sense
> >>>>>> for
> >>>>>>>> you
> >>>>>>>>> to
> >>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously we'll
> >>>>> need
> >>>>>>>> some
> >>>>>>>>>>>> buy-in from committers that have actual binding votes on
> >>>>> whether
> >>>>>>> the
> >>>>>>>>> KIP
> >>>>>>>>>>>> could be adopted.  It would be great to hear if they think
> this
> >>>>>> is
> >>>>>>> a
> >>>>>>>>> good
> >>>>>>>>>>>> idea overall.  I'm not sure if that happens just by starting a
> >>>>>>> vote,
> >>>>>>>>> or if
> >>>>>>>>>>>> there is generally some indication of interest beforehand.
> >>>>>>>>>>>>
> >>>>>>>>>>>> That being said, I'll continue the discussion a bit: assuming
> >>>>> we
> >>>>>> do
> >>>>>>>>> move
> >>>>>>>>>>>> forward the solution of "stream.branch() returns
> >>>>>> KBranchedStream",
> >>>>>>> do
> >>>>>>>>> we
> >>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I would
> >>>>> favor
> >>>>>>>>>>>> deprecating, since having two mutually exclusive APIs that
> >>>>>>> accomplish
> >>>>>>>>> the
> >>>>>>>>>>>> same thing is confusing, especially when they're fairly
> similar
> >>>>>>>>> anyway.  We
> >>>>>>>>>>>> just need to be sure we're not making something
> >>>>>>> impossible/difficult
> >>>>>>>>> that
> >>>>>>>>>>>> is currently possible/easy.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regarding my PR - I think the general structure would work,
> >>>>> it's
> >>>>>>>> just a
> >>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
> >>>>>>> particular,
> >>>>>>>>>>>> passing in the "predicates" and "children" lists which get
> >>>>>> modified
> >>>>>>>> in
> >>>>>>>>>>>> KBranchedStream but read from all the way KStreamLazyBranch is
> >>>>> a
> >>>>>>> bit
> >>>>>>>>>>>> complicated to follow.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>> Paul
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
> >>>>>> iponomarev@mail.ru
> >>>>>>>>> wrote:
> >>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I read your code carefully and now I am fully convinced: your
> >>>>>>>> proposal
> >>>>>>>>>>>>> looks better and should work. We just have to document the
> >>>>>> crucial
> >>>>>>>>> fact
> >>>>>>>>>>>>> that KStream consumers are invoked as they're added. And then
> >>>>>> it's
> >>>>>>>> all
> >>>>>>>>>>>>> going to be very nice.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> What shall we do now? I should re-write the KIP and resume
> the
> >>>>>>>>>>>>> discussion here, right?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
> >>>>> starting
> >>>>>>>> point
> >>>>>>>>> if
> >>>>>>>>>>>>> we go in this direction'? To me it looks like a good starting
> >>>>>>> point.
> >>>>>>>>> But
> >>>>>>>>>>>>> as a novice in this project I might miss some important
> >>>>> details.
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
> >>>>> stream.branch()
> >>>>>>>>> solution
> >>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
> >>>>> invoked
> >>>>>> as
> >>>>>>>>> they’re
> >>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user still
> >>>>>> ought
> >>>>>>> to
> >>>>>>>>> be
> >>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and depend on
> >>>>> the
> >>>>>>>>> branched
> >>>>>>>>>>>>> streams having been set.
> >>>>>>>>>>>>>> The issue I mean to point out is that it is hard to access
> >>>>> the
> >>>>>>>>> branched
> >>>>>>>>>>>>> streams in the same scope as the original stream (that is,
> not
> >>>>>>>> inside
> >>>>>>>>> the
> >>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
> >>>>> solutions.
> >>>>>> It
> >>>>>>>>> can be
> >>>>>>>>>>>>> worked around though.
> >>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m excited
> >>>>> to
> >>>>>>>> hear
> >>>>>>>>>>>>> your thoughts!]
> >>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
> >>>>>> iponomarev@mail.ru
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>>> Hi Paul!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
> >>>>>>>>>>>>> streamsBuilder.build() also looked great for me at first
> >>>>> glance,
> >>>>>>> but
> >>>>>>>>> ---
> >>>>>>>>>>>>>>>> the newly branched streams are not available in the same
> >>>>>> scope
> >>>>>>> as
> >>>>>>>>> each
> >>>>>>>>>>>>> other.  That is, if we wanted to merge them back together
> >>>>> again
> >>>>>> I
> >>>>>>>>> don't see
> >>>>>>>>>>>>> a way to do that.
> >>>>>>>>>>>>>>> You just took the words right out of my mouth, I was just
> >>>>>> going
> >>>>>>> to
> >>>>>>>>>>>>> write in details about this issue.
> >>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say we need
> >>>>> to
> >>>>>>>>> identify
> >>>>>>>>>>>>> customers who have bought coffee and made a purchase in the
> >>>>>>>>> electronics
> >>>>>>>>>>>>> store to give them coupons.
> >>>>>>>>>>>>>>> This is the code I usually write under these circumstances
> >>>>>> using
> >>>>>>>> my
> >>>>>>>>>>>>> 'brancher' class:
> >>>>>>>>>>>>>>> @Setter
> >>>>>>>>>>>>>>> class CouponIssuer{
> >>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
> >>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>   KStream<...> coupons(){
> >>>>>>>>>>>>>>>       return
> >>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>>>>>>>>>>       /*In the real world the code here can be complex, so
> >>>>>>>>> creation of
> >>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in order to
> >>>>>>>> separate
> >>>>>>>>>>>>> classes' responsibilities.*/
> >>>>>>>>>>>>>>>  }
> >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>>>>>>>>>>     .branch(predicate1, couponIssuer::setCoffePurchases)
> >>>>>>>>>>>>>>>     .branch(predicate2,
> >>>>>> couponIssuer::setElectronicsPurchases)
> >>>>>>>>>>>>>>>     .onTopOf(transactionStream);
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up
> everything
> >>>>>>>> later,
> >>>>>>>>>>>>> without the terminal operation!!!*/
> >>>>>>>>>>>>>>> couponIssuer.coupons()...
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Does this make sense?  In order to properly initialize the
> >>>>>>>>> CouponIssuer
> >>>>>>>>>>>>> we need the terminal operation to be called before
> >>>>>>>>> streamsBuilder.build()
> >>>>>>>>>>>>> is called.
> >>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is
> essentially
> >>>>>> the
> >>>>>>>>> next
> >>>>>>>>>>>>> KIP I was going to write here. I have some thoughts based on
> >>>>> my
> >>>>>>>>> experience,
> >>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
> >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a fluent
> >>>>> API
> >>>>>>>> based
> >>>>>>>>>>>>> off of
> >>>>>>>>>>>>>>>> KStream here (https://github.com/apache/kafka/pull/6512),
> >>>>>> and
> >>>>>>> I
> >>>>>>>>> think
> >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>> succeeded at removing both cons.
> >>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
> >>>>>>> compatibility
> >>>>>>>>>>>>> issues,
> >>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware that Java
> >>>>> is
> >>>>>>>> smart
> >>>>>>>>>>>>> enough to
> >>>>>>>>>>>>>>>>    distinguish between a branch(varargs...) returning one
> >>>>>>> thing
> >>>>>>>>> and
> >>>>>>>>>>>>> branch()
> >>>>>>>>>>>>>>>>    with no arguments returning another thing.
> >>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually need
> >>>>> it.
> >>>>>>> We
> >>>>>>>>> can
> >>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who shares
> >>>>>> its
> >>>>>>>>> state
> >>>>>>>>>>>>> with the
> >>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the branching.
> >>>>>>> It's
> >>>>>>>>> not
> >>>>>>>>>>>>> terribly
> >>>>>>>>>>>>>>>>    pretty in its current form, but I think it demonstrates
> >>>>>> its
> >>>>>>>>>>>>> feasibility.
> >>>>>>>>>>>>>>>> To be clear, I don't think that pull request should be
> >>>>> final
> >>>>>> or
> >>>>>>>>> even a
> >>>>>>>>>>>>>>>> starting point if we go in this direction, I just wanted
> to
> >>>>>> see
> >>>>>>>> how
> >>>>>>>>>>>>>>>> challenging it would be to get the API working.
> >>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing solution
> >>>>>>> could
> >>>>>>>> be
> >>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
> >>>>> suggested
> >>>>>>>> was a
> >>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched
> streams
> >>>>>> are
> >>>>>>>> not
> >>>>>>>>>>>>>>>> available in the same scope as each other.  That is, if we
> >>>>>>> wanted
> >>>>>>>>> to
> >>>>>>>>>>>>> merge
> >>>>>>>>>>>>>>>> them back together again I don't see a way to do that.
> The
> >>>>>> KIP
> >>>>>>>>>>>>> proposal
> >>>>>>>>>>>>>>>> has the same issue, though - all this means is that for
> >>>>>> either
> >>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the table.
> >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> >>>>>>>>> iponomarev@mail.ru>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to this
> >>>>>> point.
> >>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that branch API
> >>>>>>> needs
> >>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> There are two potential ways to do it:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
> >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
> >>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
> >>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf returns
> >>>>>> its
> >>>>>>>>> argument
> >>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't
> >>>>> make
> >>>>>>>> sense
> >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
> >>>>>>>>> contrasts the
> >>>>>>>>>>>>>>>>> fluency of other KStream methods.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 2. (as Paul proposes)
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
> >>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
> >>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
> >>>>>>>>> defaultBranch(..)
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> noDefault() return void
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface is
> >>>>>> defined.
> >>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
> >>>>>>>> (defaultBranch(ks->)
> >>>>>>>>> and
> >>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to miss the
> >>>>>> fact
> >>>>>>>>> that one
> >>>>>>>>>>>>>>>>> of the terminal methods should be called. If these
> methods
> >>>>>> are
> >>>>>>>> not
> >>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do better?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> >>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> >>>>>>>>>>>>>>>>>>> Paul,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I see your point when you are talking about
> >>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be implemented
> the
> >>>>>>> easy
> >>>>>>>>> way.
> >>>>>>>>>>>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> user could specify a terminal method that assumes
> >>>>> nothing
> >>>>>>>> will
> >>>>>>>>>>>>> reach
> >>>>>>>>>>>>>>>>>>>> the default branch,
> >>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only option
> >>>>>> besides
> >>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we want to
> >>>>>> just
> >>>>>>>>> silently
> >>>>>>>>>>>>>>>>>>> drop the messages that didn't match any predicate. 2)
> >>>>>>> Throwing
> >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing looks
> >>>>>> like a
> >>>>>>>> bad
> >>>>>>>>>>>>> idea.
> >>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to emit a
> >>>>>>>> special
> >>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly where
> >>>>>>> `default`
> >>>>>>>>> can
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> used.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>> InternalTopologyBuilder
> >>>>>>> to
> >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
> >>>>>>> error
> >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>> becomes an issue.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
> >>>>> compiled
> >>>>>>> and
> >>>>>>>>> run?
> >>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't compile if
> >>>>> used
> >>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a method chain
> >>>>>>>> starting
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference between
> >>>>>>>> runtime
> >>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
> >>>>> instantly
> >>>>>> on
> >>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a compilation
> >>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being
> required.
> >>>>>>> But
> >>>>>>>> is
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
> >>>>>> defaultBranch
> >>>>>>>>> they
> >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?) just
> as
> >>>>>>>>> easily.  In
> >>>>>>>>>>>>>>>>>>>> fact I
> >>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API - a
> >>>>> user
> >>>>>>>> could
> >>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach the
> >>>>>> default
> >>>>>>>>> branch,
> >>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.  That
> >>>>> seems
> >>>>>>> like
> >>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>> improvement over the current branch() API, which
> allows
> >>>>>> for
> >>>>>>>> the
> >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>> subtle
> >>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting dropped.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has to be
> >>>>>> well
> >>>>>>>>>>>>>>>>>>>> documented, but
> >>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
> >>>>> InternalTopologyBuilder
> >>>>>>> to
> >>>>>>>>> track
> >>>>>>>>>>>>>>>>>>>> dangling
> >>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a
> clear
> >>>>>>> error
> >>>>>>>>>>>>> before it
> >>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is a
> >>>>> "build
> >>>>>>>> step"
> >>>>>>>>>>>>> where
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
> >>>>>> StreamsBuilder.build()
> >>>>>>> is
> >>>>>>>>>>>>> called.
> >>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I agree
> >>>>> that
> >>>>>>> it's
> >>>>>>>>>>>>>>>>>>>> critical to
> >>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input
> stream.
> >>>>>>> With
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> fluent
> >>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all other
> >>>>>>> operations
> >>>>>>>>> do -
> >>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> want to process off the original KStream multiple
> >>>>> times,
> >>>>>>> you
> >>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>> need the
> >>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many
> operations
> >>>>>> on
> >>>>>>> it
> >>>>>>>>> as
> >>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> desire.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> >>>>>>>>> iponomarev@mail.ru
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not always
> need
> >>>>>> the
> >>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal operation we
> >>>>> don't
> >>>>>>>> know
> >>>>>>>>>>>>> when to
> >>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we
> >>>>> can
> >>>>>> do
> >>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I understand your point that the need of special
> >>>>> object
> >>>>>>>>>>>>> construction
> >>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods. But
> >>>>> here
> >>>>>> we
> >>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the flow,
> >>>>> so
> >>>>>> I
> >>>>>>>>> think
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> >>>>>>>>>>>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this API, but I
> >>>>>> find
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it contrasts the
> >>>>>>> fluency
> >>>>>>>>> of
> >>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call
> >>>>> a
> >>>>>>>>> method on
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch cases
> >>>>> are
> >>>>>>>>> defined
> >>>>>>>>>>>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very
> >>>>>> nice
> >>>>>>>>> and the
> >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around how we
> >>>>>>> specify
> >>>>>>>>> the
> >>>>>>>>>>>>> source
> >>>>>>>>>>>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1, this::handle1)
> >>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2, this::handle2)
> >>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
> >>>>>>>> KStreamBrancher
> >>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>> something,
> >>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and terminated by
> >>>>>>>>>>>>> defaultBranch()
> >>>>>>>>>>>>>>>>>>>>>> (which
> >>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously incompatible with
> >>>>> the
> >>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>> API, so
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a different
> >>>>>> name,
> >>>>>>>> but
> >>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
> >>>>>> something
> >>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>> branched()
> >>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It
> >>>>>> seems
> >>>>>>>>> like it
> >>>>>>>>>>>>>>>>>>>>>> does to
> >>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching while also
> >>>>>>>> allowing
> >>>>>>>>> you
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of
> KBranchedStreams
> >>>>>> if
> >>>>>>>>> desired.
> >>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
> >>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
> >>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1, this::handleFirstCase)
> >>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2, this::handleSecondCase)
> >>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> >>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher
> >>>>> takes a
> >>>>>>>>> Consumer
> >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the example in
> >>>>>> the
> >>>>>>>> KIP
> >>>>>>>>>>>>> shows
> >>>>>>>>>>>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
> >>>>>>>>> (KafkaStreams#to()
> >>>>>>>>>>>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would we
> >>>>> handle
> >>>>>>> the
> >>>>>>>>> case
> >>>>>>>>>>>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to continue
> >>>>>>>> processing
> >>>>>>>>> and
> >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the branched
> >>>>>> stream
> >>>>>>>>>>>>> immediately?
> >>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if we had
> >>>>>>>> something
> >>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
> >>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
> >>>>>>>>> bbejeck@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP-
> >>>>> 418.
> >>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418.
> >>>>> Please
> >>>>>>>> take
> >>>>>>>>> a
> >>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback
> :)
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
> >>>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
> >>>>> https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>
> >
>
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Paul Whalen <pg...@gmail.com>.
Ivan, I’ll definitely forfeit my point on the clumsiness of the branch(predicate, consumer) solution, I don’t see any real drawbacks for the dynamic case. 

IMO the one trade off to consider at this point is the scope question. I don’t know if I totally agree that “we rarely need them in the same scope” since merging the branches back together later seems like a perfectly plausible use case that can be a lot nicer when the branched streams are in the same scope. That being said, for the reasons Ivan listed, I think it is overall the better solution - working around the scope thing is easy enough if you need to. 

> On May 2, 2019, at 7:00 PM, Ivan Ponomarev <ip...@mail.ru.invalid> wrote:
> 
> Hello everyone, thank you all for joining the discussion!
> 
> Well, I don't think the idea of named branches, be it a LinkedHashMap (no other Map will do, because order of definition matters) or `branch` method  taking name and Consumer has more advantages than drawbacks.
> 
> In my opinion, the only real positive outcome from Michael's proposal is that all the returned branches are in the same scope. But 1) we rarely need them in the same scope 2) there is a workaround for the scope problem, described in the KIP.
> 
> 'Inlining the complex logic' is not a problem, because we can use method references instead of lambdas. In real world scenarios you tend to split the complex logic to methods anyway, so the code is going to be clean.
> 
> The drawbacks are strong. The cohesion between predicates and handlers is lost. We have to define predicates in one place, and handlers in another. This opens the door for bugs:
> 
> - what if we forget to define a handler for a name? or a name for a handler?
> - what if we misspell a name?
> - what if we copy-paste and duplicate a name?
> 
> What Michael propose would have been totally OK if we had been writing the API in Lua, Ruby or Python. In those languages the "dynamic naming" approach would have looked most concise and beautiful. But in Java we expect all the problems related to identifiers to be eliminated in compile time.
> 
> Do we have to invent duck-typing for the Java API?
> 
> And if we do, what advantage are we supposed to get besides having all the branches in the same scope? Michael, maybe I'm missing your point?
> 
> ---
> 
> Earlier in this discussion John Roesler also proposed to do without "start branching" operator, and later Paul mentioned that in the case when we have to add a dynamic number of branches, the current KIP is 'clumsier' compared to Michael's 'Map' solution. Let me address both comments here.
> 
> 1) "Start branching" operator (I think that *split* is a good name for it indeed) is critical when we need to do a dynamic branching, see example below.
> 
> 2) No, dynamic branching in current KIP is not clumsy at all. Imagine a real-world scenario when you need one branch per enum value (say, RecordType). You can have something like this:
> 
> /*John:if we had to start with stream.branch(...) here, it would have been much messier.*/
> KBranchedStream branched = stream.split();
> 
> /*Not clumsy at all :-)*/
> for (RecordType recordType : RecordType.values())
>             branched = branched.branch((k, v) -> v.getRecType() == recordType,
>                     recordType::processRecords);
> 
> Regards,
> 
> Ivan
> 
> 
> 02.05.2019 14:40, Matthias J. Sax пишет:
>> I also agree with Michael's observation about the core problem of
>> current `branch()` implementation.
>> 
>> However, I also don't like to pass in a clumsy Map object. My thinking
>> was more aligned with Paul's proposal to just add a name to each
>> `branch()` statement and return a `Map<String,KStream>`.
>> 
>> It makes the code easier to read, and also make the order of
>> `Predicates` (that is essential) easier to grasp.
>> 
>>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>>    .defaultBranch("defaultBranch");
>> An open question is the case for which no defaultBranch() should be
>> specified. Atm, `split()` and `branch()` would return `BranchedKStream`
>> and the call to `defaultBranch()` that returns the `Map` is mandatory
>> (what is not the case atm). Or is this actually not a real problem,
>> because users can just ignore the branch returned by `defaultBranch()`
>> in the result `Map` ?
>> 
>> 
>> About "inlining": So far, it seems to be a matter of personal
>> preference. I can see arguments for both, but no "killer argument" yet
>> that clearly make the case for one or the other.
>> 
>> 
>> -Matthias
>> 
>>> On 5/1/19 6:26 PM, Paul Whalen wrote:
>>> Perhaps inlining is the wrong terminology. It doesn’t require that a lambda with the full downstream topology be defined inline - it can be a method reference as with Ivan’s original suggestion.  The advantage of putting the predicate and its downstream logic (Consumer) together in branch() is that they are required to be near to each other.
>>> 
>>> Ultimately the downstream code has to live somewhere, and deep branch trees will be hard to read regardless.
>>> 
>>>> On May 1, 2019, at 1:07 PM, Michael Drogalis <mi...@confluent.io> wrote:
>>>> 
>>>> I'm less enthusiastic about inlining the branch logic with its downstream
>>>> functionality. Programs that have deep branch trees will quickly become
>>>> harder to read as a single unit.
>>>> 
>>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen <pg...@gmail.com> wrote:
>>>>> 
>>>>> Also +1 on the issues/goals as Michael outlined them, I think that sets a
>>>>> great framework for the discussion.
>>>>> 
>>>>> Regarding the SortedMap solution, my understanding is that the current
>>>>> proposal in the KIP is what is in my PR which (pending naming decisions) is
>>>>> roughly this:
>>>>> 
>>>>> stream.split()
>>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>>> 
>>>>> Obviously some ordering is necessary, since branching as a construct
>>>>> doesn't work without it, but this solution seems like it provides as much
>>>>> associativity as the SortedMap solution, because each branch() call
>>>>> directly associates the "conditional" with the "code block."  The value it
>>>>> provides over the KIP solution is the accessing of streams in the same
>>>>> scope.
>>>>> 
>>>>> The KIP solution is less "dynamic" than the SortedMap solution in the sense
>>>>> that it is slightly clumsier to add a dynamic number of branches, but it is
>>>>> certainly possible.  It seems to me like the API should favor the "static"
>>>>> case anyway, and should make it simple and readable to fluently declare and
>>>>> access your branches in-line.  It also makes it impossible to ignore a
>>>>> branch, and it is possible to build an (almost) identical SortedMap
>>>>> solution on top of it.
>>>>> 
>>>>> I could also see a middle ground where instead of a raw SortedMap being
>>>>> taken in, branch() takes a name and not a Consumer.  Something like this:
>>>>> 
>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>    .branch("branchOne", Predicate<K, V>)
>>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>>> 
>>>>> Pros for that solution:
>>>>> - accessing branched KStreams in same scope
>>>>> - no double brace initialization, hopefully slightly more readable than
>>>>> SortedMap
>>>>> 
>>>>> Cons
>>>>> - downstream branch logic cannot be specified inline which makes it harder
>>>>> to read top to bottom (like existing API and SortedMap, but unlike the KIP)
>>>>> - you can forget to "handle" one of the branched streams (like existing
>>>>> API and SortedMap, but unlike the KIP)
>>>>> 
>>>>> (KBranchedStreams could even work *both* ways but perhaps that's overdoing
>>>>> it).
>>>>> 
>>>>> Overall I'm curious how important it is to be able to easily access the
>>>>> branched KStream in the same scope as the original.  It's possible that it
>>>>> doesn't need to be handled directly by the API, but instead left up to the
>>>>> user.  I'm sort of in the middle on it.
>>>>> 
>>>>> Paul
>>>>> 
>>>>> 
>>>>> 
>>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman <so...@confluent.io>
>>>>> wrote:
>>>>> 
>>>>>> I'd like to +1 what Michael said about the issues with the existing
>>>>> branch
>>>>>> method, I agree with what he's outlined and I think we should proceed by
>>>>>> trying to alleviate these problems. Specifically it seems important to be
>>>>>> able to cleanly access the individual branches (eg by mapping
>>>>>> name->stream), which I thought was the original intention of this KIP.
>>>>>> 
>>>>>> That said, I don't think we should so easily give in to the double brace
>>>>>> anti-pattern or force ours users into it if at all possible to
>>>>> avoid...just
>>>>>> my two cents.
>>>>>> 
>>>>>> Cheers,
>>>>>> Sophie
>>>>>> 
>>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>>> michael.drogalis@confluent.io> wrote:
>>>>>> 
>>>>>>> I’d like to propose a different way of thinking about this. To me,
>>>>> there
>>>>>>> are three problems with the existing branch signature:
>>>>>>> 
>>>>>>> 1. If you use it the way most people do, Java raises unsafe type
>>>>>> warnings.
>>>>>>> 2. The way in which you use the stream branches is positionally coupled
>>>>>> to
>>>>>>> the ordering of the conditionals.
>>>>>>> 3. It is brittle to extend existing branch calls with additional code
>>>>>>> paths.
>>>>>>> 
>>>>>>> Using associative constructs instead of relying on ordered constructs
>>>>>> would
>>>>>>> be a stronger approach. Consider a signature that instead looks like
>>>>>> this:
>>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
>>>>>>> super K,? super V>>);
>>>>>>> 
>>>>>>> Branches are given names in a map, and as a result, the API returns a
>>>>>>> mapping of names to streams. The ordering of the conditionals is
>>>>>> maintained
>>>>>>> because it’s a sorted map. Insert order determines the order of
>>>>>> evaluation.
>>>>>>> This solves problem 1 because there are no more varargs. It solves
>>>>>> problem
>>>>>>> 2 because you no longer lean on ordering to access the branch you’re
>>>>>>> interested in. It solves problem 3 because you can introduce another
>>>>>>> conditional by simply attaching another name to the structure, rather
>>>>>> than
>>>>>>> messing with the existing indices.
>>>>>>> 
>>>>>>> One of the drawbacks is that creating the map inline is historically
>>>>>>> awkward in Java. I know it’s an anti-pattern to use voluminously, but
>>>>>>> double brace initialization would clean up the aesthetics.
>>>>>>> 
>>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io>
>>>>> wrote:
>>>>>>>> Hi Ivan,
>>>>>>>> 
>>>>>>>> Thanks for the update.
>>>>>>>> 
>>>>>>>> FWIW, I agree with Matthias that the current "start branching"
>>>>> operator
>>>>>>> is
>>>>>>>> confusing when named the same way as the actual branches. "Split"
>>>>> seems
>>>>>>>> like a good name. Alternatively, we can do without a "start
>>>>> branching"
>>>>>>>> operator at all, and just do:
>>>>>>>> 
>>>>>>>> stream
>>>>>>>>      .branch(Predicate)
>>>>>>>>      .branch(Predicate)
>>>>>>>>      .defaultBranch();
>>>>>>>> 
>>>>>>>> Tentatively, I think that this branching operation should be
>>>>> terminal.
>>>>>>> That
>>>>>>>> way, we don't create ambiguity about how to use it. That is, `branch`
>>>>>>>> should return `KBranchedStream`, while `defaultBranch` is `void`, to
>>>>>>>> enforce that it comes last, and that there is only one definition of
>>>>>> the
>>>>>>>> default branch. Potentially, we should log a warning if there's no
>>>>>>> default,
>>>>>>>> and additionally log a warning (or throw an exception) if a record
>>>>>> falls
>>>>>>>> though with no default.
>>>>>>>> 
>>>>>>>> Thoughts?
>>>>>>>> 
>>>>>>>> Thanks,
>>>>>>>> -John
>>>>>>>> 
>>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>>> matthias@confluent.io
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> this is to make the name similar to String#split
>>>>>>>>>>> that also returns an array, right?
>>>>>>>>> The intend was to avoid name duplication. The return type should
>>>>>> _not_
>>>>>>>>> be an array.
>>>>>>>>> 
>>>>>>>>> The current proposal is
>>>>>>>>> 
>>>>>>>>> stream.branch()
>>>>>>>>>      .branch(Predicate)
>>>>>>>>>      .branch(Predicate)
>>>>>>>>>      .defaultBranch();
>>>>>>>>> 
>>>>>>>>> IMHO, this reads a little odd, because the first `branch()` does
>>>>> not
>>>>>>>>> take any parameters and has different semantics than the later
>>>>>>>>> `branch()` calls. Note, that from the code snippet above, it's
>>>>> hidden
>>>>>>>>> that the first call is `KStream#branch()` while the others are
>>>>>>>>> `KBranchedStream#branch()` what makes reading the code harder.
>>>>>>>>> 
>>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`, I though
>>>>>> it
>>>>>>>>> might be better to also rename `KStream#branch()` to avoid the
>>>>> naming
>>>>>>>>> overlap that seems to be confusing. The following reads much
>>>>> cleaner
>>>>>> to
>>>>>>>> me:
>>>>>>>>> stream.split()
>>>>>>>>>      .branch(Predicate)
>>>>>>>>>      .branch(Predicate)
>>>>>>>>>      .defaultBranch();
>>>>>>>>> 
>>>>>>>>> Maybe there is a better alternative to `split()` though to avoid
>>>>> the
>>>>>>>>> naming overlap.
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 'default' is, however, a reserved word, so unfortunately we
>>>>> cannot
>>>>>>> have
>>>>>>>>> a method with such name :-)
>>>>>>>>> 
>>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up with a
>>>>> short
>>>>>>>> name?
>>>>>>>>> 
>>>>>>>>> Can you add the interface `KBranchedStream` to the KIP with all
>>>>> it's
>>>>>>>>> methods? It will be part of public API and should be contained in
>>>>> the
>>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>>>>> `defaultBranch()` is.
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> You did not comment on the idea to add a `KBranchedStream#get(int
>>>>>>> index)
>>>>>>>>> -> KStream` method to get the individually branched-KStreams. Would
>>>>>> be
>>>>>>>>> nice to get your feedback about it. It seems you suggest that users
>>>>>>>>> would need to write custom utility code otherwise, to access them.
>>>>> We
>>>>>>>>> should discuss the pros and cons of both approaches. It feels
>>>>>>>>> "incomplete" to me atm, if the API has no built-in support to get
>>>>> the
>>>>>>>>> branched-KStreams directly.
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> -Matthias
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>>> Hi all!
>>>>>>>>>> 
>>>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>>>>>> 
>>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>> 
>>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>>> I can see your point: this is to make the name similar to
>>>>>>> String#split
>>>>>>>>>> that also returns an array, right? But is it worth the loss of
>>>>>>>> backwards
>>>>>>>>>> compatibility? We can have overloaded branch() as well without
>>>>>>>> affecting
>>>>>>>>>> the existing code. Maybe the old array-based `branch` method
>>>>> should
>>>>>>> be
>>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>>>>> 
>>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>>> BranchingKStream#branch(),
>>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>>>>> 
>>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default' is,
>>>>>>> however, a
>>>>>>>>>> reserved word, so unfortunately we cannot have a method with such
>>>>>>> name
>>>>>>>>> :-)
>>>>>>>>>>> defaultBranch() does take an `Predicate` as argument, but I
>>>>> think
>>>>>>> that
>>>>>>>>>> is not required?
>>>>>>>>>> 
>>>>>>>>>> Absolutely! I think that was just copy-paste error or something.
>>>>>>>>>> 
>>>>>>>>>> Dear colleagues,
>>>>>>>>>> 
>>>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>> 
>>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>> 
>>>>>>>>>> Regards,
>>>>>>>>>> 
>>>>>>>>>> Ivan
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems that
>>>>>>> everybody
>>>>>>>>>>> agrees that the current branch() method using arrays is not
>>>>>> optimal.
>>>>>>>>>>> I had a quick look into the PR and I like the overall proposal.
>>>>>>> There
>>>>>>>>>>> are some minor things we need to consider. I would recommend the
>>>>>>>>>>> following renaming:
>>>>>>>>>>> 
>>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>>>>>> 
>>>>>>>>>>> It's just a suggestion to get slightly shorter method names.
>>>>>>>>>>> 
>>>>>>>>>>> In the current PR, defaultBranch() does take an `Predicate` as
>>>>>>>> argument,
>>>>>>>>>>> but I think that is not required?
>>>>>>>>>>> 
>>>>>>>>>>> Also, we should consider KIP-307, that was recently accepted and
>>>>>> is
>>>>>>>>>>> currently implemented:
>>>>>>>>>>> 
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>>> Ie, we should add overloads that accepted a `Named` parameter.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> For the issue that the created `KStream` object are in different
>>>>>>>> scopes:
>>>>>>>>>>> could we extend `KBranchedStream` with a `get(int index)` method
>>>>>>> that
>>>>>>>>>>> returns the corresponding "branched" result `KStream` object?
>>>>>> Maybe,
>>>>>>>> the
>>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>>> `Consumer<KStream>`
>>>>>>>> but
>>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return whatever
>>>>>> the
>>>>>>>>>>> `Function` returns?
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> Finally, I would also suggest to update the KIP with the current
>>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> -Matthias
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>>> Ivan,
>>>>>>>>>>>> 
>>>>>>>>>>>> I'm a bit of a novice here as well, but I think it makes sense
>>>>>> for
>>>>>>>> you
>>>>>>>>> to
>>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously we'll
>>>>> need
>>>>>>>> some
>>>>>>>>>>>> buy-in from committers that have actual binding votes on
>>>>> whether
>>>>>>> the
>>>>>>>>> KIP
>>>>>>>>>>>> could be adopted.  It would be great to hear if they think this
>>>>>> is
>>>>>>> a
>>>>>>>>> good
>>>>>>>>>>>> idea overall.  I'm not sure if that happens just by starting a
>>>>>>> vote,
>>>>>>>>> or if
>>>>>>>>>>>> there is generally some indication of interest beforehand.
>>>>>>>>>>>> 
>>>>>>>>>>>> That being said, I'll continue the discussion a bit: assuming
>>>>> we
>>>>>> do
>>>>>>>>> move
>>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>>> KBranchedStream",
>>>>>>> do
>>>>>>>>> we
>>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I would
>>>>> favor
>>>>>>>>>>>> deprecating, since having two mutually exclusive APIs that
>>>>>>> accomplish
>>>>>>>>> the
>>>>>>>>>>>> same thing is confusing, especially when they're fairly similar
>>>>>>>>> anyway.  We
>>>>>>>>>>>> just need to be sure we're not making something
>>>>>>> impossible/difficult
>>>>>>>>> that
>>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>> 
>>>>>>>>>>>> Regarding my PR - I think the general structure would work,
>>>>> it's
>>>>>>>> just a
>>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>>>> particular,
>>>>>>>>>>>> passing in the "predicates" and "children" lists which get
>>>>>> modified
>>>>>>>> in
>>>>>>>>>>>> KBranchedStream but read from all the way KStreamLazyBranch is
>>>>> a
>>>>>>> bit
>>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Paul
>>>>>>>>>>>> 
>>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>>> iponomarev@mail.ru
>>>>>>>>> wrote:
>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I read your code carefully and now I am fully convinced: your
>>>>>>>> proposal
>>>>>>>>>>>>> looks better and should work. We just have to document the
>>>>>> crucial
>>>>>>>>> fact
>>>>>>>>>>>>> that KStream consumers are invoked as they're added. And then
>>>>>> it's
>>>>>>>> all
>>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> What shall we do now? I should re-write the KIP and resume the
>>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>>>> starting
>>>>>>>> point
>>>>>>>>> if
>>>>>>>>>>>>> we go in this direction'? To me it looks like a good starting
>>>>>>> point.
>>>>>>>>> But
>>>>>>>>>>>>> as a novice in this project I might miss some important
>>>>> details.
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>>> stream.branch()
>>>>>>>>> solution
>>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>>>> invoked
>>>>>> as
>>>>>>>>> they’re
>>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user still
>>>>>> ought
>>>>>>> to
>>>>>>>>> be
>>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and depend on
>>>>> the
>>>>>>>>> branched
>>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>>> The issue I mean to point out is that it is hard to access
>>>>> the
>>>>>>>>> branched
>>>>>>>>>>>>> streams in the same scope as the original stream (that is, not
>>>>>>>> inside
>>>>>>>>> the
>>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>>>> solutions.
>>>>>> It
>>>>>>>>> can be
>>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m excited
>>>>> to
>>>>>>>> hear
>>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>>> iponomarev@mail.ru
>>>>>>>>> wrote:
>>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>>>>>>>>> streamsBuilder.build() also looked great for me at first
>>>>> glance,
>>>>>>> but
>>>>>>>>> ---
>>>>>>>>>>>>>>>> the newly branched streams are not available in the same
>>>>>> scope
>>>>>>> as
>>>>>>>>> each
>>>>>>>>>>>>> other.  That is, if we wanted to merge them back together
>>>>> again
>>>>>> I
>>>>>>>>> don't see
>>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>>> You just took the words right out of my mouth, I was just
>>>>>> going
>>>>>>> to
>>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say we need
>>>>> to
>>>>>>>>> identify
>>>>>>>>>>>>> customers who have bought coffee and made a purchase in the
>>>>>>>>> electronics
>>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>>> This is the code I usually write under these circumstances
>>>>>> using
>>>>>>>> my
>>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>>>       return
>>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>>       /*In the real world the code here can be complex, so
>>>>>>>>> creation of
>>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in order to
>>>>>>>> separate
>>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>>  }
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>>     .branch(predicate1, couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>>     .branch(predicate2,
>>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up everything
>>>>>>>> later,
>>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Does this make sense?  In order to properly initialize the
>>>>>>>>> CouponIssuer
>>>>>>>>>>>>> we need the terminal operation to be called before
>>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>>> is called.
>>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is essentially
>>>>>> the
>>>>>>>>> next
>>>>>>>>>>>>> KIP I was going to write here. I have some thoughts based on
>>>>> my
>>>>>>>>> experience,
>>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a fluent
>>>>> API
>>>>>>>> based
>>>>>>>>>>>>> off of
>>>>>>>>>>>>>>>> KStream here (https://github.com/apache/kafka/pull/6512),
>>>>>> and
>>>>>>> I
>>>>>>>>> think
>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>>>>> compatibility
>>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware that Java
>>>>> is
>>>>>>>> smart
>>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>>    distinguish between a branch(varargs...) returning one
>>>>>>> thing
>>>>>>>>> and
>>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually need
>>>>> it.
>>>>>>> We
>>>>>>>>> can
>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who shares
>>>>>> its
>>>>>>>>> state
>>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the branching.
>>>>>>> It's
>>>>>>>>> not
>>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>>    pretty in its current form, but I think it demonstrates
>>>>>> its
>>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>>> To be clear, I don't think that pull request should be
>>>>> final
>>>>>> or
>>>>>>>>> even a
>>>>>>>>>>>>>>>> starting point if we go in this direction, I just wanted to
>>>>>> see
>>>>>>>> how
>>>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing solution
>>>>>>> could
>>>>>>>> be
>>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>>>> suggested
>>>>>>>> was a
>>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched streams
>>>>>> are
>>>>>>>> not
>>>>>>>>>>>>>>>> available in the same scope as each other.  That is, if we
>>>>>>> wanted
>>>>>>>>> to
>>>>>>>>>>>>> merge
>>>>>>>>>>>>>>>> them back together again I don't see a way to do that.  The
>>>>>> KIP
>>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>>> has the same issue, though - all this means is that for
>>>>>> either
>>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the table.
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>>>>> iponomarev@mail.ru>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to this
>>>>>> point.
>>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that branch API
>>>>>>> needs
>>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf returns
>>>>>> its
>>>>>>>>> argument
>>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't
>>>>> make
>>>>>>>> sense
>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
>>>>>>>>> contrasts the
>>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>>>>>> defaultBranch(..)
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface is
>>>>>> defined.
>>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>>>> (defaultBranch(ks->)
>>>>>>>>> and
>>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to miss the
>>>>>> fact
>>>>>>>>> that one
>>>>>>>>>>>>>>>>> of the terminal methods should be called. If these methods
>>>>>> are
>>>>>>>> not
>>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do better?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be implemented the
>>>>>>> easy
>>>>>>>>> way.
>>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> user could specify a terminal method that assumes
>>>>> nothing
>>>>>>>> will
>>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only option
>>>>>> besides
>>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we want to
>>>>>> just
>>>>>>>>> silently
>>>>>>>>>>>>>>>>>>> drop the messages that didn't match any predicate. 2)
>>>>>>> Throwing
>>>>>>>>> an
>>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing looks
>>>>>> like a
>>>>>>>> bad
>>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to emit a
>>>>>>>> special
>>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly where
>>>>>>> `default`
>>>>>>>>> can
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>> InternalTopologyBuilder
>>>>>>> to
>>>>>>>>> track
>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>>>>> error
>>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>>>> compiled
>>>>>>> and
>>>>>>>>> run?
>>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't compile if
>>>>> used
>>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a method chain
>>>>>>>> starting
>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference between
>>>>>>>> runtime
>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>>>> instantly
>>>>>> on
>>>>>>>>> unit
>>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a compilation
>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being required.
>>>>>>> But
>>>>>>>> is
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>>>> defaultBranch
>>>>>>>>> they
>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
>>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API - a
>>>>> user
>>>>>>>> could
>>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach the
>>>>>> default
>>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.  That
>>>>> seems
>>>>>>> like
>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>> improvement over the current branch() API, which allows
>>>>>> for
>>>>>>>> the
>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting dropped.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has to be
>>>>>> well
>>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>>> InternalTopologyBuilder
>>>>>>> to
>>>>>>>>> track
>>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>>>>> error
>>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is a
>>>>> "build
>>>>>>>> step"
>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>>> StreamsBuilder.build()
>>>>>>> is
>>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I agree
>>>>> that
>>>>>>> it's
>>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input stream.
>>>>>>> With
>>>>>>>>> the
>>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all other
>>>>>>> operations
>>>>>>>>> do -
>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> want to process off the original KStream multiple
>>>>> times,
>>>>>>> you
>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many operations
>>>>>> on
>>>>>>> it
>>>>>>>>> as
>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not always need
>>>>>> the
>>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal operation we
>>>>> don't
>>>>>>>> know
>>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we
>>>>> can
>>>>>> do
>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> I understand your point that the need of special
>>>>> object
>>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods. But
>>>>> here
>>>>>> we
>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the flow,
>>>>> so
>>>>>> I
>>>>>>>>> think
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this API, but I
>>>>>> find
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it contrasts the
>>>>>>> fluency
>>>>>>>>> of
>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call
>>>>> a
>>>>>>>>> method on
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch cases
>>>>> are
>>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very
>>>>>> nice
>>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around how we
>>>>>>> specify
>>>>>>>>> the
>>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1, this::handle1)
>>>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2, this::handle2)
>>>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>>>>> KStreamBrancher
>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and terminated by
>>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously incompatible with
>>>>> the
>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a different
>>>>>> name,
>>>>>>>> but
>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>>>> something
>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It
>>>>>> seems
>>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching while also
>>>>>>>> allowing
>>>>>>>>> you
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams
>>>>>> if
>>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1, this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2, this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher
>>>>> takes a
>>>>>>>>> Consumer
>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the example in
>>>>>> the
>>>>>>>> KIP
>>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would we
>>>>> handle
>>>>>>> the
>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to continue
>>>>>>>> processing
>>>>>>>>> and
>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the branched
>>>>>> stream
>>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if we had
>>>>>>>> something
>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
>>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP-
>>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418.
>>>>> Please
>>>>>>>> take
>>>>>>>>> a
>>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>> 
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hello everyone, thank you all for joining the discussion!

Well, I don't think the idea of named branches, be it a LinkedHashMap 
(no other Map will do, because order of definition matters) or `branch` 
method  taking name and Consumer has more advantages than drawbacks.

In my opinion, the only real positive outcome from Michael's proposal is 
that all the returned branches are in the same scope. But 1) we rarely 
need them in the same scope 2) there is a workaround for the scope 
problem, described in the KIP.

'Inlining the complex logic' is not a problem, because we can use method 
references instead of lambdas. In real world scenarios you tend to split 
the complex logic to methods anyway, so the code is going to be clean.

The drawbacks are strong. The cohesion between predicates and handlers 
is lost. We have to define predicates in one place, and handlers in 
another. This opens the door for bugs:

- what if we forget to define a handler for a name? or a name for a handler?
- what if we misspell a name?
- what if we copy-paste and duplicate a name?

What Michael propose would have been totally OK if we had been writing 
the API in Lua, Ruby or Python. In those languages the "dynamic naming" 
approach would have looked most concise and beautiful. But in Java we 
expect all the problems related to identifiers to be eliminated in 
compile time.

Do we have to invent duck-typing for the Java API?

And if we do, what advantage are we supposed to get besides having all 
the branches in the same scope? Michael, maybe I'm missing your point?

---

Earlier in this discussion John Roesler also proposed to do without 
"start branching" operator, and later Paul mentioned that in the case 
when we have to add a dynamic number of branches, the current KIP is 
'clumsier' compared to Michael's 'Map' solution. Let me address both 
comments here.

1) "Start branching" operator (I think that *split* is a good name for 
it indeed) is critical when we need to do a dynamic branching, see 
example below.

2) No, dynamic branching in current KIP is not clumsy at all. Imagine a 
real-world scenario when you need one branch per enum value (say, 
RecordType). You can have something like this:

/*John:if we had to start with stream.branch(...) here, it would have 
been much messier.*/
KBranchedStream branched = stream.split();

/*Not clumsy at all :-)*/
for (RecordType recordType : RecordType.values())
             branched = branched.branch((k, v) -> v.getRecType() == 
recordType,
                     recordType::processRecords);

Regards,

Ivan


02.05.2019 14:40, Matthias J. Sax пишет:
> I also agree with Michael's observation about the core problem of
> current `branch()` implementation.
>
> However, I also don't like to pass in a clumsy Map object. My thinking
> was more aligned with Paul's proposal to just add a name to each
> `branch()` statement and return a `Map<String,KStream>`.
>
> It makes the code easier to read, and also make the order of
> `Predicates` (that is essential) easier to grasp.
>
>>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>>     .branch("branchOne", Predicate<K, V>)
>>>>>     .branch( "branchTwo", Predicate<K, V>)
>>>>>     .defaultBranch("defaultBranch");
> An open question is the case for which no defaultBranch() should be
> specified. Atm, `split()` and `branch()` would return `BranchedKStream`
> and the call to `defaultBranch()` that returns the `Map` is mandatory
> (what is not the case atm). Or is this actually not a real problem,
> because users can just ignore the branch returned by `defaultBranch()`
> in the result `Map` ?
>
>
> About "inlining": So far, it seems to be a matter of personal
> preference. I can see arguments for both, but no "killer argument" yet
> that clearly make the case for one or the other.
>
>
> -Matthias
>
> On 5/1/19 6:26 PM, Paul Whalen wrote:
>> Perhaps inlining is the wrong terminology. It doesn’t require that a lambda with the full downstream topology be defined inline - it can be a method reference as with Ivan’s original suggestion.  The advantage of putting the predicate and its downstream logic (Consumer) together in branch() is that they are required to be near to each other.
>>
>> Ultimately the downstream code has to live somewhere, and deep branch trees will be hard to read regardless.
>>
>>> On May 1, 2019, at 1:07 PM, Michael Drogalis <mi...@confluent.io> wrote:
>>>
>>> I'm less enthusiastic about inlining the branch logic with its downstream
>>> functionality. Programs that have deep branch trees will quickly become
>>> harder to read as a single unit.
>>>
>>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen <pg...@gmail.com> wrote:
>>>>
>>>> Also +1 on the issues/goals as Michael outlined them, I think that sets a
>>>> great framework for the discussion.
>>>>
>>>> Regarding the SortedMap solution, my understanding is that the current
>>>> proposal in the KIP is what is in my PR which (pending naming decisions) is
>>>> roughly this:
>>>>
>>>> stream.split()
>>>>     .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>     .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>>     .defaultBranch(Consumer<KStream<K, V>>);
>>>>
>>>> Obviously some ordering is necessary, since branching as a construct
>>>> doesn't work without it, but this solution seems like it provides as much
>>>> associativity as the SortedMap solution, because each branch() call
>>>> directly associates the "conditional" with the "code block."  The value it
>>>> provides over the KIP solution is the accessing of streams in the same
>>>> scope.
>>>>
>>>> The KIP solution is less "dynamic" than the SortedMap solution in the sense
>>>> that it is slightly clumsier to add a dynamic number of branches, but it is
>>>> certainly possible.  It seems to me like the API should favor the "static"
>>>> case anyway, and should make it simple and readable to fluently declare and
>>>> access your branches in-line.  It also makes it impossible to ignore a
>>>> branch, and it is possible to build an (almost) identical SortedMap
>>>> solution on top of it.
>>>>
>>>> I could also see a middle ground where instead of a raw SortedMap being
>>>> taken in, branch() takes a name and not a Consumer.  Something like this:
>>>>
>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>     .branch("branchOne", Predicate<K, V>)
>>>>     .branch( "branchTwo", Predicate<K, V>)
>>>>     .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>>
>>>> Pros for that solution:
>>>> - accessing branched KStreams in same scope
>>>> - no double brace initialization, hopefully slightly more readable than
>>>> SortedMap
>>>>
>>>> Cons
>>>> - downstream branch logic cannot be specified inline which makes it harder
>>>> to read top to bottom (like existing API and SortedMap, but unlike the KIP)
>>>> - you can forget to "handle" one of the branched streams (like existing
>>>> API and SortedMap, but unlike the KIP)
>>>>
>>>> (KBranchedStreams could even work *both* ways but perhaps that's overdoing
>>>> it).
>>>>
>>>> Overall I'm curious how important it is to be able to easily access the
>>>> branched KStream in the same scope as the original.  It's possible that it
>>>> doesn't need to be handled directly by the API, but instead left up to the
>>>> user.  I'm sort of in the middle on it.
>>>>
>>>> Paul
>>>>
>>>>
>>>>
>>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman <so...@confluent.io>
>>>> wrote:
>>>>
>>>>> I'd like to +1 what Michael said about the issues with the existing
>>>> branch
>>>>> method, I agree with what he's outlined and I think we should proceed by
>>>>> trying to alleviate these problems. Specifically it seems important to be
>>>>> able to cleanly access the individual branches (eg by mapping
>>>>> name->stream), which I thought was the original intention of this KIP.
>>>>>
>>>>> That said, I don't think we should so easily give in to the double brace
>>>>> anti-pattern or force ours users into it if at all possible to
>>>> avoid...just
>>>>> my two cents.
>>>>>
>>>>> Cheers,
>>>>> Sophie
>>>>>
>>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>>> michael.drogalis@confluent.io> wrote:
>>>>>
>>>>>> I’d like to propose a different way of thinking about this. To me,
>>>> there
>>>>>> are three problems with the existing branch signature:
>>>>>>
>>>>>> 1. If you use it the way most people do, Java raises unsafe type
>>>>> warnings.
>>>>>> 2. The way in which you use the stream branches is positionally coupled
>>>>> to
>>>>>> the ordering of the conditionals.
>>>>>> 3. It is brittle to extend existing branch calls with additional code
>>>>>> paths.
>>>>>>
>>>>>> Using associative constructs instead of relying on ordered constructs
>>>>> would
>>>>>> be a stronger approach. Consider a signature that instead looks like
>>>>> this:
>>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
>>>>>> super K,? super V>>);
>>>>>>
>>>>>> Branches are given names in a map, and as a result, the API returns a
>>>>>> mapping of names to streams. The ordering of the conditionals is
>>>>> maintained
>>>>>> because it’s a sorted map. Insert order determines the order of
>>>>> evaluation.
>>>>>> This solves problem 1 because there are no more varargs. It solves
>>>>> problem
>>>>>> 2 because you no longer lean on ordering to access the branch you’re
>>>>>> interested in. It solves problem 3 because you can introduce another
>>>>>> conditional by simply attaching another name to the structure, rather
>>>>> than
>>>>>> messing with the existing indices.
>>>>>>
>>>>>> One of the drawbacks is that creating the map inline is historically
>>>>>> awkward in Java. I know it’s an anti-pattern to use voluminously, but
>>>>>> double brace initialization would clean up the aesthetics.
>>>>>>
>>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io>
>>>> wrote:
>>>>>>> Hi Ivan,
>>>>>>>
>>>>>>> Thanks for the update.
>>>>>>>
>>>>>>> FWIW, I agree with Matthias that the current "start branching"
>>>> operator
>>>>>> is
>>>>>>> confusing when named the same way as the actual branches. "Split"
>>>> seems
>>>>>>> like a good name. Alternatively, we can do without a "start
>>>> branching"
>>>>>>> operator at all, and just do:
>>>>>>>
>>>>>>> stream
>>>>>>>       .branch(Predicate)
>>>>>>>       .branch(Predicate)
>>>>>>>       .defaultBranch();
>>>>>>>
>>>>>>> Tentatively, I think that this branching operation should be
>>>> terminal.
>>>>>> That
>>>>>>> way, we don't create ambiguity about how to use it. That is, `branch`
>>>>>>> should return `KBranchedStream`, while `defaultBranch` is `void`, to
>>>>>>> enforce that it comes last, and that there is only one definition of
>>>>> the
>>>>>>> default branch. Potentially, we should log a warning if there's no
>>>>>> default,
>>>>>>> and additionally log a warning (or throw an exception) if a record
>>>>> falls
>>>>>>> though with no default.
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>> Thanks,
>>>>>>> -John
>>>>>>>
>>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>>> matthias@confluent.io
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>>
>>>>>>>>
>>>>>>>>> this is to make the name similar to String#split
>>>>>>>>>> that also returns an array, right?
>>>>>>>> The intend was to avoid name duplication. The return type should
>>>>> _not_
>>>>>>>> be an array.
>>>>>>>>
>>>>>>>> The current proposal is
>>>>>>>>
>>>>>>>> stream.branch()
>>>>>>>>       .branch(Predicate)
>>>>>>>>       .branch(Predicate)
>>>>>>>>       .defaultBranch();
>>>>>>>>
>>>>>>>> IMHO, this reads a little odd, because the first `branch()` does
>>>> not
>>>>>>>> take any parameters and has different semantics than the later
>>>>>>>> `branch()` calls. Note, that from the code snippet above, it's
>>>> hidden
>>>>>>>> that the first call is `KStream#branch()` while the others are
>>>>>>>> `KBranchedStream#branch()` what makes reading the code harder.
>>>>>>>>
>>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`, I though
>>>>> it
>>>>>>>> might be better to also rename `KStream#branch()` to avoid the
>>>> naming
>>>>>>>> overlap that seems to be confusing. The following reads much
>>>> cleaner
>>>>> to
>>>>>>> me:
>>>>>>>> stream.split()
>>>>>>>>       .branch(Predicate)
>>>>>>>>       .branch(Predicate)
>>>>>>>>       .defaultBranch();
>>>>>>>>
>>>>>>>> Maybe there is a better alternative to `split()` though to avoid
>>>> the
>>>>>>>> naming overlap.
>>>>>>>>
>>>>>>>>
>>>>>>>>> 'default' is, however, a reserved word, so unfortunately we
>>>> cannot
>>>>>> have
>>>>>>>> a method with such name :-)
>>>>>>>>
>>>>>>>> Bummer. Didn't consider this. Maybe we can still come up with a
>>>> short
>>>>>>> name?
>>>>>>>>
>>>>>>>> Can you add the interface `KBranchedStream` to the KIP with all
>>>> it's
>>>>>>>> methods? It will be part of public API and should be contained in
>>>> the
>>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>>>> `defaultBranch()` is.
>>>>>>>>
>>>>>>>>
>>>>>>>> You did not comment on the idea to add a `KBranchedStream#get(int
>>>>>> index)
>>>>>>>> -> KStream` method to get the individually branched-KStreams. Would
>>>>> be
>>>>>>>> nice to get your feedback about it. It seems you suggest that users
>>>>>>>> would need to write custom utility code otherwise, to access them.
>>>> We
>>>>>>>> should discuss the pros and cons of both approaches. It feels
>>>>>>>> "incomplete" to me atm, if the API has no built-in support to get
>>>> the
>>>>>>>> branched-KStreams directly.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>>> Hi all!
>>>>>>>>>
>>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>>>>>
>>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>>
>>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>> I can see your point: this is to make the name similar to
>>>>>> String#split
>>>>>>>>> that also returns an array, right? But is it worth the loss of
>>>>>>> backwards
>>>>>>>>> compatibility? We can have overloaded branch() as well without
>>>>>>> affecting
>>>>>>>>> the existing code. Maybe the old array-based `branch` method
>>>> should
>>>>>> be
>>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>>>>
>>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>>> BranchingKStream#branch(),
>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>>>>
>>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default' is,
>>>>>> however, a
>>>>>>>>> reserved word, so unfortunately we cannot have a method with such
>>>>>> name
>>>>>>>> :-)
>>>>>>>>>> defaultBranch() does take an `Predicate` as argument, but I
>>>> think
>>>>>> that
>>>>>>>>> is not required?
>>>>>>>>>
>>>>>>>>> Absolutely! I think that was just copy-paste error or something.
>>>>>>>>>
>>>>>>>>> Dear colleagues,
>>>>>>>>>
>>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>>
>>>>>>>>> Any new suggestions/objections?
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>>> Thanks for driving the discussion of this KIP. It seems that
>>>>>> everybody
>>>>>>>>>> agrees that the current branch() method using arrays is not
>>>>> optimal.
>>>>>>>>>> I had a quick look into the PR and I like the overall proposal.
>>>>>> There
>>>>>>>>>> are some minor things we need to consider. I would recommend the
>>>>>>>>>> following renaming:
>>>>>>>>>>
>>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>>>>>
>>>>>>>>>> It's just a suggestion to get slightly shorter method names.
>>>>>>>>>>
>>>>>>>>>> In the current PR, defaultBranch() does take an `Predicate` as
>>>>>>> argument,
>>>>>>>>>> but I think that is not required?
>>>>>>>>>>
>>>>>>>>>> Also, we should consider KIP-307, that was recently accepted and
>>>>> is
>>>>>>>>>> currently implemented:
>>>>>>>>>>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>> Ie, we should add overloads that accepted a `Named` parameter.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> For the issue that the created `KStream` object are in different
>>>>>>> scopes:
>>>>>>>>>> could we extend `KBranchedStream` with a `get(int index)` method
>>>>>> that
>>>>>>>>>> returns the corresponding "branched" result `KStream` object?
>>>>> Maybe,
>>>>>>> the
>>>>>>>>>> second argument of `addBranch()` should not be a
>>>>> `Consumer<KStream>`
>>>>>>> but
>>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return whatever
>>>>> the
>>>>>>>>>> `Function` returns?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Finally, I would also suggest to update the KIP with the current
>>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>>> Ivan,
>>>>>>>>>>>
>>>>>>>>>>> I'm a bit of a novice here as well, but I think it makes sense
>>>>> for
>>>>>>> you
>>>>>>>> to
>>>>>>>>>>> revise the KIP and continue the discussion.  Obviously we'll
>>>> need
>>>>>>> some
>>>>>>>>>>> buy-in from committers that have actual binding votes on
>>>> whether
>>>>>> the
>>>>>>>> KIP
>>>>>>>>>>> could be adopted.  It would be great to hear if they think this
>>>>> is
>>>>>> a
>>>>>>>> good
>>>>>>>>>>> idea overall.  I'm not sure if that happens just by starting a
>>>>>> vote,
>>>>>>>> or if
>>>>>>>>>>> there is generally some indication of interest beforehand.
>>>>>>>>>>>
>>>>>>>>>>> That being said, I'll continue the discussion a bit: assuming
>>>> we
>>>>> do
>>>>>>>> move
>>>>>>>>>>> forward the solution of "stream.branch() returns
>>>>> KBranchedStream",
>>>>>> do
>>>>>>>> we
>>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I would
>>>> favor
>>>>>>>>>>> deprecating, since having two mutually exclusive APIs that
>>>>>> accomplish
>>>>>>>> the
>>>>>>>>>>> same thing is confusing, especially when they're fairly similar
>>>>>>>> anyway.  We
>>>>>>>>>>> just need to be sure we're not making something
>>>>>> impossible/difficult
>>>>>>>> that
>>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>>
>>>>>>>>>>> Regarding my PR - I think the general structure would work,
>>>> it's
>>>>>>> just a
>>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>>> particular,
>>>>>>>>>>> passing in the "predicates" and "children" lists which get
>>>>> modified
>>>>>>> in
>>>>>>>>>>> KBranchedStream but read from all the way KStreamLazyBranch is
>>>> a
>>>>>> bit
>>>>>>>>>>> complicated to follow.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Paul
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>>> iponomarev@mail.ru
>>>>>>>> wrote:
>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>
>>>>>>>>>>>> I read your code carefully and now I am fully convinced: your
>>>>>>> proposal
>>>>>>>>>>>> looks better and should work. We just have to document the
>>>>> crucial
>>>>>>>> fact
>>>>>>>>>>>> that KStream consumers are invoked as they're added. And then
>>>>> it's
>>>>>>> all
>>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>>
>>>>>>>>>>>> What shall we do now? I should re-write the KIP and resume the
>>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>>
>>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>>> starting
>>>>>>> point
>>>>>>>> if
>>>>>>>>>>>> we go in this direction'? To me it looks like a good starting
>>>>>> point.
>>>>>>>> But
>>>>>>>>>>>> as a novice in this project I might miss some important
>>>> details.
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Ivan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>>> stream.branch()
>>>>>>>> solution
>>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>>> invoked
>>>>> as
>>>>>>>> they’re
>>>>>>>>>>>> added, not during streamsBuilder.build(). So the user still
>>>>> ought
>>>>>> to
>>>>>>>> be
>>>>>>>>>>>> able to call couponIssuer.coupons() afterward and depend on
>>>> the
>>>>>>>> branched
>>>>>>>>>>>> streams having been set.
>>>>>>>>>>>>> The issue I mean to point out is that it is hard to access
>>>> the
>>>>>>>> branched
>>>>>>>>>>>> streams in the same scope as the original stream (that is, not
>>>>>>> inside
>>>>>>>> the
>>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>>> solutions.
>>>>> It
>>>>>>>> can be
>>>>>>>>>>>> worked around though.
>>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m excited
>>>> to
>>>>>>> hear
>>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>>> iponomarev@mail.ru
>>>>>>>> wrote:
>>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>>>>>>>> streamsBuilder.build() also looked great for me at first
>>>> glance,
>>>>>> but
>>>>>>>> ---
>>>>>>>>>>>>>>> the newly branched streams are not available in the same
>>>>> scope
>>>>>> as
>>>>>>>> each
>>>>>>>>>>>> other.  That is, if we wanted to merge them back together
>>>> again
>>>>> I
>>>>>>>> don't see
>>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>>> You just took the words right out of my mouth, I was just
>>>>> going
>>>>>> to
>>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say we need
>>>> to
>>>>>>>> identify
>>>>>>>>>>>> customers who have bought coffee and made a purchase in the
>>>>>>>> electronics
>>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>>> This is the code I usually write under these circumstances
>>>>> using
>>>>>>> my
>>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>>    private KStream<....> coffePurchases;
>>>>>>>>>>>>>>    private KStream<....> electronicsPurchases;
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>    KStream<...> coupons(){
>>>>>>>>>>>>>>        return
>>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>>        /*In the real world the code here can be complex, so
>>>>>>>> creation of
>>>>>>>>>>>> a separate CouponIssuer class is fully justified, in order to
>>>>>>> separate
>>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>>   }
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>>      .branch(predicate2,
>>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>>      .onTopOf(transactionStream);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up everything
>>>>>>> later,
>>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Does this make sense?  In order to properly initialize the
>>>>>>>> CouponIssuer
>>>>>>>>>>>> we need the terminal operation to be called before
>>>>>>>> streamsBuilder.build()
>>>>>>>>>>>> is called.
>>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is essentially
>>>>> the
>>>>>>>> next
>>>>>>>>>>>> KIP I was going to write here. I have some thoughts based on
>>>> my
>>>>>>>> experience,
>>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a fluent
>>>> API
>>>>>>> based
>>>>>>>>>>>> off of
>>>>>>>>>>>>>>> KStream here (https://github.com/apache/kafka/pull/6512),
>>>>> and
>>>>>> I
>>>>>>>> think
>>>>>>>>>>>> I
>>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>>     - Compatibility: I was incorrect earlier about
>>>>>> compatibility
>>>>>>>>>>>> issues,
>>>>>>>>>>>>>>>     there aren't any direct ones.  I was unaware that Java
>>>> is
>>>>>>> smart
>>>>>>>>>>>> enough to
>>>>>>>>>>>>>>>     distinguish between a branch(varargs...) returning one
>>>>>> thing
>>>>>>>> and
>>>>>>>>>>>> branch()
>>>>>>>>>>>>>>>     with no arguments returning another thing.
>>>>>>>>>>>>>>>     - Requiring a terminal method: We don't actually need
>>>> it.
>>>>>> We
>>>>>>>> can
>>>>>>>>>>>> just
>>>>>>>>>>>>>>>     build up the branches in the KBranchedStream who shares
>>>>> its
>>>>>>>> state
>>>>>>>>>>>> with the
>>>>>>>>>>>>>>>     ProcessorSupplier that will actually do the branching.
>>>>>> It's
>>>>>>>> not
>>>>>>>>>>>> terribly
>>>>>>>>>>>>>>>     pretty in its current form, but I think it demonstrates
>>>>> its
>>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>>> To be clear, I don't think that pull request should be
>>>> final
>>>>> or
>>>>>>>> even a
>>>>>>>>>>>>>>> starting point if we go in this direction, I just wanted to
>>>>> see
>>>>>>> how
>>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>>>>>>>>>>> I will say though, that I'm not sure the existing solution
>>>>>> could
>>>>>>> be
>>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>>> suggested
>>>>>>> was a
>>>>>>>>>>>>>>> possibility.  The reason is that the newly branched streams
>>>>> are
>>>>>>> not
>>>>>>>>>>>>>>> available in the same scope as each other.  That is, if we
>>>>>> wanted
>>>>>>>> to
>>>>>>>>>>>> merge
>>>>>>>>>>>>>>> them back together again I don't see a way to do that.  The
>>>>> KIP
>>>>>>>>>>>> proposal
>>>>>>>>>>>>>>> has the same issue, though - all this means is that for
>>>>> either
>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the table.
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>>>> iponomarev@mail.ru>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to this
>>>>> point.
>>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that branch API
>>>>>> needs
>>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>>     .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>>     .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>>     .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns
>>>>> its
>>>>>>>> argument
>>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't
>>>> make
>>>>>>> sense
>>>>>>>>>>>> until
>>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
>>>>>>>> contrasts the
>>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>     .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>>     .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both
>>>>>>>> defaultBranch(..)
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface is
>>>>> defined.
>>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>>> (defaultBranch(ks->)
>>>>>>>> and
>>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to miss the
>>>>> fact
>>>>>>>> that one
>>>>>>>>>>>>>>>> of the terminal methods should be called. If these methods
>>>>> are
>>>>>>> not
>>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do better?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be implemented the
>>>>>> easy
>>>>>>>> way.
>>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> user could specify a terminal method that assumes
>>>> nothing
>>>>>>> will
>>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only option
>>>>> besides
>>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we want to
>>>>> just
>>>>>>>> silently
>>>>>>>>>>>>>>>>>> drop the messages that didn't match any predicate. 2)
>>>>>> Throwing
>>>>>>>> an
>>>>>>>>>>>>>>>>>> exception in the middle of data flow processing looks
>>>>> like a
>>>>>>> bad
>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to emit a
>>>>>>> special
>>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly where
>>>>>> `default`
>>>>>>>> can
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>> InternalTopologyBuilder
>>>>>> to
>>>>>>>> track
>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>>>> error
>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>>> compiled
>>>>>> and
>>>>>>>> run?
>>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't compile if
>>>> used
>>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a method chain
>>>>>>> starting
>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference between
>>>>>>> runtime
>>>>>>>> and
>>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>>> instantly
>>>>> on
>>>>>>>> unit
>>>>>>>>>>>>>>>>>> tests, it costs more for the project than a compilation
>>>>>>> failure.
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Good point about the terminal operation being required.
>>>>>> But
>>>>>>> is
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>>> defaultBranch
>>>>>>>> they
>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
>>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API - a
>>>> user
>>>>>>> could
>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach the
>>>>> default
>>>>>>>> branch,
>>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.  That
>>>> seems
>>>>>> like
>>>>>>>> an
>>>>>>>>>>>>>>>>>>> improvement over the current branch() API, which allows
>>>>> for
>>>>>>> the
>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting dropped.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has to be
>>>>> well
>>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>>> InternalTopologyBuilder
>>>>>> to
>>>>>>>> track
>>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>>>> error
>>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is a
>>>> "build
>>>>>>> step"
>>>>>>>>>>>> where
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>>> StreamsBuilder.build()
>>>>>> is
>>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I agree
>>>> that
>>>>>> it's
>>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>>> allow users to do other operations on the input stream.
>>>>>> With
>>>>>>>> the
>>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all other
>>>>>> operations
>>>>>>>> do -
>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> want to process off the original KStream multiple
>>>> times,
>>>>>> you
>>>>>>>> just
>>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many operations
>>>>> on
>>>>>> it
>>>>>>>> as
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not always need
>>>>> the
>>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal operation we
>>>> don't
>>>>>>> know
>>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we
>>>> can
>>>>> do
>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I understand your point that the need of special
>>>> object
>>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods. But
>>>> here
>>>>> we
>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the flow,
>>>> so
>>>>> I
>>>>>>>> think
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this API, but I
>>>>> find
>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it contrasts the
>>>>>> fluency
>>>>>>>> of
>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call
>>>> a
>>>>>>>> method on
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch cases
>>>> are
>>>>>>>> defined
>>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very
>>>>> nice
>>>>>>>> and the
>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around how we
>>>>>> specify
>>>>>>>> the
>>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
>>>>>>>>>>>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
>>>>>>>>>>>>>>>>>>>>>            .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>>>> KStreamBrancher
>>>>>>>> or
>>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and terminated by
>>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously incompatible with
>>>> the
>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a different
>>>>> name,
>>>>>>> but
>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>>> something
>>>>>>> like
>>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It
>>>>> seems
>>>>>>>> like it
>>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching while also
>>>>>>> allowing
>>>>>>>> you
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams
>>>>> if
>>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>>>            ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>>>>>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>>       .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher
>>>> takes a
>>>>>>>> Consumer
>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the example in
>>>>> the
>>>>>>> KIP
>>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would we
>>>> handle
>>>>>> the
>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to continue
>>>>>>> processing
>>>>>>>> and
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the branched
>>>>> stream
>>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if we had
>>>>>>> something
>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
>>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP-
>>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418.
>>>> Please
>>>>>>> take
>>>>>>>> a
>>>>>>>>>>>> look
>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I also agree with Michael's observation about the core problem of
current `branch()` implementation.

However, I also don't like to pass in a clumsy Map object. My thinking
was more aligned with Paul's proposal to just add a name to each
`branch()` statement and return a `Map<String,KStream>`.

It makes the code easier to read, and also make the order of
`Predicates` (that is essential) easier to grasp.

>>>> Map<String, KStream<K, V>> branches = stream.split()
>>>>    .branch("branchOne", Predicate<K, V>)
>>>>    .branch( "branchTwo", Predicate<K, V>)
>>>>    .defaultBranch("defaultBranch");

An open question is the case for which no defaultBranch() should be
specified. Atm, `split()` and `branch()` would return `BranchedKStream`
and the call to `defaultBranch()` that returns the `Map` is mandatory
(what is not the case atm). Or is this actually not a real problem,
because users can just ignore the branch returned by `defaultBranch()`
in the result `Map` ?


About "inlining": So far, it seems to be a matter of personal
preference. I can see arguments for both, but no "killer argument" yet
that clearly make the case for one or the other.


-Matthias

On 5/1/19 6:26 PM, Paul Whalen wrote:
> Perhaps inlining is the wrong terminology. It doesn’t require that a lambda with the full downstream topology be defined inline - it can be a method reference as with Ivan’s original suggestion.  The advantage of putting the predicate and its downstream logic (Consumer) together in branch() is that they are required to be near to each other. 
> 
> Ultimately the downstream code has to live somewhere, and deep branch trees will be hard to read regardless.
> 
>> On May 1, 2019, at 1:07 PM, Michael Drogalis <mi...@confluent.io> wrote:
>>
>> I'm less enthusiastic about inlining the branch logic with its downstream
>> functionality. Programs that have deep branch trees will quickly become
>> harder to read as a single unit.
>>
>>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen <pg...@gmail.com> wrote:
>>>
>>> Also +1 on the issues/goals as Michael outlined them, I think that sets a
>>> great framework for the discussion.
>>>
>>> Regarding the SortedMap solution, my understanding is that the current
>>> proposal in the KIP is what is in my PR which (pending naming decisions) is
>>> roughly this:
>>>
>>> stream.split()
>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>>    .defaultBranch(Consumer<KStream<K, V>>);
>>>
>>> Obviously some ordering is necessary, since branching as a construct
>>> doesn't work without it, but this solution seems like it provides as much
>>> associativity as the SortedMap solution, because each branch() call
>>> directly associates the "conditional" with the "code block."  The value it
>>> provides over the KIP solution is the accessing of streams in the same
>>> scope.
>>>
>>> The KIP solution is less "dynamic" than the SortedMap solution in the sense
>>> that it is slightly clumsier to add a dynamic number of branches, but it is
>>> certainly possible.  It seems to me like the API should favor the "static"
>>> case anyway, and should make it simple and readable to fluently declare and
>>> access your branches in-line.  It also makes it impossible to ignore a
>>> branch, and it is possible to build an (almost) identical SortedMap
>>> solution on top of it.
>>>
>>> I could also see a middle ground where instead of a raw SortedMap being
>>> taken in, branch() takes a name and not a Consumer.  Something like this:
>>>
>>> Map<String, KStream<K, V>> branches = stream.split()
>>>    .branch("branchOne", Predicate<K, V>)
>>>    .branch( "branchTwo", Predicate<K, V>)
>>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>>>
>>> Pros for that solution:
>>> - accessing branched KStreams in same scope
>>> - no double brace initialization, hopefully slightly more readable than
>>> SortedMap
>>>
>>> Cons
>>> - downstream branch logic cannot be specified inline which makes it harder
>>> to read top to bottom (like existing API and SortedMap, but unlike the KIP)
>>> - you can forget to "handle" one of the branched streams (like existing
>>> API and SortedMap, but unlike the KIP)
>>>
>>> (KBranchedStreams could even work *both* ways but perhaps that's overdoing
>>> it).
>>>
>>> Overall I'm curious how important it is to be able to easily access the
>>> branched KStream in the same scope as the original.  It's possible that it
>>> doesn't need to be handled directly by the API, but instead left up to the
>>> user.  I'm sort of in the middle on it.
>>>
>>> Paul
>>>
>>>
>>>
>>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman <so...@confluent.io>
>>> wrote:
>>>
>>>> I'd like to +1 what Michael said about the issues with the existing
>>> branch
>>>> method, I agree with what he's outlined and I think we should proceed by
>>>> trying to alleviate these problems. Specifically it seems important to be
>>>> able to cleanly access the individual branches (eg by mapping
>>>> name->stream), which I thought was the original intention of this KIP.
>>>>
>>>> That said, I don't think we should so easily give in to the double brace
>>>> anti-pattern or force ours users into it if at all possible to
>>> avoid...just
>>>> my two cents.
>>>>
>>>> Cheers,
>>>> Sophie
>>>>
>>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>>> michael.drogalis@confluent.io> wrote:
>>>>
>>>>> I’d like to propose a different way of thinking about this. To me,
>>> there
>>>>> are three problems with the existing branch signature:
>>>>>
>>>>> 1. If you use it the way most people do, Java raises unsafe type
>>>> warnings.
>>>>> 2. The way in which you use the stream branches is positionally coupled
>>>> to
>>>>> the ordering of the conditionals.
>>>>> 3. It is brittle to extend existing branch calls with additional code
>>>>> paths.
>>>>>
>>>>> Using associative constructs instead of relying on ordered constructs
>>>> would
>>>>> be a stronger approach. Consider a signature that instead looks like
>>>> this:
>>>>>
>>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
>>>>> super K,? super V>>);
>>>>>
>>>>> Branches are given names in a map, and as a result, the API returns a
>>>>> mapping of names to streams. The ordering of the conditionals is
>>>> maintained
>>>>> because it’s a sorted map. Insert order determines the order of
>>>> evaluation.
>>>>>
>>>>> This solves problem 1 because there are no more varargs. It solves
>>>> problem
>>>>> 2 because you no longer lean on ordering to access the branch you’re
>>>>> interested in. It solves problem 3 because you can introduce another
>>>>> conditional by simply attaching another name to the structure, rather
>>>> than
>>>>> messing with the existing indices.
>>>>>
>>>>> One of the drawbacks is that creating the map inline is historically
>>>>> awkward in Java. I know it’s an anti-pattern to use voluminously, but
>>>>> double brace initialization would clean up the aesthetics.
>>>>>
>>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io>
>>> wrote:
>>>>>
>>>>>> Hi Ivan,
>>>>>>
>>>>>> Thanks for the update.
>>>>>>
>>>>>> FWIW, I agree with Matthias that the current "start branching"
>>> operator
>>>>> is
>>>>>> confusing when named the same way as the actual branches. "Split"
>>> seems
>>>>>> like a good name. Alternatively, we can do without a "start
>>> branching"
>>>>>> operator at all, and just do:
>>>>>>
>>>>>> stream
>>>>>>      .branch(Predicate)
>>>>>>      .branch(Predicate)
>>>>>>      .defaultBranch();
>>>>>>
>>>>>> Tentatively, I think that this branching operation should be
>>> terminal.
>>>>> That
>>>>>> way, we don't create ambiguity about how to use it. That is, `branch`
>>>>>> should return `KBranchedStream`, while `defaultBranch` is `void`, to
>>>>>> enforce that it comes last, and that there is only one definition of
>>>> the
>>>>>> default branch. Potentially, we should log a warning if there's no
>>>>> default,
>>>>>> and additionally log a warning (or throw an exception) if a record
>>>> falls
>>>>>> though with no default.
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>> Thanks,
>>>>>> -John
>>>>>>
>>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>>> matthias@confluent.io
>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>> Thanks for updating the KIP and your answers.
>>>>>>>
>>>>>>>
>>>>>>>> this is to make the name similar to String#split
>>>>>>>>> that also returns an array, right?
>>>>>>>
>>>>>>> The intend was to avoid name duplication. The return type should
>>>> _not_
>>>>>>> be an array.
>>>>>>>
>>>>>>> The current proposal is
>>>>>>>
>>>>>>> stream.branch()
>>>>>>>      .branch(Predicate)
>>>>>>>      .branch(Predicate)
>>>>>>>      .defaultBranch();
>>>>>>>
>>>>>>> IMHO, this reads a little odd, because the first `branch()` does
>>> not
>>>>>>> take any parameters and has different semantics than the later
>>>>>>> `branch()` calls. Note, that from the code snippet above, it's
>>> hidden
>>>>>>> that the first call is `KStream#branch()` while the others are
>>>>>>> `KBranchedStream#branch()` what makes reading the code harder.
>>>>>>>
>>>>>>> Because I suggested to rename `addBranch()` -> `branch()`, I though
>>>> it
>>>>>>> might be better to also rename `KStream#branch()` to avoid the
>>> naming
>>>>>>> overlap that seems to be confusing. The following reads much
>>> cleaner
>>>> to
>>>>>> me:
>>>>>>>
>>>>>>> stream.split()
>>>>>>>      .branch(Predicate)
>>>>>>>      .branch(Predicate)
>>>>>>>      .defaultBranch();
>>>>>>>
>>>>>>> Maybe there is a better alternative to `split()` though to avoid
>>> the
>>>>>>> naming overlap.
>>>>>>>
>>>>>>>
>>>>>>>> 'default' is, however, a reserved word, so unfortunately we
>>> cannot
>>>>> have
>>>>>>> a method with such name :-)
>>>>>>>
>>>>>>> Bummer. Didn't consider this. Maybe we can still come up with a
>>> short
>>>>>> name?
>>>>>>>
>>>>>>>
>>>>>>> Can you add the interface `KBranchedStream` to the KIP with all
>>> it's
>>>>>>> methods? It will be part of public API and should be contained in
>>> the
>>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>>> `defaultBranch()` is.
>>>>>>>
>>>>>>>
>>>>>>> You did not comment on the idea to add a `KBranchedStream#get(int
>>>>> index)
>>>>>>> -> KStream` method to get the individually branched-KStreams. Would
>>>> be
>>>>>>> nice to get your feedback about it. It seems you suggest that users
>>>>>>> would need to write custom utility code otherwise, to access them.
>>> We
>>>>>>> should discuss the pros and cons of both approaches. It feels
>>>>>>> "incomplete" to me atm, if the API has no built-in support to get
>>> the
>>>>>>> branched-KStreams directly.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>>> Hi all!
>>>>>>>>
>>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>>>>
>>>>>>>> Matthias, thanks for your comment!
>>>>>>>>
>>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>>>
>>>>>>>> I can see your point: this is to make the name similar to
>>>>> String#split
>>>>>>>> that also returns an array, right? But is it worth the loss of
>>>>>> backwards
>>>>>>>> compatibility? We can have overloaded branch() as well without
>>>>>> affecting
>>>>>>>> the existing code. Maybe the old array-based `branch` method
>>> should
>>>>> be
>>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>>>
>>>>>>>>> Renaming KBranchedStream#addBranch() ->
>>> BranchingKStream#branch(),
>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>>>
>>>>>>>> Totally agree with 'addBranch->branch' rename. 'default' is,
>>>>> however, a
>>>>>>>> reserved word, so unfortunately we cannot have a method with such
>>>>> name
>>>>>>> :-)
>>>>>>>>
>>>>>>>>> defaultBranch() does take an `Predicate` as argument, but I
>>> think
>>>>> that
>>>>>>>> is not required?
>>>>>>>>
>>>>>>>> Absolutely! I think that was just copy-paste error or something.
>>>>>>>>
>>>>>>>> Dear colleagues,
>>>>>>>>
>>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>>>
>>>>>>>> Any new suggestions/objections?
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>>
>>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>>> Thanks for driving the discussion of this KIP. It seems that
>>>>> everybody
>>>>>>>>> agrees that the current branch() method using arrays is not
>>>> optimal.
>>>>>>>>>
>>>>>>>>> I had a quick look into the PR and I like the overall proposal.
>>>>> There
>>>>>>>>> are some minor things we need to consider. I would recommend the
>>>>>>>>> following renaming:
>>>>>>>>>
>>>>>>>>> KStream#branch() -> #split()
>>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>>>>
>>>>>>>>> It's just a suggestion to get slightly shorter method names.
>>>>>>>>>
>>>>>>>>> In the current PR, defaultBranch() does take an `Predicate` as
>>>>>> argument,
>>>>>>>>> but I think that is not required?
>>>>>>>>>
>>>>>>>>> Also, we should consider KIP-307, that was recently accepted and
>>>> is
>>>>>>>>> currently implemented:
>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>>>
>>>>>>>>> Ie, we should add overloads that accepted a `Named` parameter.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> For the issue that the created `KStream` object are in different
>>>>>> scopes:
>>>>>>>>> could we extend `KBranchedStream` with a `get(int index)` method
>>>>> that
>>>>>>>>> returns the corresponding "branched" result `KStream` object?
>>>> Maybe,
>>>>>> the
>>>>>>>>> second argument of `addBranch()` should not be a
>>>> `Consumer<KStream>`
>>>>>> but
>>>>>>>>> a `Function<KStream,KStream>` and `get()` could return whatever
>>>> the
>>>>>>>>> `Function` returns?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Finally, I would also suggest to update the KIP with the current
>>>>>>>>> proposal. That makes it easier to review.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>>> Ivan,
>>>>>>>>>>
>>>>>>>>>> I'm a bit of a novice here as well, but I think it makes sense
>>>> for
>>>>>> you
>>>>>>> to
>>>>>>>>>> revise the KIP and continue the discussion.  Obviously we'll
>>> need
>>>>>> some
>>>>>>>>>> buy-in from committers that have actual binding votes on
>>> whether
>>>>> the
>>>>>>> KIP
>>>>>>>>>> could be adopted.  It would be great to hear if they think this
>>>> is
>>>>> a
>>>>>>> good
>>>>>>>>>> idea overall.  I'm not sure if that happens just by starting a
>>>>> vote,
>>>>>>> or if
>>>>>>>>>> there is generally some indication of interest beforehand.
>>>>>>>>>>
>>>>>>>>>> That being said, I'll continue the discussion a bit: assuming
>>> we
>>>> do
>>>>>>> move
>>>>>>>>>> forward the solution of "stream.branch() returns
>>>> KBranchedStream",
>>>>> do
>>>>>>> we
>>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I would
>>> favor
>>>>>>>>>> deprecating, since having two mutually exclusive APIs that
>>>>> accomplish
>>>>>>> the
>>>>>>>>>> same thing is confusing, especially when they're fairly similar
>>>>>>> anyway.  We
>>>>>>>>>> just need to be sure we're not making something
>>>>> impossible/difficult
>>>>>>> that
>>>>>>>>>> is currently possible/easy.
>>>>>>>>>>
>>>>>>>>>> Regarding my PR - I think the general structure would work,
>>> it's
>>>>>> just a
>>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>>> particular,
>>>>>>>>>> passing in the "predicates" and "children" lists which get
>>>> modified
>>>>>> in
>>>>>>>>>> KBranchedStream but read from all the way KStreamLazyBranch is
>>> a
>>>>> bit
>>>>>>>>>> complicated to follow.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Paul
>>>>>>>>>>
>>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>>> iponomarev@mail.ru
>>>>>>
>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>
>>>>>>>>>>> I read your code carefully and now I am fully convinced: your
>>>>>> proposal
>>>>>>>>>>> looks better and should work. We just have to document the
>>>> crucial
>>>>>>> fact
>>>>>>>>>>> that KStream consumers are invoked as they're added. And then
>>>> it's
>>>>>> all
>>>>>>>>>>> going to be very nice.
>>>>>>>>>>>
>>>>>>>>>>> What shall we do now? I should re-write the KIP and resume the
>>>>>>>>>>> discussion here, right?
>>>>>>>>>>>
>>>>>>>>>>> Why are you telling that your PR 'should not be even a
>>> starting
>>>>>> point
>>>>>>> if
>>>>>>>>>>> we go in this direction'? To me it looks like a good starting
>>>>> point.
>>>>>>> But
>>>>>>>>>>> as a novice in this project I might miss some important
>>> details.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>
>>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>>> stream.branch()
>>>>>>> solution
>>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>>> invoked
>>>> as
>>>>>>> they’re
>>>>>>>>>>> added, not during streamsBuilder.build(). So the user still
>>>> ought
>>>>> to
>>>>>>> be
>>>>>>>>>>> able to call couponIssuer.coupons() afterward and depend on
>>> the
>>>>>>> branched
>>>>>>>>>>> streams having been set.
>>>>>>>>>>>> The issue I mean to point out is that it is hard to access
>>> the
>>>>>>> branched
>>>>>>>>>>> streams in the same scope as the original stream (that is, not
>>>>>> inside
>>>>>>> the
>>>>>>>>>>> couponIssuer), which is a problem with both proposed
>>> solutions.
>>>> It
>>>>>>> can be
>>>>>>>>>>> worked around though.
>>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m excited
>>> to
>>>>>> hear
>>>>>>>>>>> your thoughts!]
>>>>>>>>>>>> Paul
>>>>>>>>>>>>
>>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>>> iponomarev@mail.ru
>>>>>>
>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>>>
>>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>>>>>>> streamsBuilder.build() also looked great for me at first
>>> glance,
>>>>> but
>>>>>>> ---
>>>>>>>>>>>>>> the newly branched streams are not available in the same
>>>> scope
>>>>> as
>>>>>>> each
>>>>>>>>>>> other.  That is, if we wanted to merge them back together
>>> again
>>>> I
>>>>>>> don't see
>>>>>>>>>>> a way to do that.
>>>>>>>>>>>>> You just took the words right out of my mouth, I was just
>>>> going
>>>>> to
>>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say we need
>>> to
>>>>>>> identify
>>>>>>>>>>> customers who have bought coffee and made a purchase in the
>>>>>>> electronics
>>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>>> This is the code I usually write under these circumstances
>>>> using
>>>>>> my
>>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>>> @Setter
>>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>>>>>>>>
>>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>>       return
>>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>>>
>>>>>>>>>>>>>       /*In the real world the code here can be complex, so
>>>>>>> creation of
>>>>>>>>>>> a separate CouponIssuer class is fully justified, in order to
>>>>>> separate
>>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>>  }
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>>>>>>>>
>>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>>     .branch(predicate1, couponIssuer::setCoffePurchases)
>>>>>>>>>>>>>     .branch(predicate2,
>>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>>>
>>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up everything
>>>>>> later,
>>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>>>
>>>>>>>>>>>>> Does this make sense?  In order to properly initialize the
>>>>>>> CouponIssuer
>>>>>>>>>>> we need the terminal operation to be called before
>>>>>>> streamsBuilder.build()
>>>>>>>>>>> is called.
>>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is essentially
>>>> the
>>>>>>> next
>>>>>>>>>>> KIP I was going to write here. I have some thoughts based on
>>> my
>>>>>>> experience,
>>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>> I tried to make a very rough proof of concept of a fluent
>>> API
>>>>>> based
>>>>>>>>>>> off of
>>>>>>>>>>>>>> KStream here (https://github.com/apache/kafka/pull/6512),
>>>> and
>>>>> I
>>>>>>> think
>>>>>>>>>>> I
>>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>>> compatibility
>>>>>>>>>>> issues,
>>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware that Java
>>> is
>>>>>> smart
>>>>>>>>>>> enough to
>>>>>>>>>>>>>>    distinguish between a branch(varargs...) returning one
>>>>> thing
>>>>>>> and
>>>>>>>>>>> branch()
>>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually need
>>> it.
>>>>> We
>>>>>>> can
>>>>>>>>>>> just
>>>>>>>>>>>>>>    build up the branches in the KBranchedStream who shares
>>>> its
>>>>>>> state
>>>>>>>>>>> with the
>>>>>>>>>>>>>>    ProcessorSupplier that will actually do the branching.
>>>>> It's
>>>>>>> not
>>>>>>>>>>> terribly
>>>>>>>>>>>>>>    pretty in its current form, but I think it demonstrates
>>>> its
>>>>>>>>>>> feasibility.
>>>>>>>>>>>>>> To be clear, I don't think that pull request should be
>>> final
>>>> or
>>>>>>> even a
>>>>>>>>>>>>>> starting point if we go in this direction, I just wanted to
>>>> see
>>>>>> how
>>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>>>>>>>>>> I will say though, that I'm not sure the existing solution
>>>>> could
>>>>>> be
>>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>>> suggested
>>>>>> was a
>>>>>>>>>>>>>> possibility.  The reason is that the newly branched streams
>>>> are
>>>>>> not
>>>>>>>>>>>>>> available in the same scope as each other.  That is, if we
>>>>> wanted
>>>>>>> to
>>>>>>>>>>> merge
>>>>>>>>>>>>>> them back together again I don't see a way to do that.  The
>>>> KIP
>>>>>>>>>>> proposal
>>>>>>>>>>>>>> has the same issue, though - all this means is that for
>>>> either
>>>>>>>>>>> solution,
>>>>>>>>>>>>>> deprecating the existing branch(...) is not on the table.
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>>> iponomarev@mail.ru>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to this
>>>> point.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> First, it seems that it's commonly agreed that branch API
>>>>> needs
>>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf returns
>>>> its
>>>>>>> argument
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't
>>> make
>>>>>> sense
>>>>>>>>>>> until
>>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
>>>>>>> contrasts the
>>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>>>> defaultBranch(..)
>>>>>>>>>>> and
>>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface is
>>>> defined.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>>> (defaultBranch(ks->)
>>>>>>> and
>>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to miss the
>>>> fact
>>>>>>> that one
>>>>>>>>>>>>>>> of the terminal methods should be called. If these methods
>>>> are
>>>>>> not
>>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do better?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Still, I believe that this cannot not be implemented the
>>>>> easy
>>>>>>> way.
>>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> user could specify a terminal method that assumes
>>> nothing
>>>>>> will
>>>>>>>>>>> reach
>>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only option
>>>> besides
>>>>>>>>>>>>>>>>> `default`, because there are scenarios when we want to
>>>> just
>>>>>>> silently
>>>>>>>>>>>>>>>>> drop the messages that didn't match any predicate. 2)
>>>>> Throwing
>>>>>>> an
>>>>>>>>>>>>>>>>> exception in the middle of data flow processing looks
>>>> like a
>>>>>> bad
>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to emit a
>>>>>> special
>>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly where
>>>>> `default`
>>>>>>> can
>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>> InternalTopologyBuilder
>>>>> to
>>>>>>> track
>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>>> error
>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>>> compiled
>>>>> and
>>>>>>> run?
>>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't compile if
>>> used
>>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a method chain
>>>>>> starting
>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference between
>>>>>> runtime
>>>>>>> and
>>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>>> instantly
>>>> on
>>>>>>> unit
>>>>>>>>>>>>>>>>> tests, it costs more for the project than a compilation
>>>>>> failure.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Good point about the terminal operation being required.
>>>>> But
>>>>>> is
>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>>> defaultBranch
>>>>>>> they
>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
>>>>>>> easily.  In
>>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API - a
>>> user
>>>>>> could
>>>>>>>>>>> specify
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach the
>>>> default
>>>>>>> branch,
>>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.  That
>>> seems
>>>>> like
>>>>>>> an
>>>>>>>>>>>>>>>>>> improvement over the current branch() API, which allows
>>>> for
>>>>>> the
>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting dropped.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has to be
>>>> well
>>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>>> it would be fairly easily for the
>>> InternalTopologyBuilder
>>>>> to
>>>>>>> track
>>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>>> error
>>>>>>>>>>> before it
>>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is a
>>> "build
>>>>>> step"
>>>>>>>>>>> where
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>>> StreamsBuilder.build()
>>>>> is
>>>>>>>>>>> called.
>>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I agree
>>> that
>>>>> it's
>>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>>> allow users to do other operations on the input stream.
>>>>> With
>>>>>>> the
>>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>>> solution, it ought to work the same way all other
>>>>> operations
>>>>>>> do -
>>>>>>>>>>> if
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> want to process off the original KStream multiple
>>> times,
>>>>> you
>>>>>>> just
>>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>>> stream as a variable so you can call as many operations
>>>> on
>>>>> it
>>>>>>> as
>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not always need
>>>> the
>>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal operation we
>>> don't
>>>>>> know
>>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we
>>> can
>>>> do
>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I understand your point that the need of special
>>> object
>>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods. But
>>> here
>>>> we
>>>>>>> have a
>>>>>>>>>>>>>>>>>>> special case: we build the switch to split the flow,
>>> so
>>>> I
>>>>>>> think
>>>>>>>>>>> this
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this API, but I
>>>> find
>>>>>> the
>>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it contrasts the
>>>>> fluency
>>>>>>> of
>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call
>>> a
>>>>>>> method on
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch cases
>>> are
>>>>>>> defined
>>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very
>>>> nice
>>>>>>> and the
>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around how we
>>>>> specify
>>>>>>> the
>>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1, this::handle1)
>>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2, this::handle2)
>>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>>> KStreamBrancher
>>>>>>> or
>>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and terminated by
>>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>>> returns void).  This is obviously incompatible with
>>> the
>>>>>>> current
>>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a different
>>>> name,
>>>>>> but
>>>>>>> that
>>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>>> something
>>>>>> like
>>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It
>>>> seems
>>>>>>> like it
>>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching while also
>>>>>> allowing
>>>>>>> you
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams
>>>> if
>>>>>>> desired.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1, this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2, this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher
>>> takes a
>>>>>>> Consumer
>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the example in
>>>> the
>>>>>> KIP
>>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would we
>>> handle
>>>>> the
>>>>>>> case
>>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to continue
>>>>>> processing
>>>>>>> and
>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the branched
>>>> stream
>>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if we had
>>>>>> something
>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
>>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP-
>>> 418.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418.
>>> Please
>>>>>> take
>>>>>>> a
>>>>>>>>>>> look
>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Paul Whalen <pg...@gmail.com>.
Perhaps inlining is the wrong terminology. It doesn’t require that a lambda with the full downstream topology be defined inline - it can be a method reference as with Ivan’s original suggestion.  The advantage of putting the predicate and its downstream logic (Consumer) together in branch() is that they are required to be near to each other. 

Ultimately the downstream code has to live somewhere, and deep branch trees will be hard to read regardless.

> On May 1, 2019, at 1:07 PM, Michael Drogalis <mi...@confluent.io> wrote:
> 
> I'm less enthusiastic about inlining the branch logic with its downstream
> functionality. Programs that have deep branch trees will quickly become
> harder to read as a single unit.
> 
>> On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen <pg...@gmail.com> wrote:
>> 
>> Also +1 on the issues/goals as Michael outlined them, I think that sets a
>> great framework for the discussion.
>> 
>> Regarding the SortedMap solution, my understanding is that the current
>> proposal in the KIP is what is in my PR which (pending naming decisions) is
>> roughly this:
>> 
>> stream.split()
>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>>    .defaultBranch(Consumer<KStream<K, V>>);
>> 
>> Obviously some ordering is necessary, since branching as a construct
>> doesn't work without it, but this solution seems like it provides as much
>> associativity as the SortedMap solution, because each branch() call
>> directly associates the "conditional" with the "code block."  The value it
>> provides over the KIP solution is the accessing of streams in the same
>> scope.
>> 
>> The KIP solution is less "dynamic" than the SortedMap solution in the sense
>> that it is slightly clumsier to add a dynamic number of branches, but it is
>> certainly possible.  It seems to me like the API should favor the "static"
>> case anyway, and should make it simple and readable to fluently declare and
>> access your branches in-line.  It also makes it impossible to ignore a
>> branch, and it is possible to build an (almost) identical SortedMap
>> solution on top of it.
>> 
>> I could also see a middle ground where instead of a raw SortedMap being
>> taken in, branch() takes a name and not a Consumer.  Something like this:
>> 
>> Map<String, KStream<K, V>> branches = stream.split()
>>    .branch("branchOne", Predicate<K, V>)
>>    .branch( "branchTwo", Predicate<K, V>)
>>    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>> 
>> Pros for that solution:
>> - accessing branched KStreams in same scope
>> - no double brace initialization, hopefully slightly more readable than
>> SortedMap
>> 
>> Cons
>> - downstream branch logic cannot be specified inline which makes it harder
>> to read top to bottom (like existing API and SortedMap, but unlike the KIP)
>> - you can forget to "handle" one of the branched streams (like existing
>> API and SortedMap, but unlike the KIP)
>> 
>> (KBranchedStreams could even work *both* ways but perhaps that's overdoing
>> it).
>> 
>> Overall I'm curious how important it is to be able to easily access the
>> branched KStream in the same scope as the original.  It's possible that it
>> doesn't need to be handled directly by the API, but instead left up to the
>> user.  I'm sort of in the middle on it.
>> 
>> Paul
>> 
>> 
>> 
>> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman <so...@confluent.io>
>> wrote:
>> 
>>> I'd like to +1 what Michael said about the issues with the existing
>> branch
>>> method, I agree with what he's outlined and I think we should proceed by
>>> trying to alleviate these problems. Specifically it seems important to be
>>> able to cleanly access the individual branches (eg by mapping
>>> name->stream), which I thought was the original intention of this KIP.
>>> 
>>> That said, I don't think we should so easily give in to the double brace
>>> anti-pattern or force ours users into it if at all possible to
>> avoid...just
>>> my two cents.
>>> 
>>> Cheers,
>>> Sophie
>>> 
>>> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
>>> michael.drogalis@confluent.io> wrote:
>>> 
>>>> I’d like to propose a different way of thinking about this. To me,
>> there
>>>> are three problems with the existing branch signature:
>>>> 
>>>> 1. If you use it the way most people do, Java raises unsafe type
>>> warnings.
>>>> 2. The way in which you use the stream branches is positionally coupled
>>> to
>>>> the ordering of the conditionals.
>>>> 3. It is brittle to extend existing branch calls with additional code
>>>> paths.
>>>> 
>>>> Using associative constructs instead of relying on ordered constructs
>>> would
>>>> be a stronger approach. Consider a signature that instead looks like
>>> this:
>>>> 
>>>> Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
>>>> super K,? super V>>);
>>>> 
>>>> Branches are given names in a map, and as a result, the API returns a
>>>> mapping of names to streams. The ordering of the conditionals is
>>> maintained
>>>> because it’s a sorted map. Insert order determines the order of
>>> evaluation.
>>>> 
>>>> This solves problem 1 because there are no more varargs. It solves
>>> problem
>>>> 2 because you no longer lean on ordering to access the branch you’re
>>>> interested in. It solves problem 3 because you can introduce another
>>>> conditional by simply attaching another name to the structure, rather
>>> than
>>>> messing with the existing indices.
>>>> 
>>>> One of the drawbacks is that creating the map inline is historically
>>>> awkward in Java. I know it’s an anti-pattern to use voluminously, but
>>>> double brace initialization would clean up the aesthetics.
>>>> 
>>>> On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io>
>> wrote:
>>>> 
>>>>> Hi Ivan,
>>>>> 
>>>>> Thanks for the update.
>>>>> 
>>>>> FWIW, I agree with Matthias that the current "start branching"
>> operator
>>>> is
>>>>> confusing when named the same way as the actual branches. "Split"
>> seems
>>>>> like a good name. Alternatively, we can do without a "start
>> branching"
>>>>> operator at all, and just do:
>>>>> 
>>>>> stream
>>>>>      .branch(Predicate)
>>>>>      .branch(Predicate)
>>>>>      .defaultBranch();
>>>>> 
>>>>> Tentatively, I think that this branching operation should be
>> terminal.
>>>> That
>>>>> way, we don't create ambiguity about how to use it. That is, `branch`
>>>>> should return `KBranchedStream`, while `defaultBranch` is `void`, to
>>>>> enforce that it comes last, and that there is only one definition of
>>> the
>>>>> default branch. Potentially, we should log a warning if there's no
>>>> default,
>>>>> and additionally log a warning (or throw an exception) if a record
>>> falls
>>>>> though with no default.
>>>>> 
>>>>> Thoughts?
>>>>> 
>>>>> Thanks,
>>>>> -John
>>>>> 
>>>>> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
>> matthias@confluent.io
>>>> 
>>>>> wrote:
>>>>> 
>>>>>> Thanks for updating the KIP and your answers.
>>>>>> 
>>>>>> 
>>>>>>> this is to make the name similar to String#split
>>>>>>>> that also returns an array, right?
>>>>>> 
>>>>>> The intend was to avoid name duplication. The return type should
>>> _not_
>>>>>> be an array.
>>>>>> 
>>>>>> The current proposal is
>>>>>> 
>>>>>> stream.branch()
>>>>>>      .branch(Predicate)
>>>>>>      .branch(Predicate)
>>>>>>      .defaultBranch();
>>>>>> 
>>>>>> IMHO, this reads a little odd, because the first `branch()` does
>> not
>>>>>> take any parameters and has different semantics than the later
>>>>>> `branch()` calls. Note, that from the code snippet above, it's
>> hidden
>>>>>> that the first call is `KStream#branch()` while the others are
>>>>>> `KBranchedStream#branch()` what makes reading the code harder.
>>>>>> 
>>>>>> Because I suggested to rename `addBranch()` -> `branch()`, I though
>>> it
>>>>>> might be better to also rename `KStream#branch()` to avoid the
>> naming
>>>>>> overlap that seems to be confusing. The following reads much
>> cleaner
>>> to
>>>>> me:
>>>>>> 
>>>>>> stream.split()
>>>>>>      .branch(Predicate)
>>>>>>      .branch(Predicate)
>>>>>>      .defaultBranch();
>>>>>> 
>>>>>> Maybe there is a better alternative to `split()` though to avoid
>> the
>>>>>> naming overlap.
>>>>>> 
>>>>>> 
>>>>>>> 'default' is, however, a reserved word, so unfortunately we
>> cannot
>>>> have
>>>>>> a method with such name :-)
>>>>>> 
>>>>>> Bummer. Didn't consider this. Maybe we can still come up with a
>> short
>>>>> name?
>>>>>> 
>>>>>> 
>>>>>> Can you add the interface `KBranchedStream` to the KIP with all
>> it's
>>>>>> methods? It will be part of public API and should be contained in
>> the
>>>>>> KIP. For example, it's unclear atm, what the return type of
>>>>>> `defaultBranch()` is.
>>>>>> 
>>>>>> 
>>>>>> You did not comment on the idea to add a `KBranchedStream#get(int
>>>> index)
>>>>>> -> KStream` method to get the individually branched-KStreams. Would
>>> be
>>>>>> nice to get your feedback about it. It seems you suggest that users
>>>>>> would need to write custom utility code otherwise, to access them.
>> We
>>>>>> should discuss the pros and cons of both approaches. It feels
>>>>>> "incomplete" to me atm, if the API has no built-in support to get
>> the
>>>>>> branched-KStreams directly.
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> -Matthias
>>>>>> 
>>>>>> 
>>>>>>> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
>>>>>>> Hi all!
>>>>>>> 
>>>>>>> I have updated the KIP-418 according to the new vision.
>>>>>>> 
>>>>>>> Matthias, thanks for your comment!
>>>>>>> 
>>>>>>>> Renaming KStream#branch() -> #split()
>>>>>>> 
>>>>>>> I can see your point: this is to make the name similar to
>>>> String#split
>>>>>>> that also returns an array, right? But is it worth the loss of
>>>>> backwards
>>>>>>> compatibility? We can have overloaded branch() as well without
>>>>> affecting
>>>>>>> the existing code. Maybe the old array-based `branch` method
>> should
>>>> be
>>>>>>> deprecated, but this is a subject for discussion.
>>>>>>> 
>>>>>>>> Renaming KBranchedStream#addBranch() ->
>> BranchingKStream#branch(),
>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>> 
>>>>>>> Totally agree with 'addBranch->branch' rename. 'default' is,
>>>> however, a
>>>>>>> reserved word, so unfortunately we cannot have a method with such
>>>> name
>>>>>> :-)
>>>>>>> 
>>>>>>>> defaultBranch() does take an `Predicate` as argument, but I
>> think
>>>> that
>>>>>>> is not required?
>>>>>>> 
>>>>>>> Absolutely! I think that was just copy-paste error or something.
>>>>>>> 
>>>>>>> Dear colleagues,
>>>>>>> 
>>>>>>> please revise the new version of the KIP and Paul's PR
>>>>>>> (https://github.com/apache/kafka/pull/6512)
>>>>>>> 
>>>>>>> Any new suggestions/objections?
>>>>>>> 
>>>>>>> Regards,
>>>>>>> 
>>>>>>> Ivan
>>>>>>> 
>>>>>>> 
>>>>>>> 11.04.2019 11:47, Matthias J. Sax пишет:
>>>>>>>> Thanks for driving the discussion of this KIP. It seems that
>>>> everybody
>>>>>>>> agrees that the current branch() method using arrays is not
>>> optimal.
>>>>>>>> 
>>>>>>>> I had a quick look into the PR and I like the overall proposal.
>>>> There
>>>>>>>> are some minor things we need to consider. I would recommend the
>>>>>>>> following renaming:
>>>>>>>> 
>>>>>>>> KStream#branch() -> #split()
>>>>>>>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>>>>>>>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>>>>>>> 
>>>>>>>> It's just a suggestion to get slightly shorter method names.
>>>>>>>> 
>>>>>>>> In the current PR, defaultBranch() does take an `Predicate` as
>>>>> argument,
>>>>>>>> but I think that is not required?
>>>>>>>> 
>>>>>>>> Also, we should consider KIP-307, that was recently accepted and
>>> is
>>>>>>>> currently implemented:
>>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>>>>>>> 
>>>>>>>> Ie, we should add overloads that accepted a `Named` parameter.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> For the issue that the created `KStream` object are in different
>>>>> scopes:
>>>>>>>> could we extend `KBranchedStream` with a `get(int index)` method
>>>> that
>>>>>>>> returns the corresponding "branched" result `KStream` object?
>>> Maybe,
>>>>> the
>>>>>>>> second argument of `addBranch()` should not be a
>>> `Consumer<KStream>`
>>>>> but
>>>>>>>> a `Function<KStream,KStream>` and `get()` could return whatever
>>> the
>>>>>>>> `Function` returns?
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Finally, I would also suggest to update the KIP with the current
>>>>>>>> proposal. That makes it easier to review.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> -Matthias
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>>>>>>>> Ivan,
>>>>>>>>> 
>>>>>>>>> I'm a bit of a novice here as well, but I think it makes sense
>>> for
>>>>> you
>>>>>> to
>>>>>>>>> revise the KIP and continue the discussion.  Obviously we'll
>> need
>>>>> some
>>>>>>>>> buy-in from committers that have actual binding votes on
>> whether
>>>> the
>>>>>> KIP
>>>>>>>>> could be adopted.  It would be great to hear if they think this
>>> is
>>>> a
>>>>>> good
>>>>>>>>> idea overall.  I'm not sure if that happens just by starting a
>>>> vote,
>>>>>> or if
>>>>>>>>> there is generally some indication of interest beforehand.
>>>>>>>>> 
>>>>>>>>> That being said, I'll continue the discussion a bit: assuming
>> we
>>> do
>>>>>> move
>>>>>>>>> forward the solution of "stream.branch() returns
>>> KBranchedStream",
>>>> do
>>>>>> we
>>>>>>>>> deprecate "stream.branch(...) returns KStream[]"?  I would
>> favor
>>>>>>>>> deprecating, since having two mutually exclusive APIs that
>>>> accomplish
>>>>>> the
>>>>>>>>> same thing is confusing, especially when they're fairly similar
>>>>>> anyway.  We
>>>>>>>>> just need to be sure we're not making something
>>>> impossible/difficult
>>>>>> that
>>>>>>>>> is currently possible/easy.
>>>>>>>>> 
>>>>>>>>> Regarding my PR - I think the general structure would work,
>> it's
>>>>> just a
>>>>>>>>> little sloppy overall in terms of naming and clarity. In
>>>> particular,
>>>>>>>>> passing in the "predicates" and "children" lists which get
>>> modified
>>>>> in
>>>>>>>>> KBranchedStream but read from all the way KStreamLazyBranch is
>> a
>>>> bit
>>>>>>>>> complicated to follow.
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> Paul
>>>>>>>>> 
>>>>>>>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
>>> iponomarev@mail.ru
>>>>> 
>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hi Paul!
>>>>>>>>>> 
>>>>>>>>>> I read your code carefully and now I am fully convinced: your
>>>>> proposal
>>>>>>>>>> looks better and should work. We just have to document the
>>> crucial
>>>>>> fact
>>>>>>>>>> that KStream consumers are invoked as they're added. And then
>>> it's
>>>>> all
>>>>>>>>>> going to be very nice.
>>>>>>>>>> 
>>>>>>>>>> What shall we do now? I should re-write the KIP and resume the
>>>>>>>>>> discussion here, right?
>>>>>>>>>> 
>>>>>>>>>> Why are you telling that your PR 'should not be even a
>> starting
>>>>> point
>>>>>> if
>>>>>>>>>> we go in this direction'? To me it looks like a good starting
>>>> point.
>>>>>> But
>>>>>>>>>> as a novice in this project I might miss some important
>> details.
>>>>>>>>>> 
>>>>>>>>>> Regards,
>>>>>>>>>> 
>>>>>>>>>> Ivan
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>>>>>>>> Ivan,
>>>>>>>>>>> 
>>>>>>>>>>> Maybe I’m missing the point, but I believe the
>> stream.branch()
>>>>>> solution
>>>>>>>>>> supports this. The couponIssuer::set* consumers will be
>> invoked
>>> as
>>>>>> they’re
>>>>>>>>>> added, not during streamsBuilder.build(). So the user still
>>> ought
>>>> to
>>>>>> be
>>>>>>>>>> able to call couponIssuer.coupons() afterward and depend on
>> the
>>>>>> branched
>>>>>>>>>> streams having been set.
>>>>>>>>>>> The issue I mean to point out is that it is hard to access
>> the
>>>>>> branched
>>>>>>>>>> streams in the same scope as the original stream (that is, not
>>>>> inside
>>>>>> the
>>>>>>>>>> couponIssuer), which is a problem with both proposed
>> solutions.
>>> It
>>>>>> can be
>>>>>>>>>> worked around though.
>>>>>>>>>>> [Also, great to hear additional interest in 401, I’m excited
>> to
>>>>> hear
>>>>>>>>>> your thoughts!]
>>>>>>>>>>> Paul
>>>>>>>>>>> 
>>>>>>>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
>>> iponomarev@mail.ru
>>>>> 
>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> Hi Paul!
>>>>>>>>>>>> 
>>>>>>>>>>>> The idea to postpone the wiring of branches to the
>>>>>>>>>> streamsBuilder.build() also looked great for me at first
>> glance,
>>>> but
>>>>>> ---
>>>>>>>>>>>>> the newly branched streams are not available in the same
>>> scope
>>>> as
>>>>>> each
>>>>>>>>>> other.  That is, if we wanted to merge them back together
>> again
>>> I
>>>>>> don't see
>>>>>>>>>> a way to do that.
>>>>>>>>>>>> You just took the words right out of my mouth, I was just
>>> going
>>>> to
>>>>>>>>>> write in details about this issue.
>>>>>>>>>>>> Consider the example from Bill's book, p. 101: say we need
>> to
>>>>>> identify
>>>>>>>>>> customers who have bought coffee and made a purchase in the
>>>>>> electronics
>>>>>>>>>> store to give them coupons.
>>>>>>>>>>>> This is the code I usually write under these circumstances
>>> using
>>>>> my
>>>>>>>>>> 'brancher' class:
>>>>>>>>>>>> @Setter
>>>>>>>>>>>> class CouponIssuer{
>>>>>>>>>>>>   private KStream<....> coffePurchases;
>>>>>>>>>>>>   private KStream<....> electronicsPurchases;
>>>>>>>>>>>> 
>>>>>>>>>>>>   KStream<...> coupons(){
>>>>>>>>>>>>       return
>>>>>> coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>>>>>>> 
>>>>>>>>>>>>       /*In the real world the code here can be complex, so
>>>>>> creation of
>>>>>>>>>> a separate CouponIssuer class is fully justified, in order to
>>>>> separate
>>>>>>>>>> classes' responsibilities.*/
>>>>>>>>>>>>  }
>>>>>>>>>>>> }
>>>>>>>>>>>> 
>>>>>>>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>>>>>>> 
>>>>>>>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>>>>>>>     .branch(predicate1, couponIssuer::setCoffePurchases)
>>>>>>>>>>>>     .branch(predicate2,
>>> couponIssuer::setElectronicsPurchases)
>>>>>>>>>>>>     .onTopOf(transactionStream);
>>>>>>>>>>>> 
>>>>>>>>>>>> /*Alas, this won't work if we're going to wire up everything
>>>>> later,
>>>>>>>>>> without the terminal operation!!!*/
>>>>>>>>>>>> couponIssuer.coupons()...
>>>>>>>>>>>> 
>>>>>>>>>>>> Does this make sense?  In order to properly initialize the
>>>>>> CouponIssuer
>>>>>>>>>> we need the terminal operation to be called before
>>>>>> streamsBuilder.build()
>>>>>>>>>> is called.
>>>>>>>>>>>> [BTW Paul, I just found out that your KIP-401 is essentially
>>> the
>>>>>> next
>>>>>>>>>> KIP I was going to write here. I have some thoughts based on
>> my
>>>>>> experience,
>>>>>>>>>> so I will join the discussion on KIP-401 soon.]
>>>>>>>>>>>> Regards,
>>>>>>>>>>>> 
>>>>>>>>>>>> Ivan
>>>>>>>>>>>> 
>>>>>>>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>> I tried to make a very rough proof of concept of a fluent
>> API
>>>>> based
>>>>>>>>>> off of
>>>>>>>>>>>>> KStream here (https://github.com/apache/kafka/pull/6512),
>>> and
>>>> I
>>>>>> think
>>>>>>>>>> I
>>>>>>>>>>>>> succeeded at removing both cons.
>>>>>>>>>>>>>    - Compatibility: I was incorrect earlier about
>>>> compatibility
>>>>>>>>>> issues,
>>>>>>>>>>>>>    there aren't any direct ones.  I was unaware that Java
>> is
>>>>> smart
>>>>>>>>>> enough to
>>>>>>>>>>>>>    distinguish between a branch(varargs...) returning one
>>>> thing
>>>>>> and
>>>>>>>>>> branch()
>>>>>>>>>>>>>    with no arguments returning another thing.
>>>>>>>>>>>>>    - Requiring a terminal method: We don't actually need
>> it.
>>>> We
>>>>>> can
>>>>>>>>>> just
>>>>>>>>>>>>>    build up the branches in the KBranchedStream who shares
>>> its
>>>>>> state
>>>>>>>>>> with the
>>>>>>>>>>>>>    ProcessorSupplier that will actually do the branching.
>>>> It's
>>>>>> not
>>>>>>>>>> terribly
>>>>>>>>>>>>>    pretty in its current form, but I think it demonstrates
>>> its
>>>>>>>>>> feasibility.
>>>>>>>>>>>>> To be clear, I don't think that pull request should be
>> final
>>> or
>>>>>> even a
>>>>>>>>>>>>> starting point if we go in this direction, I just wanted to
>>> see
>>>>> how
>>>>>>>>>>>>> challenging it would be to get the API working.
>>>>>>>>>>>>> I will say though, that I'm not sure the existing solution
>>>> could
>>>>> be
>>>>>>>>>>>>> deprecated in favor of this, which I had originally
>> suggested
>>>>> was a
>>>>>>>>>>>>> possibility.  The reason is that the newly branched streams
>>> are
>>>>> not
>>>>>>>>>>>>> available in the same scope as each other.  That is, if we
>>>> wanted
>>>>>> to
>>>>>>>>>> merge
>>>>>>>>>>>>> them back together again I don't see a way to do that.  The
>>> KIP
>>>>>>>>>> proposal
>>>>>>>>>>>>> has the same issue, though - all this means is that for
>>> either
>>>>>>>>>> solution,
>>>>>>>>>>>>> deprecating the existing branch(...) is not on the table.
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
>>>>>> iponomarev@mail.ru>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>> OK, let me summarize what we have discussed up to this
>>> point.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> First, it seems that it's commonly agreed that branch API
>>>> needs
>>>>>>>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> There are two potential ways to do it:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>>>>>>>    .branch(predicate1, ks ->..)
>>>>>>>>>>>>>>    .branch(predicate2, ks->..)
>>>>>>>>>>>>>>    .defaultBranch(ks->..) //optional
>>>>>>>>>>>>>>    .onTopOf(stream).mapValues(...).... //onTopOf returns
>>> its
>>>>>> argument
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't
>> make
>>>>> sense
>>>>>>>>>> until
>>>>>>>>>>>>>> all the necessary ingredients are provided.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
>>>>>> contrasts the
>>>>>>>>>>>>>> fluency of other KStream methods.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 2. (as Paul proposes)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>    .branch(predicate1, ks ->...)
>>>>>>>>>>>>>>    .branch(predicate2, ks->...)
>>>>>>>>>>>>>>    .defaultBranch(ks->...) //or noDefault(). Both
>>>>>> defaultBranch(..)
>>>>>>>>>> and
>>>>>>>>>>>>>> noDefault() return void
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> PROS: Generally follows the way KStreams interface is
>>> defined.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> CONS: We need to define two terminal methods
>>>>> (defaultBranch(ks->)
>>>>>> and
>>>>>>>>>>>>>> noDefault()). And for a user it is very easy to miss the
>>> fact
>>>>>> that one
>>>>>>>>>>>>>> of the terminal methods should be called. If these methods
>>> are
>>>>> not
>>>>>>>>>>>>>> called, we can throw an exception in runtime.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Colleagues, what are your thoughts? Can we do better?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>>>>>>>> Paul,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> I see your point when you are talking about
>>>>>>>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Still, I believe that this cannot not be implemented the
>>>> easy
>>>>>> way.
>>>>>>>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> user could specify a terminal method that assumes
>> nothing
>>>>> will
>>>>>>>>>> reach
>>>>>>>>>>>>>>>>> the default branch,
>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1) OK, apparently this should not be the only option
>>> besides
>>>>>>>>>>>>>>>> `default`, because there are scenarios when we want to
>>> just
>>>>>> silently
>>>>>>>>>>>>>>>> drop the messages that didn't match any predicate. 2)
>>>> Throwing
>>>>>> an
>>>>>>>>>>>>>>>> exception in the middle of data flow processing looks
>>> like a
>>>>> bad
>>>>>>>>>> idea.
>>>>>>>>>>>>>>>> In stream processing paradigm, I would prefer to emit a
>>>>> special
>>>>>>>>>>>>>>>> message to a dedicated stream. This is exactly where
>>>> `default`
>>>>>> can
>>>>>>>>>> be
>>>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> it would be fairly easily for the
>> InternalTopologyBuilder
>>>> to
>>>>>> track
>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>> error
>>>>>>>>>> before it
>>>>>>>>>>>>>>>> becomes an issue.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> You mean a runtime exception, when the program is
>> compiled
>>>> and
>>>>>> run?
>>>>>>>>>>>>>>>> Well,  I'd prefer an API that simply won't compile if
>> used
>>>>>>>>>>>>>>>> incorrectly. Can we build such an API as a method chain
>>>>> starting
>>>>>>>>>> from
>>>>>>>>>>>>>>>> KStream object? There is a huge cost difference between
>>>>> runtime
>>>>>> and
>>>>>>>>>>>>>>>> compile-time errors. Even if a failure uncovers
>> instantly
>>> on
>>>>>> unit
>>>>>>>>>>>>>>>> tests, it costs more for the project than a compilation
>>>>> failure.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Good point about the terminal operation being required.
>>>> But
>>>>> is
>>>>>>>>>> that
>>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>> such a bad thing?  If the user doesn't want a
>>> defaultBranch
>>>>>> they
>>>>>>>>>> can
>>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
>>>>>> easily.  In
>>>>>>>>>>>>>>>>> fact I
>>>>>>>>>>>>>>>>> think it creates an opportunity for a nicer API - a
>> user
>>>>> could
>>>>>>>>>> specify
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> terminal method that assumes nothing will reach the
>>> default
>>>>>> branch,
>>>>>>>>>>>>>>>>> throwing an exception if such a case occurs.  That
>> seems
>>>> like
>>>>>> an
>>>>>>>>>>>>>>>>> improvement over the current branch() API, which allows
>>> for
>>>>> the
>>>>>>>>>> more
>>>>>>>>>>>>>>>>> subtle
>>>>>>>>>>>>>>>>> behavior of records unexpectedly getting dropped.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> The need for a terminal operation certainly has to be
>>> well
>>>>>>>>>>>>>>>>> documented, but
>>>>>>>>>>>>>>>>> it would be fairly easily for the
>> InternalTopologyBuilder
>>>> to
>>>>>> track
>>>>>>>>>>>>>>>>> dangling
>>>>>>>>>>>>>>>>> branches that haven't been terminated and raise a clear
>>>> error
>>>>>>>>>> before it
>>>>>>>>>>>>>>>>> becomes an issue.  Especially now that there is a
>> "build
>>>>> step"
>>>>>>>>>> where
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> topology is actually wired up, when
>>> StreamsBuilder.build()
>>>> is
>>>>>>>>>> called.
>>>>>>>>>>>>>>>>> Regarding onTopOf() returning its argument, I agree
>> that
>>>> it's
>>>>>>>>>>>>>>>>> critical to
>>>>>>>>>>>>>>>>> allow users to do other operations on the input stream.
>>>> With
>>>>>> the
>>>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>> solution, it ought to work the same way all other
>>>> operations
>>>>>> do -
>>>>>>>>>> if
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>> want to process off the original KStream multiple
>> times,
>>>> you
>>>>>> just
>>>>>>>>>>>>>>>>> need the
>>>>>>>>>>>>>>>>> stream as a variable so you can call as many operations
>>> on
>>>> it
>>>>>> as
>>>>>>>>>> you
>>>>>>>>>>>>>>>>> desire.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
>>>>>> iponomarev@mail.ru
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> I afraid this won't work because we do not always need
>>> the
>>>>>>>>>>>>>>>>>> defaultBranch. And without a terminal operation we
>> don't
>>>>> know
>>>>>>>>>> when to
>>>>>>>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we
>> can
>>> do
>>>>>>>>>> something
>>>>>>>>>>>>>>>>>> more with the original branch after branching.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> I understand your point that the need of special
>> object
>>>>>>>>>> construction
>>>>>>>>>>>>>>>>>> contrasts the fluency of most KStream methods. But
>> here
>>> we
>>>>>> have a
>>>>>>>>>>>>>>>>>> special case: we build the switch to split the flow,
>> so
>>> I
>>>>>> think
>>>>>>>>>> this
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> I think it's a great idea to improve this API, but I
>>> find
>>>>> the
>>>>>>>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>>>>>>>> mechanism a little confusing since it contrasts the
>>>> fluency
>>>>>> of
>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call
>> a
>>>>>> method on
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>> so it still reads top to bottom if the branch cases
>> are
>>>>>> defined
>>>>>>>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very
>>> nice
>>>>>> and the
>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>> to do things, but what if we flipped around how we
>>>> specify
>>>>>> the
>>>>>>>>>> source
>>>>>>>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>>>>>>>           .addBranch(predicate1, this::handle1)
>>>>>>>>>>>>>>>>>>>           .addBranch(predicate2, this::handle2)
>>>>>>>>>>>>>>>>>>>           .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
>>>>> KStreamBrancher
>>>>>> or
>>>>>>>>>>>>>>>>>> something,
>>>>>>>>>>>>>>>>>>> which is added to by addBranch() and terminated by
>>>>>>>>>> defaultBranch()
>>>>>>>>>>>>>>>>>>> (which
>>>>>>>>>>>>>>>>>>> returns void).  This is obviously incompatible with
>> the
>>>>>> current
>>>>>>>>>>>>>>>>>>> API, so
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> new stream.branch() would have to have a different
>>> name,
>>>>> but
>>>>>> that
>>>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>> like a fairly small problem - we could call it
>>> something
>>>>> like
>>>>>>>>>>>>>>>>>>> branched()
>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It
>>> seems
>>>>>> like it
>>>>>>>>>>>>>>>>>>> does to
>>>>>>>>>>>>>>>>>>> me, allowing for clear in-line branching while also
>>>>> allowing
>>>>>> you
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams
>>> if
>>>>>> desired.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>           ks.filter(....).mapValues(...)
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
>>>>>>>>>>>>>>>>>>>>           ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate1, this::handleFirstCase)
>>>>>>>>>>>>>>>>>>>>      .addBranch(predicate2, this::handleSecondCase)
>>>>>>>>>>>>>>>>>>>>      .onTopOf(....)
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher
>> takes a
>>>>>> Consumer
>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>> argument which returns nothing, and the example in
>>> the
>>>>> KIP
>>>>>>>>>> shows
>>>>>>>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>>> stream from the branch using a terminal node
>>>>>> (KafkaStreams#to()
>>>>>>>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Maybe I've missed something, but how would we
>> handle
>>>> the
>>>>>> case
>>>>>>>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>>>>>>>> user has created a branch but wants to continue
>>>>> processing
>>>>>> and
>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> necessarily use a terminal node on the branched
>>> stream
>>>>>>>>>> immediately?
>>>>>>>>>>>>>>>>>>>>> For example, using today's logic as is if we had
>>>>> something
>>>>>> like
>>>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
>>>>>> bbejeck@gmail.com
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP-
>> 418.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418.
>> Please
>>>>> take
>>>>>> a
>>>>>>>>>> look
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> KIP-418:
>>>>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>>>>>>>> PR#6164:
>> https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Michael Drogalis <mi...@confluent.io>.
I'm less enthusiastic about inlining the branch logic with its downstream
functionality. Programs that have deep branch trees will quickly become
harder to read as a single unit.

On Tue, Apr 30, 2019 at 8:34 PM Paul Whalen <pg...@gmail.com> wrote:

> Also +1 on the issues/goals as Michael outlined them, I think that sets a
> great framework for the discussion.
>
> Regarding the SortedMap solution, my understanding is that the current
> proposal in the KIP is what is in my PR which (pending naming decisions) is
> roughly this:
>
> stream.split()
>     .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>     .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
>     .defaultBranch(Consumer<KStream<K, V>>);
>
> Obviously some ordering is necessary, since branching as a construct
> doesn't work without it, but this solution seems like it provides as much
> associativity as the SortedMap solution, because each branch() call
> directly associates the "conditional" with the "code block."  The value it
> provides over the KIP solution is the accessing of streams in the same
> scope.
>
> The KIP solution is less "dynamic" than the SortedMap solution in the sense
> that it is slightly clumsier to add a dynamic number of branches, but it is
> certainly possible.  It seems to me like the API should favor the "static"
> case anyway, and should make it simple and readable to fluently declare and
> access your branches in-line.  It also makes it impossible to ignore a
> branch, and it is possible to build an (almost) identical SortedMap
> solution on top of it.
>
> I could also see a middle ground where instead of a raw SortedMap being
> taken in, branch() takes a name and not a Consumer.  Something like this:
>
> Map<String, KStream<K, V>> branches = stream.split()
>     .branch("branchOne", Predicate<K, V>)
>     .branch( "branchTwo", Predicate<K, V>)
>     .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);
>
> Pros for that solution:
>  - accessing branched KStreams in same scope
>  - no double brace initialization, hopefully slightly more readable than
> SortedMap
>
> Cons
>  - downstream branch logic cannot be specified inline which makes it harder
> to read top to bottom (like existing API and SortedMap, but unlike the KIP)
>  - you can forget to "handle" one of the branched streams (like existing
> API and SortedMap, but unlike the KIP)
>
> (KBranchedStreams could even work *both* ways but perhaps that's overdoing
> it).
>
> Overall I'm curious how important it is to be able to easily access the
> branched KStream in the same scope as the original.  It's possible that it
> doesn't need to be handled directly by the API, but instead left up to the
> user.  I'm sort of in the middle on it.
>
> Paul
>
>
>
> On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman <so...@confluent.io>
> wrote:
>
> > I'd like to +1 what Michael said about the issues with the existing
> branch
> > method, I agree with what he's outlined and I think we should proceed by
> > trying to alleviate these problems. Specifically it seems important to be
> > able to cleanly access the individual branches (eg by mapping
> > name->stream), which I thought was the original intention of this KIP.
> >
> > That said, I don't think we should so easily give in to the double brace
> > anti-pattern or force ours users into it if at all possible to
> avoid...just
> > my two cents.
> >
> > Cheers,
> > Sophie
> >
> > On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
> > michael.drogalis@confluent.io> wrote:
> >
> > > I’d like to propose a different way of thinking about this. To me,
> there
> > > are three problems with the existing branch signature:
> > >
> > > 1. If you use it the way most people do, Java raises unsafe type
> > warnings.
> > > 2. The way in which you use the stream branches is positionally coupled
> > to
> > > the ordering of the conditionals.
> > > 3. It is brittle to extend existing branch calls with additional code
> > > paths.
> > >
> > > Using associative constructs instead of relying on ordered constructs
> > would
> > > be a stronger approach. Consider a signature that instead looks like
> > this:
> > >
> > > Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
> > > super K,? super V>>);
> > >
> > > Branches are given names in a map, and as a result, the API returns a
> > > mapping of names to streams. The ordering of the conditionals is
> > maintained
> > > because it’s a sorted map. Insert order determines the order of
> > evaluation.
> > >
> > > This solves problem 1 because there are no more varargs. It solves
> > problem
> > > 2 because you no longer lean on ordering to access the branch you’re
> > > interested in. It solves problem 3 because you can introduce another
> > > conditional by simply attaching another name to the structure, rather
> > than
> > > messing with the existing indices.
> > >
> > > One of the drawbacks is that creating the map inline is historically
> > > awkward in Java. I know it’s an anti-pattern to use voluminously, but
> > > double brace initialization would clean up the aesthetics.
> > >
> > > On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io>
> wrote:
> > >
> > > > Hi Ivan,
> > > >
> > > > Thanks for the update.
> > > >
> > > > FWIW, I agree with Matthias that the current "start branching"
> operator
> > > is
> > > > confusing when named the same way as the actual branches. "Split"
> seems
> > > > like a good name. Alternatively, we can do without a "start
> branching"
> > > > operator at all, and just do:
> > > >
> > > > stream
> > > >       .branch(Predicate)
> > > >       .branch(Predicate)
> > > >       .defaultBranch();
> > > >
> > > > Tentatively, I think that this branching operation should be
> terminal.
> > > That
> > > > way, we don't create ambiguity about how to use it. That is, `branch`
> > > > should return `KBranchedStream`, while `defaultBranch` is `void`, to
> > > > enforce that it comes last, and that there is only one definition of
> > the
> > > > default branch. Potentially, we should log a warning if there's no
> > > default,
> > > > and additionally log a warning (or throw an exception) if a record
> > falls
> > > > though with no default.
> > > >
> > > > Thoughts?
> > > >
> > > > Thanks,
> > > > -John
> > > >
> > > > On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <
> matthias@confluent.io
> > >
> > > > wrote:
> > > >
> > > > > Thanks for updating the KIP and your answers.
> > > > >
> > > > >
> > > > > >  this is to make the name similar to String#split
> > > > > >> that also returns an array, right?
> > > > >
> > > > > The intend was to avoid name duplication. The return type should
> > _not_
> > > > > be an array.
> > > > >
> > > > > The current proposal is
> > > > >
> > > > > stream.branch()
> > > > >       .branch(Predicate)
> > > > >       .branch(Predicate)
> > > > >       .defaultBranch();
> > > > >
> > > > > IMHO, this reads a little odd, because the first `branch()` does
> not
> > > > > take any parameters and has different semantics than the later
> > > > > `branch()` calls. Note, that from the code snippet above, it's
> hidden
> > > > > that the first call is `KStream#branch()` while the others are
> > > > > `KBranchedStream#branch()` what makes reading the code harder.
> > > > >
> > > > > Because I suggested to rename `addBranch()` -> `branch()`, I though
> > it
> > > > > might be better to also rename `KStream#branch()` to avoid the
> naming
> > > > > overlap that seems to be confusing. The following reads much
> cleaner
> > to
> > > > me:
> > > > >
> > > > > stream.split()
> > > > >       .branch(Predicate)
> > > > >       .branch(Predicate)
> > > > >       .defaultBranch();
> > > > >
> > > > > Maybe there is a better alternative to `split()` though to avoid
> the
> > > > > naming overlap.
> > > > >
> > > > >
> > > > > > 'default' is, however, a reserved word, so unfortunately we
> cannot
> > > have
> > > > > a method with such name :-)
> > > > >
> > > > > Bummer. Didn't consider this. Maybe we can still come up with a
> short
> > > > name?
> > > > >
> > > > >
> > > > > Can you add the interface `KBranchedStream` to the KIP with all
> it's
> > > > > methods? It will be part of public API and should be contained in
> the
> > > > > KIP. For example, it's unclear atm, what the return type of
> > > > > `defaultBranch()` is.
> > > > >
> > > > >
> > > > > You did not comment on the idea to add a `KBranchedStream#get(int
> > > index)
> > > > > -> KStream` method to get the individually branched-KStreams. Would
> > be
> > > > > nice to get your feedback about it. It seems you suggest that users
> > > > > would need to write custom utility code otherwise, to access them.
> We
> > > > > should discuss the pros and cons of both approaches. It feels
> > > > > "incomplete" to me atm, if the API has no built-in support to get
> the
> > > > > branched-KStreams directly.
> > > > >
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > >
> > > > > On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> > > > > > Hi all!
> > > > > >
> > > > > > I have updated the KIP-418 according to the new vision.
> > > > > >
> > > > > > Matthias, thanks for your comment!
> > > > > >
> > > > > >> Renaming KStream#branch() -> #split()
> > > > > >
> > > > > > I can see your point: this is to make the name similar to
> > > String#split
> > > > > > that also returns an array, right? But is it worth the loss of
> > > > backwards
> > > > > > compatibility? We can have overloaded branch() as well without
> > > > affecting
> > > > > > the existing code. Maybe the old array-based `branch` method
> should
> > > be
> > > > > > deprecated, but this is a subject for discussion.
> > > > > >
> > > > > >> Renaming KBranchedStream#addBranch() ->
> BranchingKStream#branch(),
> > > > > > KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > > > > >
> > > > > > Totally agree with 'addBranch->branch' rename. 'default' is,
> > > however, a
> > > > > > reserved word, so unfortunately we cannot have a method with such
> > > name
> > > > > :-)
> > > > > >
> > > > > >> defaultBranch() does take an `Predicate` as argument, but I
> think
> > > that
> > > > > > is not required?
> > > > > >
> > > > > > Absolutely! I think that was just copy-paste error or something.
> > > > > >
> > > > > > Dear colleagues,
> > > > > >
> > > > > > please revise the new version of the KIP and Paul's PR
> > > > > > (https://github.com/apache/kafka/pull/6512)
> > > > > >
> > > > > > Any new suggestions/objections?
> > > > > >
> > > > > > Regards,
> > > > > >
> > > > > > Ivan
> > > > > >
> > > > > >
> > > > > > 11.04.2019 11:47, Matthias J. Sax пишет:
> > > > > >> Thanks for driving the discussion of this KIP. It seems that
> > > everybody
> > > > > >> agrees that the current branch() method using arrays is not
> > optimal.
> > > > > >>
> > > > > >> I had a quick look into the PR and I like the overall proposal.
> > > There
> > > > > >> are some minor things we need to consider. I would recommend the
> > > > > >> following renaming:
> > > > > >>
> > > > > >> KStream#branch() -> #split()
> > > > > >> KBranchedStream#addBranch() -> BranchingKStream#branch()
> > > > > >> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > > > > >>
> > > > > >> It's just a suggestion to get slightly shorter method names.
> > > > > >>
> > > > > >> In the current PR, defaultBranch() does take an `Predicate` as
> > > > argument,
> > > > > >> but I think that is not required?
> > > > > >>
> > > > > >> Also, we should consider KIP-307, that was recently accepted and
> > is
> > > > > >> currently implemented:
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> > > > > >>
> > > > > >> Ie, we should add overloads that accepted a `Named` parameter.
> > > > > >>
> > > > > >>
> > > > > >> For the issue that the created `KStream` object are in different
> > > > scopes:
> > > > > >> could we extend `KBranchedStream` with a `get(int index)` method
> > > that
> > > > > >> returns the corresponding "branched" result `KStream` object?
> > Maybe,
> > > > the
> > > > > >> second argument of `addBranch()` should not be a
> > `Consumer<KStream>`
> > > > but
> > > > > >> a `Function<KStream,KStream>` and `get()` could return whatever
> > the
> > > > > >> `Function` returns?
> > > > > >>
> > > > > >>
> > > > > >> Finally, I would also suggest to update the KIP with the current
> > > > > >> proposal. That makes it easier to review.
> > > > > >>
> > > > > >>
> > > > > >> -Matthias
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >> On 3/31/19 12:22 PM, Paul Whalen wrote:
> > > > > >>> Ivan,
> > > > > >>>
> > > > > >>> I'm a bit of a novice here as well, but I think it makes sense
> > for
> > > > you
> > > > > to
> > > > > >>> revise the KIP and continue the discussion.  Obviously we'll
> need
> > > > some
> > > > > >>> buy-in from committers that have actual binding votes on
> whether
> > > the
> > > > > KIP
> > > > > >>> could be adopted.  It would be great to hear if they think this
> > is
> > > a
> > > > > good
> > > > > >>> idea overall.  I'm not sure if that happens just by starting a
> > > vote,
> > > > > or if
> > > > > >>> there is generally some indication of interest beforehand.
> > > > > >>>
> > > > > >>> That being said, I'll continue the discussion a bit: assuming
> we
> > do
> > > > > move
> > > > > >>> forward the solution of "stream.branch() returns
> > KBranchedStream",
> > > do
> > > > > we
> > > > > >>> deprecate "stream.branch(...) returns KStream[]"?  I would
> favor
> > > > > >>> deprecating, since having two mutually exclusive APIs that
> > > accomplish
> > > > > the
> > > > > >>> same thing is confusing, especially when they're fairly similar
> > > > > anyway.  We
> > > > > >>> just need to be sure we're not making something
> > > impossible/difficult
> > > > > that
> > > > > >>> is currently possible/easy.
> > > > > >>>
> > > > > >>> Regarding my PR - I think the general structure would work,
> it's
> > > > just a
> > > > > >>> little sloppy overall in terms of naming and clarity. In
> > > particular,
> > > > > >>> passing in the "predicates" and "children" lists which get
> > modified
> > > > in
> > > > > >>> KBranchedStream but read from all the way KStreamLazyBranch is
> a
> > > bit
> > > > > >>> complicated to follow.
> > > > > >>>
> > > > > >>> Thanks,
> > > > > >>> Paul
> > > > > >>>
> > > > > >>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
> > iponomarev@mail.ru
> > > >
> > > > > wrote:
> > > > > >>>
> > > > > >>>> Hi Paul!
> > > > > >>>>
> > > > > >>>> I read your code carefully and now I am fully convinced: your
> > > > proposal
> > > > > >>>> looks better and should work. We just have to document the
> > crucial
> > > > > fact
> > > > > >>>> that KStream consumers are invoked as they're added. And then
> > it's
> > > > all
> > > > > >>>> going to be very nice.
> > > > > >>>>
> > > > > >>>> What shall we do now? I should re-write the KIP and resume the
> > > > > >>>> discussion here, right?
> > > > > >>>>
> > > > > >>>> Why are you telling that your PR 'should not be even a
> starting
> > > > point
> > > > > if
> > > > > >>>> we go in this direction'? To me it looks like a good starting
> > > point.
> > > > > But
> > > > > >>>> as a novice in this project I might miss some important
> details.
> > > > > >>>>
> > > > > >>>> Regards,
> > > > > >>>>
> > > > > >>>> Ivan
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> 28.03.2019 17:38, Paul Whalen пишет:
> > > > > >>>>> Ivan,
> > > > > >>>>>
> > > > > >>>>> Maybe I’m missing the point, but I believe the
> stream.branch()
> > > > > solution
> > > > > >>>> supports this. The couponIssuer::set* consumers will be
> invoked
> > as
> > > > > they’re
> > > > > >>>> added, not during streamsBuilder.build(). So the user still
> > ought
> > > to
> > > > > be
> > > > > >>>> able to call couponIssuer.coupons() afterward and depend on
> the
> > > > > branched
> > > > > >>>> streams having been set.
> > > > > >>>>> The issue I mean to point out is that it is hard to access
> the
> > > > > branched
> > > > > >>>> streams in the same scope as the original stream (that is, not
> > > > inside
> > > > > the
> > > > > >>>> couponIssuer), which is a problem with both proposed
> solutions.
> > It
> > > > > can be
> > > > > >>>> worked around though.
> > > > > >>>>> [Also, great to hear additional interest in 401, I’m excited
> to
> > > > hear
> > > > > >>>> your thoughts!]
> > > > > >>>>> Paul
> > > > > >>>>>
> > > > > >>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
> > iponomarev@mail.ru
> > > >
> > > > > wrote:
> > > > > >>>>>>
> > > > > >>>>>> Hi Paul!
> > > > > >>>>>>
> > > > > >>>>>> The idea to postpone the wiring of branches to the
> > > > > >>>> streamsBuilder.build() also looked great for me at first
> glance,
> > > but
> > > > > ---
> > > > > >>>>>>> the newly branched streams are not available in the same
> > scope
> > > as
> > > > > each
> > > > > >>>> other.  That is, if we wanted to merge them back together
> again
> > I
> > > > > don't see
> > > > > >>>> a way to do that.
> > > > > >>>>>> You just took the words right out of my mouth, I was just
> > going
> > > to
> > > > > >>>> write in details about this issue.
> > > > > >>>>>> Consider the example from Bill's book, p. 101: say we need
> to
> > > > > identify
> > > > > >>>> customers who have bought coffee and made a purchase in the
> > > > > electronics
> > > > > >>>> store to give them coupons.
> > > > > >>>>>> This is the code I usually write under these circumstances
> > using
> > > > my
> > > > > >>>> 'brancher' class:
> > > > > >>>>>> @Setter
> > > > > >>>>>> class CouponIssuer{
> > > > > >>>>>>    private KStream<....> coffePurchases;
> > > > > >>>>>>    private KStream<....> electronicsPurchases;
> > > > > >>>>>>
> > > > > >>>>>>    KStream<...> coupons(){
> > > > > >>>>>>        return
> > > > > coffePurchases.join(electronicsPurchases...)...whatever
> > > > > >>>>>>
> > > > > >>>>>>        /*In the real world the code here can be complex, so
> > > > > creation of
> > > > > >>>> a separate CouponIssuer class is fully justified, in order to
> > > > separate
> > > > > >>>> classes' responsibilities.*/
> > > > > >>>>>>   }
> > > > > >>>>>> }
> > > > > >>>>>>
> > > > > >>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> > > > > >>>>>>
> > > > > >>>>>> new KafkaStreamsBrancher<....>()
> > > > > >>>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
> > > > > >>>>>>      .branch(predicate2,
> > couponIssuer::setElectronicsPurchases)
> > > > > >>>>>>      .onTopOf(transactionStream);
> > > > > >>>>>>
> > > > > >>>>>> /*Alas, this won't work if we're going to wire up everything
> > > > later,
> > > > > >>>> without the terminal operation!!!*/
> > > > > >>>>>> couponIssuer.coupons()...
> > > > > >>>>>>
> > > > > >>>>>> Does this make sense?  In order to properly initialize the
> > > > > CouponIssuer
> > > > > >>>> we need the terminal operation to be called before
> > > > > streamsBuilder.build()
> > > > > >>>> is called.
> > > > > >>>>>> [BTW Paul, I just found out that your KIP-401 is essentially
> > the
> > > > > next
> > > > > >>>> KIP I was going to write here. I have some thoughts based on
> my
> > > > > experience,
> > > > > >>>> so I will join the discussion on KIP-401 soon.]
> > > > > >>>>>> Regards,
> > > > > >>>>>>
> > > > > >>>>>> Ivan
> > > > > >>>>>>
> > > > > >>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> > > > > >>>>>>> Ivan,
> > > > > >>>>>>> I tried to make a very rough proof of concept of a fluent
> API
> > > > based
> > > > > >>>> off of
> > > > > >>>>>>> KStream here (https://github.com/apache/kafka/pull/6512),
> > and
> > > I
> > > > > think
> > > > > >>>> I
> > > > > >>>>>>> succeeded at removing both cons.
> > > > > >>>>>>>     - Compatibility: I was incorrect earlier about
> > > compatibility
> > > > > >>>> issues,
> > > > > >>>>>>>     there aren't any direct ones.  I was unaware that Java
> is
> > > > smart
> > > > > >>>> enough to
> > > > > >>>>>>>     distinguish between a branch(varargs...) returning one
> > > thing
> > > > > and
> > > > > >>>> branch()
> > > > > >>>>>>>     with no arguments returning another thing.
> > > > > >>>>>>>     - Requiring a terminal method: We don't actually need
> it.
> > > We
> > > > > can
> > > > > >>>> just
> > > > > >>>>>>>     build up the branches in the KBranchedStream who shares
> > its
> > > > > state
> > > > > >>>> with the
> > > > > >>>>>>>     ProcessorSupplier that will actually do the branching.
> > > It's
> > > > > not
> > > > > >>>> terribly
> > > > > >>>>>>>     pretty in its current form, but I think it demonstrates
> > its
> > > > > >>>> feasibility.
> > > > > >>>>>>> To be clear, I don't think that pull request should be
> final
> > or
> > > > > even a
> > > > > >>>>>>> starting point if we go in this direction, I just wanted to
> > see
> > > > how
> > > > > >>>>>>> challenging it would be to get the API working.
> > > > > >>>>>>> I will say though, that I'm not sure the existing solution
> > > could
> > > > be
> > > > > >>>>>>> deprecated in favor of this, which I had originally
> suggested
> > > > was a
> > > > > >>>>>>> possibility.  The reason is that the newly branched streams
> > are
> > > > not
> > > > > >>>>>>> available in the same scope as each other.  That is, if we
> > > wanted
> > > > > to
> > > > > >>>> merge
> > > > > >>>>>>> them back together again I don't see a way to do that.  The
> > KIP
> > > > > >>>> proposal
> > > > > >>>>>>> has the same issue, though - all this means is that for
> > either
> > > > > >>>> solution,
> > > > > >>>>>>> deprecating the existing branch(...) is not on the table.
> > > > > >>>>>>> Thanks,
> > > > > >>>>>>> Paul
> > > > > >>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> > > > > iponomarev@mail.ru>
> > > > > >>>> wrote:
> > > > > >>>>>>>> OK, let me summarize what we have discussed up to this
> > point.
> > > > > >>>>>>>>
> > > > > >>>>>>>> First, it seems that it's commonly agreed that branch API
> > > needs
> > > > > >>>>>>>> improvement. Motivation is given in the KIP.
> > > > > >>>>>>>>
> > > > > >>>>>>>> There are two potential ways to do it:
> > > > > >>>>>>>>
> > > > > >>>>>>>> 1. (as origianlly proposed)
> > > > > >>>>>>>>
> > > > > >>>>>>>> new KafkaStreamsBrancher<..>()
> > > > > >>>>>>>>     .branch(predicate1, ks ->..)
> > > > > >>>>>>>>     .branch(predicate2, ks->..)
> > > > > >>>>>>>>     .defaultBranch(ks->..) //optional
> > > > > >>>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns
> > its
> > > > > argument
> > > > > >>>>>>>>
> > > > > >>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't
> make
> > > > sense
> > > > > >>>> until
> > > > > >>>>>>>> all the necessary ingredients are provided.
> > > > > >>>>>>>>
> > > > > >>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
> > > > > contrasts the
> > > > > >>>>>>>> fluency of other KStream methods.
> > > > > >>>>>>>>
> > > > > >>>>>>>> 2. (as Paul proposes)
> > > > > >>>>>>>>
> > > > > >>>>>>>> stream
> > > > > >>>>>>>>     .branch(predicate1, ks ->...)
> > > > > >>>>>>>>     .branch(predicate2, ks->...)
> > > > > >>>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both
> > > > > defaultBranch(..)
> > > > > >>>> and
> > > > > >>>>>>>> noDefault() return void
> > > > > >>>>>>>>
> > > > > >>>>>>>> PROS: Generally follows the way KStreams interface is
> > defined.
> > > > > >>>>>>>>
> > > > > >>>>>>>> CONS: We need to define two terminal methods
> > > > (defaultBranch(ks->)
> > > > > and
> > > > > >>>>>>>> noDefault()). And for a user it is very easy to miss the
> > fact
> > > > > that one
> > > > > >>>>>>>> of the terminal methods should be called. If these methods
> > are
> > > > not
> > > > > >>>>>>>> called, we can throw an exception in runtime.
> > > > > >>>>>>>>
> > > > > >>>>>>>> Colleagues, what are your thoughts? Can we do better?
> > > > > >>>>>>>>
> > > > > >>>>>>>> Regards,
> > > > > >>>>>>>>
> > > > > >>>>>>>> Ivan
> > > > > >>>>>>>>
> > > > > >>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> > > > > >>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> > > > > >>>>>>>>>> Paul,
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> I see your point when you are talking about
> > > > > >>>>>>>>>> stream..branch..branch...default..
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Still, I believe that this cannot not be implemented the
> > > easy
> > > > > way.
> > > > > >>>>>>>>>> Maybe we all should think further.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Let me comment on two of your ideas.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>> user could specify a terminal method that assumes
> nothing
> > > > will
> > > > > >>>> reach
> > > > > >>>>>>>>>>> the default branch,
> > > > > >>>>>>>>>> throwing an exception if such a case occurs.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> 1) OK, apparently this should not be the only option
> > besides
> > > > > >>>>>>>>>> `default`, because there are scenarios when we want to
> > just
> > > > > silently
> > > > > >>>>>>>>>> drop the messages that didn't match any predicate. 2)
> > > Throwing
> > > > > an
> > > > > >>>>>>>>>> exception in the middle of data flow processing looks
> > like a
> > > > bad
> > > > > >>>> idea.
> > > > > >>>>>>>>>> In stream processing paradigm, I would prefer to emit a
> > > > special
> > > > > >>>>>>>>>> message to a dedicated stream. This is exactly where
> > > `default`
> > > > > can
> > > > > >>>> be
> > > > > >>>>>>>>>> used.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>> it would be fairly easily for the
> InternalTopologyBuilder
> > > to
> > > > > track
> > > > > >>>>>>>>>>> dangling
> > > > > >>>>>>>>>> branches that haven't been terminated and raise a clear
> > > error
> > > > > >>>> before it
> > > > > >>>>>>>>>> becomes an issue.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> You mean a runtime exception, when the program is
> compiled
> > > and
> > > > > run?
> > > > > >>>>>>>>>> Well,  I'd prefer an API that simply won't compile if
> used
> > > > > >>>>>>>>>> incorrectly. Can we build such an API as a method chain
> > > > starting
> > > > > >>>> from
> > > > > >>>>>>>>>> KStream object? There is a huge cost difference between
> > > > runtime
> > > > > and
> > > > > >>>>>>>>>> compile-time errors. Even if a failure uncovers
> instantly
> > on
> > > > > unit
> > > > > >>>>>>>>>> tests, it costs more for the project than a compilation
> > > > failure.
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Regards,
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> Ivan
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> > > > > >>>>>>>>>>> Ivan,
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Good point about the terminal operation being required.
> > > But
> > > > is
> > > > > >>>> that
> > > > > >>>>>>>>>>> really
> > > > > >>>>>>>>>>> such a bad thing?  If the user doesn't want a
> > defaultBranch
> > > > > they
> > > > > >>>> can
> > > > > >>>>>>>>>>> call
> > > > > >>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
> > > > > easily.  In
> > > > > >>>>>>>>>>> fact I
> > > > > >>>>>>>>>>> think it creates an opportunity for a nicer API - a
> user
> > > > could
> > > > > >>>> specify
> > > > > >>>>>>>> a
> > > > > >>>>>>>>>>> terminal method that assumes nothing will reach the
> > default
> > > > > branch,
> > > > > >>>>>>>>>>> throwing an exception if such a case occurs.  That
> seems
> > > like
> > > > > an
> > > > > >>>>>>>>>>> improvement over the current branch() API, which allows
> > for
> > > > the
> > > > > >>>> more
> > > > > >>>>>>>>>>> subtle
> > > > > >>>>>>>>>>> behavior of records unexpectedly getting dropped.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> The need for a terminal operation certainly has to be
> > well
> > > > > >>>>>>>>>>> documented, but
> > > > > >>>>>>>>>>> it would be fairly easily for the
> InternalTopologyBuilder
> > > to
> > > > > track
> > > > > >>>>>>>>>>> dangling
> > > > > >>>>>>>>>>> branches that haven't been terminated and raise a clear
> > > error
> > > > > >>>> before it
> > > > > >>>>>>>>>>> becomes an issue.  Especially now that there is a
> "build
> > > > step"
> > > > > >>>> where
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>> topology is actually wired up, when
> > StreamsBuilder.build()
> > > is
> > > > > >>>> called.
> > > > > >>>>>>>>>>> Regarding onTopOf() returning its argument, I agree
> that
> > > it's
> > > > > >>>>>>>>>>> critical to
> > > > > >>>>>>>>>>> allow users to do other operations on the input stream.
> > > With
> > > > > the
> > > > > >>>>>>>> fluent
> > > > > >>>>>>>>>>> solution, it ought to work the same way all other
> > > operations
> > > > > do -
> > > > > >>>> if
> > > > > >>>>>>>> you
> > > > > >>>>>>>>>>> want to process off the original KStream multiple
> times,
> > > you
> > > > > just
> > > > > >>>>>>>>>>> need the
> > > > > >>>>>>>>>>> stream as a variable so you can call as many operations
> > on
> > > it
> > > > > as
> > > > > >>>> you
> > > > > >>>>>>>>>>> desire.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Thoughts?
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Best,
> > > > > >>>>>>>>>>> Paul
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> > > > > iponomarev@mail.ru
> > > > > >>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>> Hello Paul,
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> I afraid this won't work because we do not always need
> > the
> > > > > >>>>>>>>>>>> defaultBranch. And without a terminal operation we
> don't
> > > > know
> > > > > >>>> when to
> > > > > >>>>>>>>>>>> finalize and build the 'branch switch'.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we
> can
> > do
> > > > > >>>> something
> > > > > >>>>>>>>>>>> more with the original branch after branching.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> I understand your point that the need of special
> object
> > > > > >>>> construction
> > > > > >>>>>>>>>>>> contrasts the fluency of most KStream methods. But
> here
> > we
> > > > > have a
> > > > > >>>>>>>>>>>> special case: we build the switch to split the flow,
> so
> > I
> > > > > think
> > > > > >>>> this
> > > > > >>>>>>>> is
> > > > > >>>>>>>>>>>> still idiomatic.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Ivan
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> > > > > >>>>>>>>>>>>> Ivan,
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> I think it's a great idea to improve this API, but I
> > find
> > > > the
> > > > > >>>>>>>>>>>>> onTopOff()
> > > > > >>>>>>>>>>>>> mechanism a little confusing since it contrasts the
> > > fluency
> > > > > of
> > > > > >>>> other
> > > > > >>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call
> a
> > > > > method on
> > > > > >>>> the
> > > > > >>>>>>>>>>>> stream
> > > > > >>>>>>>>>>>>> so it still reads top to bottom if the branch cases
> are
> > > > > defined
> > > > > >>>>>>>>>>>>> fluently.
> > > > > >>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very
> > nice
> > > > > and the
> > > > > >>>>>>>>>>>>> right
> > > > > >>>>>>>>>>>> way
> > > > > >>>>>>>>>>>>> to do things, but what if we flipped around how we
> > > specify
> > > > > the
> > > > > >>>> source
> > > > > >>>>>>>>>>>>> stream.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Like:
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> stream.branch()
> > > > > >>>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
> > > > > >>>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
> > > > > >>>>>>>>>>>>>            .defaultBranch(this::handleDefault);
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
> > > > KStreamBrancher
> > > > > or
> > > > > >>>>>>>>>>>> something,
> > > > > >>>>>>>>>>>>> which is added to by addBranch() and terminated by
> > > > > >>>> defaultBranch()
> > > > > >>>>>>>>>>>>> (which
> > > > > >>>>>>>>>>>>> returns void).  This is obviously incompatible with
> the
> > > > > current
> > > > > >>>>>>>>>>>>> API, so
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>> new stream.branch() would have to have a different
> > name,
> > > > but
> > > > > that
> > > > > >>>>>>>>>>>>> seems
> > > > > >>>>>>>>>>>>> like a fairly small problem - we could call it
> > something
> > > > like
> > > > > >>>>>>>>>>>>> branched()
> > > > > >>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It
> > seems
> > > > > like it
> > > > > >>>>>>>>>>>>> does to
> > > > > >>>>>>>>>>>>> me, allowing for clear in-line branching while also
> > > > allowing
> > > > > you
> > > > > >>>> to
> > > > > >>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams
> > if
> > > > > desired.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Thanks,
> > > > > >>>>>>>>>>>>> Paul
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> > > > > >>>>>>>>>>>> <ip...@mail.ru.invalid>
> > > > > >>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Hi Bill,
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Thank you for your reply!
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> This is how I usually do it:
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
> > > > > >>>>>>>>>>>>>>            ks.filter(....).mapValues(...)
> > > > > >>>>>>>>>>>>>> }
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
> > > > > >>>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
> > > > > >>>>>>>>>>>>>> }
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> ......
> > > > > >>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> > > > > >>>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
> > > > > >>>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
> > > > > >>>>>>>>>>>>>>       .onTopOf(....)
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Ivan
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> > > > > >>>>>>>>>>>>>>> Hi Ivan,
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Thanks for the KIP.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher
> takes a
> > > > > Consumer
> > > > > >>>> as a
> > > > > >>>>>>>>>>>>>> second
> > > > > >>>>>>>>>>>>>>> argument which returns nothing, and the example in
> > the
> > > > KIP
> > > > > >>>> shows
> > > > > >>>>>>>>>>>>>>> each
> > > > > >>>>>>>>>>>>>>> stream from the branch using a terminal node
> > > > > (KafkaStreams#to()
> > > > > >>>>>>>>>>>>>>> in this
> > > > > >>>>>>>>>>>>>>> case).
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Maybe I've missed something, but how would we
> handle
> > > the
> > > > > case
> > > > > >>>>>>>>>>>>>>> where the
> > > > > >>>>>>>>>>>>>>> user has created a branch but wants to continue
> > > > processing
> > > > > and
> > > > > >>>> not
> > > > > >>>>>>>>>>>>>>> necessarily use a terminal node on the branched
> > stream
> > > > > >>>> immediately?
> > > > > >>>>>>>>>>>>>>> For example, using today's logic as is if we had
> > > > something
> > > > > like
> > > > > >>>>>>>>>>>>>>> this:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> KStream<String, String>[] branches =
> > > > > >>>>>>>>>>>>>>> originalStream.branch(predicate1,
> > > > > >>>>>>>>>>>>>>> predicate2);
> > > > > >>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> > > > > >>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Thanks!
> > > > > >>>>>>>>>>>>>>> Bill
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
> > > > > bbejeck@gmail.com
> > > > > >>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> All,
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP-
> 418.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Here's the original message:
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Hello,
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418.
> Please
> > > > take
> > > > > a
> > > > > >>>> look
> > > > > >>>>>>>> at
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> KIP-418:
> > > > > >>>>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> > > > > >>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> > > > > >>>> https://issues.apache.org/jira/browse/KAFKA-5488
> > > > > >>>>>>>>>>>>>>>> PR#6164:
> https://github.com/apache/kafka/pull/6164
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Ivan Ponomarev
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Paul Whalen <pg...@gmail.com>.
Also +1 on the issues/goals as Michael outlined them, I think that sets a
great framework for the discussion.

Regarding the SortedMap solution, my understanding is that the current
proposal in the KIP is what is in my PR which (pending naming decisions) is
roughly this:

stream.split()
    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
    .branch(Predicate<K, V>, Consumer<KStream<K, V>>)
    .defaultBranch(Consumer<KStream<K, V>>);

Obviously some ordering is necessary, since branching as a construct
doesn't work without it, but this solution seems like it provides as much
associativity as the SortedMap solution, because each branch() call
directly associates the "conditional" with the "code block."  The value it
provides over the KIP solution is the accessing of streams in the same
scope.

The KIP solution is less "dynamic" than the SortedMap solution in the sense
that it is slightly clumsier to add a dynamic number of branches, but it is
certainly possible.  It seems to me like the API should favor the "static"
case anyway, and should make it simple and readable to fluently declare and
access your branches in-line.  It also makes it impossible to ignore a
branch, and it is possible to build an (almost) identical SortedMap
solution on top of it.

I could also see a middle ground where instead of a raw SortedMap being
taken in, branch() takes a name and not a Consumer.  Something like this:

Map<String, KStream<K, V>> branches = stream.split()
    .branch("branchOne", Predicate<K, V>)
    .branch( "branchTwo", Predicate<K, V>)
    .defaultBranch("defaultBranch", Consumer<KStream<K, V>>);

Pros for that solution:
 - accessing branched KStreams in same scope
 - no double brace initialization, hopefully slightly more readable than
SortedMap

Cons
 - downstream branch logic cannot be specified inline which makes it harder
to read top to bottom (like existing API and SortedMap, but unlike the KIP)
 - you can forget to "handle" one of the branched streams (like existing
API and SortedMap, but unlike the KIP)

(KBranchedStreams could even work *both* ways but perhaps that's overdoing
it).

Overall I'm curious how important it is to be able to easily access the
branched KStream in the same scope as the original.  It's possible that it
doesn't need to be handled directly by the API, but instead left up to the
user.  I'm sort of in the middle on it.

Paul



On Tue, Apr 30, 2019 at 8:48 PM Sophie Blee-Goldman <so...@confluent.io>
wrote:

> I'd like to +1 what Michael said about the issues with the existing branch
> method, I agree with what he's outlined and I think we should proceed by
> trying to alleviate these problems. Specifically it seems important to be
> able to cleanly access the individual branches (eg by mapping
> name->stream), which I thought was the original intention of this KIP.
>
> That said, I don't think we should so easily give in to the double brace
> anti-pattern or force ours users into it if at all possible to avoid...just
> my two cents.
>
> Cheers,
> Sophie
>
> On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
> michael.drogalis@confluent.io> wrote:
>
> > I’d like to propose a different way of thinking about this. To me, there
> > are three problems with the existing branch signature:
> >
> > 1. If you use it the way most people do, Java raises unsafe type
> warnings.
> > 2. The way in which you use the stream branches is positionally coupled
> to
> > the ordering of the conditionals.
> > 3. It is brittle to extend existing branch calls with additional code
> > paths.
> >
> > Using associative constructs instead of relying on ordered constructs
> would
> > be a stronger approach. Consider a signature that instead looks like
> this:
> >
> > Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
> > super K,? super V>>);
> >
> > Branches are given names in a map, and as a result, the API returns a
> > mapping of names to streams. The ordering of the conditionals is
> maintained
> > because it’s a sorted map. Insert order determines the order of
> evaluation.
> >
> > This solves problem 1 because there are no more varargs. It solves
> problem
> > 2 because you no longer lean on ordering to access the branch you’re
> > interested in. It solves problem 3 because you can introduce another
> > conditional by simply attaching another name to the structure, rather
> than
> > messing with the existing indices.
> >
> > One of the drawbacks is that creating the map inline is historically
> > awkward in Java. I know it’s an anti-pattern to use voluminously, but
> > double brace initialization would clean up the aesthetics.
> >
> > On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io> wrote:
> >
> > > Hi Ivan,
> > >
> > > Thanks for the update.
> > >
> > > FWIW, I agree with Matthias that the current "start branching" operator
> > is
> > > confusing when named the same way as the actual branches. "Split" seems
> > > like a good name. Alternatively, we can do without a "start branching"
> > > operator at all, and just do:
> > >
> > > stream
> > >       .branch(Predicate)
> > >       .branch(Predicate)
> > >       .defaultBranch();
> > >
> > > Tentatively, I think that this branching operation should be terminal.
> > That
> > > way, we don't create ambiguity about how to use it. That is, `branch`
> > > should return `KBranchedStream`, while `defaultBranch` is `void`, to
> > > enforce that it comes last, and that there is only one definition of
> the
> > > default branch. Potentially, we should log a warning if there's no
> > default,
> > > and additionally log a warning (or throw an exception) if a record
> falls
> > > though with no default.
> > >
> > > Thoughts?
> > >
> > > Thanks,
> > > -John
> > >
> > > On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <matthias@confluent.io
> >
> > > wrote:
> > >
> > > > Thanks for updating the KIP and your answers.
> > > >
> > > >
> > > > >  this is to make the name similar to String#split
> > > > >> that also returns an array, right?
> > > >
> > > > The intend was to avoid name duplication. The return type should
> _not_
> > > > be an array.
> > > >
> > > > The current proposal is
> > > >
> > > > stream.branch()
> > > >       .branch(Predicate)
> > > >       .branch(Predicate)
> > > >       .defaultBranch();
> > > >
> > > > IMHO, this reads a little odd, because the first `branch()` does not
> > > > take any parameters and has different semantics than the later
> > > > `branch()` calls. Note, that from the code snippet above, it's hidden
> > > > that the first call is `KStream#branch()` while the others are
> > > > `KBranchedStream#branch()` what makes reading the code harder.
> > > >
> > > > Because I suggested to rename `addBranch()` -> `branch()`, I though
> it
> > > > might be better to also rename `KStream#branch()` to avoid the naming
> > > > overlap that seems to be confusing. The following reads much cleaner
> to
> > > me:
> > > >
> > > > stream.split()
> > > >       .branch(Predicate)
> > > >       .branch(Predicate)
> > > >       .defaultBranch();
> > > >
> > > > Maybe there is a better alternative to `split()` though to avoid the
> > > > naming overlap.
> > > >
> > > >
> > > > > 'default' is, however, a reserved word, so unfortunately we cannot
> > have
> > > > a method with such name :-)
> > > >
> > > > Bummer. Didn't consider this. Maybe we can still come up with a short
> > > name?
> > > >
> > > >
> > > > Can you add the interface `KBranchedStream` to the KIP with all it's
> > > > methods? It will be part of public API and should be contained in the
> > > > KIP. For example, it's unclear atm, what the return type of
> > > > `defaultBranch()` is.
> > > >
> > > >
> > > > You did not comment on the idea to add a `KBranchedStream#get(int
> > index)
> > > > -> KStream` method to get the individually branched-KStreams. Would
> be
> > > > nice to get your feedback about it. It seems you suggest that users
> > > > would need to write custom utility code otherwise, to access them. We
> > > > should discuss the pros and cons of both approaches. It feels
> > > > "incomplete" to me atm, if the API has no built-in support to get the
> > > > branched-KStreams directly.
> > > >
> > > >
> > > >
> > > > -Matthias
> > > >
> > > >
> > > > On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> > > > > Hi all!
> > > > >
> > > > > I have updated the KIP-418 according to the new vision.
> > > > >
> > > > > Matthias, thanks for your comment!
> > > > >
> > > > >> Renaming KStream#branch() -> #split()
> > > > >
> > > > > I can see your point: this is to make the name similar to
> > String#split
> > > > > that also returns an array, right? But is it worth the loss of
> > > backwards
> > > > > compatibility? We can have overloaded branch() as well without
> > > affecting
> > > > > the existing code. Maybe the old array-based `branch` method should
> > be
> > > > > deprecated, but this is a subject for discussion.
> > > > >
> > > > >> Renaming KBranchedStream#addBranch() -> BranchingKStream#branch(),
> > > > > KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > > > >
> > > > > Totally agree with 'addBranch->branch' rename. 'default' is,
> > however, a
> > > > > reserved word, so unfortunately we cannot have a method with such
> > name
> > > > :-)
> > > > >
> > > > >> defaultBranch() does take an `Predicate` as argument, but I think
> > that
> > > > > is not required?
> > > > >
> > > > > Absolutely! I think that was just copy-paste error or something.
> > > > >
> > > > > Dear colleagues,
> > > > >
> > > > > please revise the new version of the KIP and Paul's PR
> > > > > (https://github.com/apache/kafka/pull/6512)
> > > > >
> > > > > Any new suggestions/objections?
> > > > >
> > > > > Regards,
> > > > >
> > > > > Ivan
> > > > >
> > > > >
> > > > > 11.04.2019 11:47, Matthias J. Sax пишет:
> > > > >> Thanks for driving the discussion of this KIP. It seems that
> > everybody
> > > > >> agrees that the current branch() method using arrays is not
> optimal.
> > > > >>
> > > > >> I had a quick look into the PR and I like the overall proposal.
> > There
> > > > >> are some minor things we need to consider. I would recommend the
> > > > >> following renaming:
> > > > >>
> > > > >> KStream#branch() -> #split()
> > > > >> KBranchedStream#addBranch() -> BranchingKStream#branch()
> > > > >> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > > > >>
> > > > >> It's just a suggestion to get slightly shorter method names.
> > > > >>
> > > > >> In the current PR, defaultBranch() does take an `Predicate` as
> > > argument,
> > > > >> but I think that is not required?
> > > > >>
> > > > >> Also, we should consider KIP-307, that was recently accepted and
> is
> > > > >> currently implemented:
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> > > > >>
> > > > >> Ie, we should add overloads that accepted a `Named` parameter.
> > > > >>
> > > > >>
> > > > >> For the issue that the created `KStream` object are in different
> > > scopes:
> > > > >> could we extend `KBranchedStream` with a `get(int index)` method
> > that
> > > > >> returns the corresponding "branched" result `KStream` object?
> Maybe,
> > > the
> > > > >> second argument of `addBranch()` should not be a
> `Consumer<KStream>`
> > > but
> > > > >> a `Function<KStream,KStream>` and `get()` could return whatever
> the
> > > > >> `Function` returns?
> > > > >>
> > > > >>
> > > > >> Finally, I would also suggest to update the KIP with the current
> > > > >> proposal. That makes it easier to review.
> > > > >>
> > > > >>
> > > > >> -Matthias
> > > > >>
> > > > >>
> > > > >>
> > > > >> On 3/31/19 12:22 PM, Paul Whalen wrote:
> > > > >>> Ivan,
> > > > >>>
> > > > >>> I'm a bit of a novice here as well, but I think it makes sense
> for
> > > you
> > > > to
> > > > >>> revise the KIP and continue the discussion.  Obviously we'll need
> > > some
> > > > >>> buy-in from committers that have actual binding votes on whether
> > the
> > > > KIP
> > > > >>> could be adopted.  It would be great to hear if they think this
> is
> > a
> > > > good
> > > > >>> idea overall.  I'm not sure if that happens just by starting a
> > vote,
> > > > or if
> > > > >>> there is generally some indication of interest beforehand.
> > > > >>>
> > > > >>> That being said, I'll continue the discussion a bit: assuming we
> do
> > > > move
> > > > >>> forward the solution of "stream.branch() returns
> KBranchedStream",
> > do
> > > > we
> > > > >>> deprecate "stream.branch(...) returns KStream[]"?  I would favor
> > > > >>> deprecating, since having two mutually exclusive APIs that
> > accomplish
> > > > the
> > > > >>> same thing is confusing, especially when they're fairly similar
> > > > anyway.  We
> > > > >>> just need to be sure we're not making something
> > impossible/difficult
> > > > that
> > > > >>> is currently possible/easy.
> > > > >>>
> > > > >>> Regarding my PR - I think the general structure would work, it's
> > > just a
> > > > >>> little sloppy overall in terms of naming and clarity. In
> > particular,
> > > > >>> passing in the "predicates" and "children" lists which get
> modified
> > > in
> > > > >>> KBranchedStream but read from all the way KStreamLazyBranch is a
> > bit
> > > > >>> complicated to follow.
> > > > >>>
> > > > >>> Thanks,
> > > > >>> Paul
> > > > >>>
> > > > >>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <
> iponomarev@mail.ru
> > >
> > > > wrote:
> > > > >>>
> > > > >>>> Hi Paul!
> > > > >>>>
> > > > >>>> I read your code carefully and now I am fully convinced: your
> > > proposal
> > > > >>>> looks better and should work. We just have to document the
> crucial
> > > > fact
> > > > >>>> that KStream consumers are invoked as they're added. And then
> it's
> > > all
> > > > >>>> going to be very nice.
> > > > >>>>
> > > > >>>> What shall we do now? I should re-write the KIP and resume the
> > > > >>>> discussion here, right?
> > > > >>>>
> > > > >>>> Why are you telling that your PR 'should not be even a starting
> > > point
> > > > if
> > > > >>>> we go in this direction'? To me it looks like a good starting
> > point.
> > > > But
> > > > >>>> as a novice in this project I might miss some important details.
> > > > >>>>
> > > > >>>> Regards,
> > > > >>>>
> > > > >>>> Ivan
> > > > >>>>
> > > > >>>>
> > > > >>>> 28.03.2019 17:38, Paul Whalen пишет:
> > > > >>>>> Ivan,
> > > > >>>>>
> > > > >>>>> Maybe I’m missing the point, but I believe the stream.branch()
> > > > solution
> > > > >>>> supports this. The couponIssuer::set* consumers will be invoked
> as
> > > > they’re
> > > > >>>> added, not during streamsBuilder.build(). So the user still
> ought
> > to
> > > > be
> > > > >>>> able to call couponIssuer.coupons() afterward and depend on the
> > > > branched
> > > > >>>> streams having been set.
> > > > >>>>> The issue I mean to point out is that it is hard to access the
> > > > branched
> > > > >>>> streams in the same scope as the original stream (that is, not
> > > inside
> > > > the
> > > > >>>> couponIssuer), which is a problem with both proposed solutions.
> It
> > > > can be
> > > > >>>> worked around though.
> > > > >>>>> [Also, great to hear additional interest in 401, I’m excited to
> > > hear
> > > > >>>> your thoughts!]
> > > > >>>>> Paul
> > > > >>>>>
> > > > >>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <
> iponomarev@mail.ru
> > >
> > > > wrote:
> > > > >>>>>>
> > > > >>>>>> Hi Paul!
> > > > >>>>>>
> > > > >>>>>> The idea to postpone the wiring of branches to the
> > > > >>>> streamsBuilder.build() also looked great for me at first glance,
> > but
> > > > ---
> > > > >>>>>>> the newly branched streams are not available in the same
> scope
> > as
> > > > each
> > > > >>>> other.  That is, if we wanted to merge them back together again
> I
> > > > don't see
> > > > >>>> a way to do that.
> > > > >>>>>> You just took the words right out of my mouth, I was just
> going
> > to
> > > > >>>> write in details about this issue.
> > > > >>>>>> Consider the example from Bill's book, p. 101: say we need to
> > > > identify
> > > > >>>> customers who have bought coffee and made a purchase in the
> > > > electronics
> > > > >>>> store to give them coupons.
> > > > >>>>>> This is the code I usually write under these circumstances
> using
> > > my
> > > > >>>> 'brancher' class:
> > > > >>>>>> @Setter
> > > > >>>>>> class CouponIssuer{
> > > > >>>>>>    private KStream<....> coffePurchases;
> > > > >>>>>>    private KStream<....> electronicsPurchases;
> > > > >>>>>>
> > > > >>>>>>    KStream<...> coupons(){
> > > > >>>>>>        return
> > > > coffePurchases.join(electronicsPurchases...)...whatever
> > > > >>>>>>
> > > > >>>>>>        /*In the real world the code here can be complex, so
> > > > creation of
> > > > >>>> a separate CouponIssuer class is fully justified, in order to
> > > separate
> > > > >>>> classes' responsibilities.*/
> > > > >>>>>>   }
> > > > >>>>>> }
> > > > >>>>>>
> > > > >>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> > > > >>>>>>
> > > > >>>>>> new KafkaStreamsBrancher<....>()
> > > > >>>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
> > > > >>>>>>      .branch(predicate2,
> couponIssuer::setElectronicsPurchases)
> > > > >>>>>>      .onTopOf(transactionStream);
> > > > >>>>>>
> > > > >>>>>> /*Alas, this won't work if we're going to wire up everything
> > > later,
> > > > >>>> without the terminal operation!!!*/
> > > > >>>>>> couponIssuer.coupons()...
> > > > >>>>>>
> > > > >>>>>> Does this make sense?  In order to properly initialize the
> > > > CouponIssuer
> > > > >>>> we need the terminal operation to be called before
> > > > streamsBuilder.build()
> > > > >>>> is called.
> > > > >>>>>> [BTW Paul, I just found out that your KIP-401 is essentially
> the
> > > > next
> > > > >>>> KIP I was going to write here. I have some thoughts based on my
> > > > experience,
> > > > >>>> so I will join the discussion on KIP-401 soon.]
> > > > >>>>>> Regards,
> > > > >>>>>>
> > > > >>>>>> Ivan
> > > > >>>>>>
> > > > >>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> > > > >>>>>>> Ivan,
> > > > >>>>>>> I tried to make a very rough proof of concept of a fluent API
> > > based
> > > > >>>> off of
> > > > >>>>>>> KStream here (https://github.com/apache/kafka/pull/6512),
> and
> > I
> > > > think
> > > > >>>> I
> > > > >>>>>>> succeeded at removing both cons.
> > > > >>>>>>>     - Compatibility: I was incorrect earlier about
> > compatibility
> > > > >>>> issues,
> > > > >>>>>>>     there aren't any direct ones.  I was unaware that Java is
> > > smart
> > > > >>>> enough to
> > > > >>>>>>>     distinguish between a branch(varargs...) returning one
> > thing
> > > > and
> > > > >>>> branch()
> > > > >>>>>>>     with no arguments returning another thing.
> > > > >>>>>>>     - Requiring a terminal method: We don't actually need it.
> > We
> > > > can
> > > > >>>> just
> > > > >>>>>>>     build up the branches in the KBranchedStream who shares
> its
> > > > state
> > > > >>>> with the
> > > > >>>>>>>     ProcessorSupplier that will actually do the branching.
> > It's
> > > > not
> > > > >>>> terribly
> > > > >>>>>>>     pretty in its current form, but I think it demonstrates
> its
> > > > >>>> feasibility.
> > > > >>>>>>> To be clear, I don't think that pull request should be final
> or
> > > > even a
> > > > >>>>>>> starting point if we go in this direction, I just wanted to
> see
> > > how
> > > > >>>>>>> challenging it would be to get the API working.
> > > > >>>>>>> I will say though, that I'm not sure the existing solution
> > could
> > > be
> > > > >>>>>>> deprecated in favor of this, which I had originally suggested
> > > was a
> > > > >>>>>>> possibility.  The reason is that the newly branched streams
> are
> > > not
> > > > >>>>>>> available in the same scope as each other.  That is, if we
> > wanted
> > > > to
> > > > >>>> merge
> > > > >>>>>>> them back together again I don't see a way to do that.  The
> KIP
> > > > >>>> proposal
> > > > >>>>>>> has the same issue, though - all this means is that for
> either
> > > > >>>> solution,
> > > > >>>>>>> deprecating the existing branch(...) is not on the table.
> > > > >>>>>>> Thanks,
> > > > >>>>>>> Paul
> > > > >>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> > > > iponomarev@mail.ru>
> > > > >>>> wrote:
> > > > >>>>>>>> OK, let me summarize what we have discussed up to this
> point.
> > > > >>>>>>>>
> > > > >>>>>>>> First, it seems that it's commonly agreed that branch API
> > needs
> > > > >>>>>>>> improvement. Motivation is given in the KIP.
> > > > >>>>>>>>
> > > > >>>>>>>> There are two potential ways to do it:
> > > > >>>>>>>>
> > > > >>>>>>>> 1. (as origianlly proposed)
> > > > >>>>>>>>
> > > > >>>>>>>> new KafkaStreamsBrancher<..>()
> > > > >>>>>>>>     .branch(predicate1, ks ->..)
> > > > >>>>>>>>     .branch(predicate2, ks->..)
> > > > >>>>>>>>     .defaultBranch(ks->..) //optional
> > > > >>>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns
> its
> > > > argument
> > > > >>>>>>>>
> > > > >>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't make
> > > sense
> > > > >>>> until
> > > > >>>>>>>> all the necessary ingredients are provided.
> > > > >>>>>>>>
> > > > >>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
> > > > contrasts the
> > > > >>>>>>>> fluency of other KStream methods.
> > > > >>>>>>>>
> > > > >>>>>>>> 2. (as Paul proposes)
> > > > >>>>>>>>
> > > > >>>>>>>> stream
> > > > >>>>>>>>     .branch(predicate1, ks ->...)
> > > > >>>>>>>>     .branch(predicate2, ks->...)
> > > > >>>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both
> > > > defaultBranch(..)
> > > > >>>> and
> > > > >>>>>>>> noDefault() return void
> > > > >>>>>>>>
> > > > >>>>>>>> PROS: Generally follows the way KStreams interface is
> defined.
> > > > >>>>>>>>
> > > > >>>>>>>> CONS: We need to define two terminal methods
> > > (defaultBranch(ks->)
> > > > and
> > > > >>>>>>>> noDefault()). And for a user it is very easy to miss the
> fact
> > > > that one
> > > > >>>>>>>> of the terminal methods should be called. If these methods
> are
> > > not
> > > > >>>>>>>> called, we can throw an exception in runtime.
> > > > >>>>>>>>
> > > > >>>>>>>> Colleagues, what are your thoughts? Can we do better?
> > > > >>>>>>>>
> > > > >>>>>>>> Regards,
> > > > >>>>>>>>
> > > > >>>>>>>> Ivan
> > > > >>>>>>>>
> > > > >>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> > > > >>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> > > > >>>>>>>>>> Paul,
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> I see your point when you are talking about
> > > > >>>>>>>>>> stream..branch..branch...default..
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Still, I believe that this cannot not be implemented the
> > easy
> > > > way.
> > > > >>>>>>>>>> Maybe we all should think further.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Let me comment on two of your ideas.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>> user could specify a terminal method that assumes nothing
> > > will
> > > > >>>> reach
> > > > >>>>>>>>>>> the default branch,
> > > > >>>>>>>>>> throwing an exception if such a case occurs.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 1) OK, apparently this should not be the only option
> besides
> > > > >>>>>>>>>> `default`, because there are scenarios when we want to
> just
> > > > silently
> > > > >>>>>>>>>> drop the messages that didn't match any predicate. 2)
> > Throwing
> > > > an
> > > > >>>>>>>>>> exception in the middle of data flow processing looks
> like a
> > > bad
> > > > >>>> idea.
> > > > >>>>>>>>>> In stream processing paradigm, I would prefer to emit a
> > > special
> > > > >>>>>>>>>> message to a dedicated stream. This is exactly where
> > `default`
> > > > can
> > > > >>>> be
> > > > >>>>>>>>>> used.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder
> > to
> > > > track
> > > > >>>>>>>>>>> dangling
> > > > >>>>>>>>>> branches that haven't been terminated and raise a clear
> > error
> > > > >>>> before it
> > > > >>>>>>>>>> becomes an issue.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> You mean a runtime exception, when the program is compiled
> > and
> > > > run?
> > > > >>>>>>>>>> Well,  I'd prefer an API that simply won't compile if used
> > > > >>>>>>>>>> incorrectly. Can we build such an API as a method chain
> > > starting
> > > > >>>> from
> > > > >>>>>>>>>> KStream object? There is a huge cost difference between
> > > runtime
> > > > and
> > > > >>>>>>>>>> compile-time errors. Even if a failure uncovers instantly
> on
> > > > unit
> > > > >>>>>>>>>> tests, it costs more for the project than a compilation
> > > failure.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Regards,
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Ivan
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> > > > >>>>>>>>>>> Ivan,
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Good point about the terminal operation being required.
> > But
> > > is
> > > > >>>> that
> > > > >>>>>>>>>>> really
> > > > >>>>>>>>>>> such a bad thing?  If the user doesn't want a
> defaultBranch
> > > > they
> > > > >>>> can
> > > > >>>>>>>>>>> call
> > > > >>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
> > > > easily.  In
> > > > >>>>>>>>>>> fact I
> > > > >>>>>>>>>>> think it creates an opportunity for a nicer API - a user
> > > could
> > > > >>>> specify
> > > > >>>>>>>> a
> > > > >>>>>>>>>>> terminal method that assumes nothing will reach the
> default
> > > > branch,
> > > > >>>>>>>>>>> throwing an exception if such a case occurs.  That seems
> > like
> > > > an
> > > > >>>>>>>>>>> improvement over the current branch() API, which allows
> for
> > > the
> > > > >>>> more
> > > > >>>>>>>>>>> subtle
> > > > >>>>>>>>>>> behavior of records unexpectedly getting dropped.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> The need for a terminal operation certainly has to be
> well
> > > > >>>>>>>>>>> documented, but
> > > > >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder
> > to
> > > > track
> > > > >>>>>>>>>>> dangling
> > > > >>>>>>>>>>> branches that haven't been terminated and raise a clear
> > error
> > > > >>>> before it
> > > > >>>>>>>>>>> becomes an issue.  Especially now that there is a "build
> > > step"
> > > > >>>> where
> > > > >>>>>>>> the
> > > > >>>>>>>>>>> topology is actually wired up, when
> StreamsBuilder.build()
> > is
> > > > >>>> called.
> > > > >>>>>>>>>>> Regarding onTopOf() returning its argument, I agree that
> > it's
> > > > >>>>>>>>>>> critical to
> > > > >>>>>>>>>>> allow users to do other operations on the input stream.
> > With
> > > > the
> > > > >>>>>>>> fluent
> > > > >>>>>>>>>>> solution, it ought to work the same way all other
> > operations
> > > > do -
> > > > >>>> if
> > > > >>>>>>>> you
> > > > >>>>>>>>>>> want to process off the original KStream multiple times,
> > you
> > > > just
> > > > >>>>>>>>>>> need the
> > > > >>>>>>>>>>> stream as a variable so you can call as many operations
> on
> > it
> > > > as
> > > > >>>> you
> > > > >>>>>>>>>>> desire.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Thoughts?
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Best,
> > > > >>>>>>>>>>> Paul
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> > > > iponomarev@mail.ru
> > > > >>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> Hello Paul,
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> I afraid this won't work because we do not always need
> the
> > > > >>>>>>>>>>>> defaultBranch. And without a terminal operation we don't
> > > know
> > > > >>>> when to
> > > > >>>>>>>>>>>> finalize and build the 'branch switch'.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we can
> do
> > > > >>>> something
> > > > >>>>>>>>>>>> more with the original branch after branching.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> I understand your point that the need of special object
> > > > >>>> construction
> > > > >>>>>>>>>>>> contrasts the fluency of most KStream methods. But here
> we
> > > > have a
> > > > >>>>>>>>>>>> special case: we build the switch to split the flow, so
> I
> > > > think
> > > > >>>> this
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>> still idiomatic.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Ivan
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> > > > >>>>>>>>>>>>> Ivan,
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> I think it's a great idea to improve this API, but I
> find
> > > the
> > > > >>>>>>>>>>>>> onTopOff()
> > > > >>>>>>>>>>>>> mechanism a little confusing since it contrasts the
> > fluency
> > > > of
> > > > >>>> other
> > > > >>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call a
> > > > method on
> > > > >>>> the
> > > > >>>>>>>>>>>> stream
> > > > >>>>>>>>>>>>> so it still reads top to bottom if the branch cases are
> > > > defined
> > > > >>>>>>>>>>>>> fluently.
> > > > >>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very
> nice
> > > > and the
> > > > >>>>>>>>>>>>> right
> > > > >>>>>>>>>>>> way
> > > > >>>>>>>>>>>>> to do things, but what if we flipped around how we
> > specify
> > > > the
> > > > >>>> source
> > > > >>>>>>>>>>>>> stream.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Like:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> stream.branch()
> > > > >>>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
> > > > >>>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
> > > > >>>>>>>>>>>>>            .defaultBranch(this::handleDefault);
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
> > > KStreamBrancher
> > > > or
> > > > >>>>>>>>>>>> something,
> > > > >>>>>>>>>>>>> which is added to by addBranch() and terminated by
> > > > >>>> defaultBranch()
> > > > >>>>>>>>>>>>> (which
> > > > >>>>>>>>>>>>> returns void).  This is obviously incompatible with the
> > > > current
> > > > >>>>>>>>>>>>> API, so
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>> new stream.branch() would have to have a different
> name,
> > > but
> > > > that
> > > > >>>>>>>>>>>>> seems
> > > > >>>>>>>>>>>>> like a fairly small problem - we could call it
> something
> > > like
> > > > >>>>>>>>>>>>> branched()
> > > > >>>>>>>>>>>> or
> > > > >>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It
> seems
> > > > like it
> > > > >>>>>>>>>>>>> does to
> > > > >>>>>>>>>>>>> me, allowing for clear in-line branching while also
> > > allowing
> > > > you
> > > > >>>> to
> > > > >>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams
> if
> > > > desired.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Thanks,
> > > > >>>>>>>>>>>>> Paul
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> > > > >>>>>>>>>>>> <ip...@mail.ru.invalid>
> > > > >>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Hi Bill,
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Thank you for your reply!
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> This is how I usually do it:
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
> > > > >>>>>>>>>>>>>>            ks.filter(....).mapValues(...)
> > > > >>>>>>>>>>>>>> }
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
> > > > >>>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
> > > > >>>>>>>>>>>>>> }
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> ......
> > > > >>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> > > > >>>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
> > > > >>>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
> > > > >>>>>>>>>>>>>>       .onTopOf(....)
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Ivan
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> > > > >>>>>>>>>>>>>>> Hi Ivan,
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Thanks for the KIP.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher takes a
> > > > Consumer
> > > > >>>> as a
> > > > >>>>>>>>>>>>>> second
> > > > >>>>>>>>>>>>>>> argument which returns nothing, and the example in
> the
> > > KIP
> > > > >>>> shows
> > > > >>>>>>>>>>>>>>> each
> > > > >>>>>>>>>>>>>>> stream from the branch using a terminal node
> > > > (KafkaStreams#to()
> > > > >>>>>>>>>>>>>>> in this
> > > > >>>>>>>>>>>>>>> case).
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Maybe I've missed something, but how would we handle
> > the
> > > > case
> > > > >>>>>>>>>>>>>>> where the
> > > > >>>>>>>>>>>>>>> user has created a branch but wants to continue
> > > processing
> > > > and
> > > > >>>> not
> > > > >>>>>>>>>>>>>>> necessarily use a terminal node on the branched
> stream
> > > > >>>> immediately?
> > > > >>>>>>>>>>>>>>> For example, using today's logic as is if we had
> > > something
> > > > like
> > > > >>>>>>>>>>>>>>> this:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> KStream<String, String>[] branches =
> > > > >>>>>>>>>>>>>>> originalStream.branch(predicate1,
> > > > >>>>>>>>>>>>>>> predicate2);
> > > > >>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> > > > >>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Thanks!
> > > > >>>>>>>>>>>>>>> Bill
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
> > > > bbejeck@gmail.com
> > > > >>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> All,
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP- 418.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Here's the original message:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Hello,
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418. Please
> > > take
> > > > a
> > > > >>>> look
> > > > >>>>>>>> at
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> KIP-418:
> > > > >>>>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> > > > >>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> > > > >>>> https://issues.apache.org/jira/browse/KAFKA-5488
> > > > >>>>>>>>>>>>>>>> PR#6164: https://github.com/apache/kafka/pull/6164
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Ivan Ponomarev
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >
> > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Sophie Blee-Goldman <so...@confluent.io>.
I'd like to +1 what Michael said about the issues with the existing branch
method, I agree with what he's outlined and I think we should proceed by
trying to alleviate these problems. Specifically it seems important to be
able to cleanly access the individual branches (eg by mapping
name->stream), which I thought was the original intention of this KIP.

That said, I don't think we should so easily give in to the double brace
anti-pattern or force ours users into it if at all possible to avoid...just
my two cents.

Cheers,
Sophie

On Tue, Apr 30, 2019 at 12:59 PM Michael Drogalis <
michael.drogalis@confluent.io> wrote:

> I’d like to propose a different way of thinking about this. To me, there
> are three problems with the existing branch signature:
>
> 1. If you use it the way most people do, Java raises unsafe type warnings.
> 2. The way in which you use the stream branches is positionally coupled to
> the ordering of the conditionals.
> 3. It is brittle to extend existing branch calls with additional code
> paths.
>
> Using associative constructs instead of relying on ordered constructs would
> be a stronger approach. Consider a signature that instead looks like this:
>
> Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
> super K,? super V>>);
>
> Branches are given names in a map, and as a result, the API returns a
> mapping of names to streams. The ordering of the conditionals is maintained
> because it’s a sorted map. Insert order determines the order of evaluation.
>
> This solves problem 1 because there are no more varargs. It solves problem
> 2 because you no longer lean on ordering to access the branch you’re
> interested in. It solves problem 3 because you can introduce another
> conditional by simply attaching another name to the structure, rather than
> messing with the existing indices.
>
> One of the drawbacks is that creating the map inline is historically
> awkward in Java. I know it’s an anti-pattern to use voluminously, but
> double brace initialization would clean up the aesthetics.
>
> On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io> wrote:
>
> > Hi Ivan,
> >
> > Thanks for the update.
> >
> > FWIW, I agree with Matthias that the current "start branching" operator
> is
> > confusing when named the same way as the actual branches. "Split" seems
> > like a good name. Alternatively, we can do without a "start branching"
> > operator at all, and just do:
> >
> > stream
> >       .branch(Predicate)
> >       .branch(Predicate)
> >       .defaultBranch();
> >
> > Tentatively, I think that this branching operation should be terminal.
> That
> > way, we don't create ambiguity about how to use it. That is, `branch`
> > should return `KBranchedStream`, while `defaultBranch` is `void`, to
> > enforce that it comes last, and that there is only one definition of the
> > default branch. Potentially, we should log a warning if there's no
> default,
> > and additionally log a warning (or throw an exception) if a record falls
> > though with no default.
> >
> > Thoughts?
> >
> > Thanks,
> > -John
> >
> > On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > Thanks for updating the KIP and your answers.
> > >
> > >
> > > >  this is to make the name similar to String#split
> > > >> that also returns an array, right?
> > >
> > > The intend was to avoid name duplication. The return type should _not_
> > > be an array.
> > >
> > > The current proposal is
> > >
> > > stream.branch()
> > >       .branch(Predicate)
> > >       .branch(Predicate)
> > >       .defaultBranch();
> > >
> > > IMHO, this reads a little odd, because the first `branch()` does not
> > > take any parameters and has different semantics than the later
> > > `branch()` calls. Note, that from the code snippet above, it's hidden
> > > that the first call is `KStream#branch()` while the others are
> > > `KBranchedStream#branch()` what makes reading the code harder.
> > >
> > > Because I suggested to rename `addBranch()` -> `branch()`, I though it
> > > might be better to also rename `KStream#branch()` to avoid the naming
> > > overlap that seems to be confusing. The following reads much cleaner to
> > me:
> > >
> > > stream.split()
> > >       .branch(Predicate)
> > >       .branch(Predicate)
> > >       .defaultBranch();
> > >
> > > Maybe there is a better alternative to `split()` though to avoid the
> > > naming overlap.
> > >
> > >
> > > > 'default' is, however, a reserved word, so unfortunately we cannot
> have
> > > a method with such name :-)
> > >
> > > Bummer. Didn't consider this. Maybe we can still come up with a short
> > name?
> > >
> > >
> > > Can you add the interface `KBranchedStream` to the KIP with all it's
> > > methods? It will be part of public API and should be contained in the
> > > KIP. For example, it's unclear atm, what the return type of
> > > `defaultBranch()` is.
> > >
> > >
> > > You did not comment on the idea to add a `KBranchedStream#get(int
> index)
> > > -> KStream` method to get the individually branched-KStreams. Would be
> > > nice to get your feedback about it. It seems you suggest that users
> > > would need to write custom utility code otherwise, to access them. We
> > > should discuss the pros and cons of both approaches. It feels
> > > "incomplete" to me atm, if the API has no built-in support to get the
> > > branched-KStreams directly.
> > >
> > >
> > >
> > > -Matthias
> > >
> > >
> > > On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> > > > Hi all!
> > > >
> > > > I have updated the KIP-418 according to the new vision.
> > > >
> > > > Matthias, thanks for your comment!
> > > >
> > > >> Renaming KStream#branch() -> #split()
> > > >
> > > > I can see your point: this is to make the name similar to
> String#split
> > > > that also returns an array, right? But is it worth the loss of
> > backwards
> > > > compatibility? We can have overloaded branch() as well without
> > affecting
> > > > the existing code. Maybe the old array-based `branch` method should
> be
> > > > deprecated, but this is a subject for discussion.
> > > >
> > > >> Renaming KBranchedStream#addBranch() -> BranchingKStream#branch(),
> > > > KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > > >
> > > > Totally agree with 'addBranch->branch' rename. 'default' is,
> however, a
> > > > reserved word, so unfortunately we cannot have a method with such
> name
> > > :-)
> > > >
> > > >> defaultBranch() does take an `Predicate` as argument, but I think
> that
> > > > is not required?
> > > >
> > > > Absolutely! I think that was just copy-paste error or something.
> > > >
> > > > Dear colleagues,
> > > >
> > > > please revise the new version of the KIP and Paul's PR
> > > > (https://github.com/apache/kafka/pull/6512)
> > > >
> > > > Any new suggestions/objections?
> > > >
> > > > Regards,
> > > >
> > > > Ivan
> > > >
> > > >
> > > > 11.04.2019 11:47, Matthias J. Sax пишет:
> > > >> Thanks for driving the discussion of this KIP. It seems that
> everybody
> > > >> agrees that the current branch() method using arrays is not optimal.
> > > >>
> > > >> I had a quick look into the PR and I like the overall proposal.
> There
> > > >> are some minor things we need to consider. I would recommend the
> > > >> following renaming:
> > > >>
> > > >> KStream#branch() -> #split()
> > > >> KBranchedStream#addBranch() -> BranchingKStream#branch()
> > > >> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > > >>
> > > >> It's just a suggestion to get slightly shorter method names.
> > > >>
> > > >> In the current PR, defaultBranch() does take an `Predicate` as
> > argument,
> > > >> but I think that is not required?
> > > >>
> > > >> Also, we should consider KIP-307, that was recently accepted and is
> > > >> currently implemented:
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> > > >>
> > > >> Ie, we should add overloads that accepted a `Named` parameter.
> > > >>
> > > >>
> > > >> For the issue that the created `KStream` object are in different
> > scopes:
> > > >> could we extend `KBranchedStream` with a `get(int index)` method
> that
> > > >> returns the corresponding "branched" result `KStream` object? Maybe,
> > the
> > > >> second argument of `addBranch()` should not be a `Consumer<KStream>`
> > but
> > > >> a `Function<KStream,KStream>` and `get()` could return whatever the
> > > >> `Function` returns?
> > > >>
> > > >>
> > > >> Finally, I would also suggest to update the KIP with the current
> > > >> proposal. That makes it easier to review.
> > > >>
> > > >>
> > > >> -Matthias
> > > >>
> > > >>
> > > >>
> > > >> On 3/31/19 12:22 PM, Paul Whalen wrote:
> > > >>> Ivan,
> > > >>>
> > > >>> I'm a bit of a novice here as well, but I think it makes sense for
> > you
> > > to
> > > >>> revise the KIP and continue the discussion.  Obviously we'll need
> > some
> > > >>> buy-in from committers that have actual binding votes on whether
> the
> > > KIP
> > > >>> could be adopted.  It would be great to hear if they think this is
> a
> > > good
> > > >>> idea overall.  I'm not sure if that happens just by starting a
> vote,
> > > or if
> > > >>> there is generally some indication of interest beforehand.
> > > >>>
> > > >>> That being said, I'll continue the discussion a bit: assuming we do
> > > move
> > > >>> forward the solution of "stream.branch() returns KBranchedStream",
> do
> > > we
> > > >>> deprecate "stream.branch(...) returns KStream[]"?  I would favor
> > > >>> deprecating, since having two mutually exclusive APIs that
> accomplish
> > > the
> > > >>> same thing is confusing, especially when they're fairly similar
> > > anyway.  We
> > > >>> just need to be sure we're not making something
> impossible/difficult
> > > that
> > > >>> is currently possible/easy.
> > > >>>
> > > >>> Regarding my PR - I think the general structure would work, it's
> > just a
> > > >>> little sloppy overall in terms of naming and clarity. In
> particular,
> > > >>> passing in the "predicates" and "children" lists which get modified
> > in
> > > >>> KBranchedStream but read from all the way KStreamLazyBranch is a
> bit
> > > >>> complicated to follow.
> > > >>>
> > > >>> Thanks,
> > > >>> Paul
> > > >>>
> > > >>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <iponomarev@mail.ru
> >
> > > wrote:
> > > >>>
> > > >>>> Hi Paul!
> > > >>>>
> > > >>>> I read your code carefully and now I am fully convinced: your
> > proposal
> > > >>>> looks better and should work. We just have to document the crucial
> > > fact
> > > >>>> that KStream consumers are invoked as they're added. And then it's
> > all
> > > >>>> going to be very nice.
> > > >>>>
> > > >>>> What shall we do now? I should re-write the KIP and resume the
> > > >>>> discussion here, right?
> > > >>>>
> > > >>>> Why are you telling that your PR 'should not be even a starting
> > point
> > > if
> > > >>>> we go in this direction'? To me it looks like a good starting
> point.
> > > But
> > > >>>> as a novice in this project I might miss some important details.
> > > >>>>
> > > >>>> Regards,
> > > >>>>
> > > >>>> Ivan
> > > >>>>
> > > >>>>
> > > >>>> 28.03.2019 17:38, Paul Whalen пишет:
> > > >>>>> Ivan,
> > > >>>>>
> > > >>>>> Maybe I’m missing the point, but I believe the stream.branch()
> > > solution
> > > >>>> supports this. The couponIssuer::set* consumers will be invoked as
> > > they’re
> > > >>>> added, not during streamsBuilder.build(). So the user still ought
> to
> > > be
> > > >>>> able to call couponIssuer.coupons() afterward and depend on the
> > > branched
> > > >>>> streams having been set.
> > > >>>>> The issue I mean to point out is that it is hard to access the
> > > branched
> > > >>>> streams in the same scope as the original stream (that is, not
> > inside
> > > the
> > > >>>> couponIssuer), which is a problem with both proposed solutions. It
> > > can be
> > > >>>> worked around though.
> > > >>>>> [Also, great to hear additional interest in 401, I’m excited to
> > hear
> > > >>>> your thoughts!]
> > > >>>>> Paul
> > > >>>>>
> > > >>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <iponomarev@mail.ru
> >
> > > wrote:
> > > >>>>>>
> > > >>>>>> Hi Paul!
> > > >>>>>>
> > > >>>>>> The idea to postpone the wiring of branches to the
> > > >>>> streamsBuilder.build() also looked great for me at first glance,
> but
> > > ---
> > > >>>>>>> the newly branched streams are not available in the same scope
> as
> > > each
> > > >>>> other.  That is, if we wanted to merge them back together again I
> > > don't see
> > > >>>> a way to do that.
> > > >>>>>> You just took the words right out of my mouth, I was just going
> to
> > > >>>> write in details about this issue.
> > > >>>>>> Consider the example from Bill's book, p. 101: say we need to
> > > identify
> > > >>>> customers who have bought coffee and made a purchase in the
> > > electronics
> > > >>>> store to give them coupons.
> > > >>>>>> This is the code I usually write under these circumstances using
> > my
> > > >>>> 'brancher' class:
> > > >>>>>> @Setter
> > > >>>>>> class CouponIssuer{
> > > >>>>>>    private KStream<....> coffePurchases;
> > > >>>>>>    private KStream<....> electronicsPurchases;
> > > >>>>>>
> > > >>>>>>    KStream<...> coupons(){
> > > >>>>>>        return
> > > coffePurchases.join(electronicsPurchases...)...whatever
> > > >>>>>>
> > > >>>>>>        /*In the real world the code here can be complex, so
> > > creation of
> > > >>>> a separate CouponIssuer class is fully justified, in order to
> > separate
> > > >>>> classes' responsibilities.*/
> > > >>>>>>   }
> > > >>>>>> }
> > > >>>>>>
> > > >>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> > > >>>>>>
> > > >>>>>> new KafkaStreamsBrancher<....>()
> > > >>>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
> > > >>>>>>      .branch(predicate2, couponIssuer::setElectronicsPurchases)
> > > >>>>>>      .onTopOf(transactionStream);
> > > >>>>>>
> > > >>>>>> /*Alas, this won't work if we're going to wire up everything
> > later,
> > > >>>> without the terminal operation!!!*/
> > > >>>>>> couponIssuer.coupons()...
> > > >>>>>>
> > > >>>>>> Does this make sense?  In order to properly initialize the
> > > CouponIssuer
> > > >>>> we need the terminal operation to be called before
> > > streamsBuilder.build()
> > > >>>> is called.
> > > >>>>>> [BTW Paul, I just found out that your KIP-401 is essentially the
> > > next
> > > >>>> KIP I was going to write here. I have some thoughts based on my
> > > experience,
> > > >>>> so I will join the discussion on KIP-401 soon.]
> > > >>>>>> Regards,
> > > >>>>>>
> > > >>>>>> Ivan
> > > >>>>>>
> > > >>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> > > >>>>>>> Ivan,
> > > >>>>>>> I tried to make a very rough proof of concept of a fluent API
> > based
> > > >>>> off of
> > > >>>>>>> KStream here (https://github.com/apache/kafka/pull/6512), and
> I
> > > think
> > > >>>> I
> > > >>>>>>> succeeded at removing both cons.
> > > >>>>>>>     - Compatibility: I was incorrect earlier about
> compatibility
> > > >>>> issues,
> > > >>>>>>>     there aren't any direct ones.  I was unaware that Java is
> > smart
> > > >>>> enough to
> > > >>>>>>>     distinguish between a branch(varargs...) returning one
> thing
> > > and
> > > >>>> branch()
> > > >>>>>>>     with no arguments returning another thing.
> > > >>>>>>>     - Requiring a terminal method: We don't actually need it.
> We
> > > can
> > > >>>> just
> > > >>>>>>>     build up the branches in the KBranchedStream who shares its
> > > state
> > > >>>> with the
> > > >>>>>>>     ProcessorSupplier that will actually do the branching.
> It's
> > > not
> > > >>>> terribly
> > > >>>>>>>     pretty in its current form, but I think it demonstrates its
> > > >>>> feasibility.
> > > >>>>>>> To be clear, I don't think that pull request should be final or
> > > even a
> > > >>>>>>> starting point if we go in this direction, I just wanted to see
> > how
> > > >>>>>>> challenging it would be to get the API working.
> > > >>>>>>> I will say though, that I'm not sure the existing solution
> could
> > be
> > > >>>>>>> deprecated in favor of this, which I had originally suggested
> > was a
> > > >>>>>>> possibility.  The reason is that the newly branched streams are
> > not
> > > >>>>>>> available in the same scope as each other.  That is, if we
> wanted
> > > to
> > > >>>> merge
> > > >>>>>>> them back together again I don't see a way to do that.  The KIP
> > > >>>> proposal
> > > >>>>>>> has the same issue, though - all this means is that for either
> > > >>>> solution,
> > > >>>>>>> deprecating the existing branch(...) is not on the table.
> > > >>>>>>> Thanks,
> > > >>>>>>> Paul
> > > >>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> > > iponomarev@mail.ru>
> > > >>>> wrote:
> > > >>>>>>>> OK, let me summarize what we have discussed up to this point.
> > > >>>>>>>>
> > > >>>>>>>> First, it seems that it's commonly agreed that branch API
> needs
> > > >>>>>>>> improvement. Motivation is given in the KIP.
> > > >>>>>>>>
> > > >>>>>>>> There are two potential ways to do it:
> > > >>>>>>>>
> > > >>>>>>>> 1. (as origianlly proposed)
> > > >>>>>>>>
> > > >>>>>>>> new KafkaStreamsBrancher<..>()
> > > >>>>>>>>     .branch(predicate1, ks ->..)
> > > >>>>>>>>     .branch(predicate2, ks->..)
> > > >>>>>>>>     .defaultBranch(ks->..) //optional
> > > >>>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns its
> > > argument
> > > >>>>>>>>
> > > >>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't make
> > sense
> > > >>>> until
> > > >>>>>>>> all the necessary ingredients are provided.
> > > >>>>>>>>
> > > >>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
> > > contrasts the
> > > >>>>>>>> fluency of other KStream methods.
> > > >>>>>>>>
> > > >>>>>>>> 2. (as Paul proposes)
> > > >>>>>>>>
> > > >>>>>>>> stream
> > > >>>>>>>>     .branch(predicate1, ks ->...)
> > > >>>>>>>>     .branch(predicate2, ks->...)
> > > >>>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both
> > > defaultBranch(..)
> > > >>>> and
> > > >>>>>>>> noDefault() return void
> > > >>>>>>>>
> > > >>>>>>>> PROS: Generally follows the way KStreams interface is defined.
> > > >>>>>>>>
> > > >>>>>>>> CONS: We need to define two terminal methods
> > (defaultBranch(ks->)
> > > and
> > > >>>>>>>> noDefault()). And for a user it is very easy to miss the fact
> > > that one
> > > >>>>>>>> of the terminal methods should be called. If these methods are
> > not
> > > >>>>>>>> called, we can throw an exception in runtime.
> > > >>>>>>>>
> > > >>>>>>>> Colleagues, what are your thoughts? Can we do better?
> > > >>>>>>>>
> > > >>>>>>>> Regards,
> > > >>>>>>>>
> > > >>>>>>>> Ivan
> > > >>>>>>>>
> > > >>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> > > >>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> > > >>>>>>>>>> Paul,
> > > >>>>>>>>>>
> > > >>>>>>>>>> I see your point when you are talking about
> > > >>>>>>>>>> stream..branch..branch...default..
> > > >>>>>>>>>>
> > > >>>>>>>>>> Still, I believe that this cannot not be implemented the
> easy
> > > way.
> > > >>>>>>>>>> Maybe we all should think further.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Let me comment on two of your ideas.
> > > >>>>>>>>>>
> > > >>>>>>>>>>> user could specify a terminal method that assumes nothing
> > will
> > > >>>> reach
> > > >>>>>>>>>>> the default branch,
> > > >>>>>>>>>> throwing an exception if such a case occurs.
> > > >>>>>>>>>>
> > > >>>>>>>>>> 1) OK, apparently this should not be the only option besides
> > > >>>>>>>>>> `default`, because there are scenarios when we want to just
> > > silently
> > > >>>>>>>>>> drop the messages that didn't match any predicate. 2)
> Throwing
> > > an
> > > >>>>>>>>>> exception in the middle of data flow processing looks like a
> > bad
> > > >>>> idea.
> > > >>>>>>>>>> In stream processing paradigm, I would prefer to emit a
> > special
> > > >>>>>>>>>> message to a dedicated stream. This is exactly where
> `default`
> > > can
> > > >>>> be
> > > >>>>>>>>>> used.
> > > >>>>>>>>>>
> > > >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder
> to
> > > track
> > > >>>>>>>>>>> dangling
> > > >>>>>>>>>> branches that haven't been terminated and raise a clear
> error
> > > >>>> before it
> > > >>>>>>>>>> becomes an issue.
> > > >>>>>>>>>>
> > > >>>>>>>>>> You mean a runtime exception, when the program is compiled
> and
> > > run?
> > > >>>>>>>>>> Well,  I'd prefer an API that simply won't compile if used
> > > >>>>>>>>>> incorrectly. Can we build such an API as a method chain
> > starting
> > > >>>> from
> > > >>>>>>>>>> KStream object? There is a huge cost difference between
> > runtime
> > > and
> > > >>>>>>>>>> compile-time errors. Even if a failure uncovers instantly on
> > > unit
> > > >>>>>>>>>> tests, it costs more for the project than a compilation
> > failure.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Regards,
> > > >>>>>>>>>>
> > > >>>>>>>>>> Ivan
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> > > >>>>>>>>>>> Ivan,
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Good point about the terminal operation being required.
> But
> > is
> > > >>>> that
> > > >>>>>>>>>>> really
> > > >>>>>>>>>>> such a bad thing?  If the user doesn't want a defaultBranch
> > > they
> > > >>>> can
> > > >>>>>>>>>>> call
> > > >>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
> > > easily.  In
> > > >>>>>>>>>>> fact I
> > > >>>>>>>>>>> think it creates an opportunity for a nicer API - a user
> > could
> > > >>>> specify
> > > >>>>>>>> a
> > > >>>>>>>>>>> terminal method that assumes nothing will reach the default
> > > branch,
> > > >>>>>>>>>>> throwing an exception if such a case occurs.  That seems
> like
> > > an
> > > >>>>>>>>>>> improvement over the current branch() API, which allows for
> > the
> > > >>>> more
> > > >>>>>>>>>>> subtle
> > > >>>>>>>>>>> behavior of records unexpectedly getting dropped.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> The need for a terminal operation certainly has to be well
> > > >>>>>>>>>>> documented, but
> > > >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder
> to
> > > track
> > > >>>>>>>>>>> dangling
> > > >>>>>>>>>>> branches that haven't been terminated and raise a clear
> error
> > > >>>> before it
> > > >>>>>>>>>>> becomes an issue.  Especially now that there is a "build
> > step"
> > > >>>> where
> > > >>>>>>>> the
> > > >>>>>>>>>>> topology is actually wired up, when StreamsBuilder.build()
> is
> > > >>>> called.
> > > >>>>>>>>>>> Regarding onTopOf() returning its argument, I agree that
> it's
> > > >>>>>>>>>>> critical to
> > > >>>>>>>>>>> allow users to do other operations on the input stream.
> With
> > > the
> > > >>>>>>>> fluent
> > > >>>>>>>>>>> solution, it ought to work the same way all other
> operations
> > > do -
> > > >>>> if
> > > >>>>>>>> you
> > > >>>>>>>>>>> want to process off the original KStream multiple times,
> you
> > > just
> > > >>>>>>>>>>> need the
> > > >>>>>>>>>>> stream as a variable so you can call as many operations on
> it
> > > as
> > > >>>> you
> > > >>>>>>>>>>> desire.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Thoughts?
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Best,
> > > >>>>>>>>>>> Paul
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> > > iponomarev@mail.ru
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> Hello Paul,
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> I afraid this won't work because we do not always need the
> > > >>>>>>>>>>>> defaultBranch. And without a terminal operation we don't
> > know
> > > >>>> when to
> > > >>>>>>>>>>>> finalize and build the 'branch switch'.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we can do
> > > >>>> something
> > > >>>>>>>>>>>> more with the original branch after branching.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> I understand your point that the need of special object
> > > >>>> construction
> > > >>>>>>>>>>>> contrasts the fluency of most KStream methods. But here we
> > > have a
> > > >>>>>>>>>>>> special case: we build the switch to split the flow, so I
> > > think
> > > >>>> this
> > > >>>>>>>> is
> > > >>>>>>>>>>>> still idiomatic.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Ivan
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> > > >>>>>>>>>>>>> Ivan,
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I think it's a great idea to improve this API, but I find
> > the
> > > >>>>>>>>>>>>> onTopOff()
> > > >>>>>>>>>>>>> mechanism a little confusing since it contrasts the
> fluency
> > > of
> > > >>>> other
> > > >>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call a
> > > method on
> > > >>>> the
> > > >>>>>>>>>>>> stream
> > > >>>>>>>>>>>>> so it still reads top to bottom if the branch cases are
> > > defined
> > > >>>>>>>>>>>>> fluently.
> > > >>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very nice
> > > and the
> > > >>>>>>>>>>>>> right
> > > >>>>>>>>>>>> way
> > > >>>>>>>>>>>>> to do things, but what if we flipped around how we
> specify
> > > the
> > > >>>> source
> > > >>>>>>>>>>>>> stream.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Like:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> stream.branch()
> > > >>>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
> > > >>>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
> > > >>>>>>>>>>>>>            .defaultBranch(this::handleDefault);
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
> > KStreamBrancher
> > > or
> > > >>>>>>>>>>>> something,
> > > >>>>>>>>>>>>> which is added to by addBranch() and terminated by
> > > >>>> defaultBranch()
> > > >>>>>>>>>>>>> (which
> > > >>>>>>>>>>>>> returns void).  This is obviously incompatible with the
> > > current
> > > >>>>>>>>>>>>> API, so
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>> new stream.branch() would have to have a different name,
> > but
> > > that
> > > >>>>>>>>>>>>> seems
> > > >>>>>>>>>>>>> like a fairly small problem - we could call it something
> > like
> > > >>>>>>>>>>>>> branched()
> > > >>>>>>>>>>>> or
> > > >>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It seems
> > > like it
> > > >>>>>>>>>>>>> does to
> > > >>>>>>>>>>>>> me, allowing for clear in-line branching while also
> > allowing
> > > you
> > > >>>> to
> > > >>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams if
> > > desired.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>> Paul
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> > > >>>>>>>>>>>> <ip...@mail.ru.invalid>
> > > >>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Hi Bill,
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Thank you for your reply!
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> This is how I usually do it:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
> > > >>>>>>>>>>>>>>            ks.filter(....).mapValues(...)
> > > >>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
> > > >>>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
> > > >>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> ......
> > > >>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> > > >>>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
> > > >>>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
> > > >>>>>>>>>>>>>>       .onTopOf(....)
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Ivan
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> > > >>>>>>>>>>>>>>> Hi Ivan,
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Thanks for the KIP.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher takes a
> > > Consumer
> > > >>>> as a
> > > >>>>>>>>>>>>>> second
> > > >>>>>>>>>>>>>>> argument which returns nothing, and the example in the
> > KIP
> > > >>>> shows
> > > >>>>>>>>>>>>>>> each
> > > >>>>>>>>>>>>>>> stream from the branch using a terminal node
> > > (KafkaStreams#to()
> > > >>>>>>>>>>>>>>> in this
> > > >>>>>>>>>>>>>>> case).
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Maybe I've missed something, but how would we handle
> the
> > > case
> > > >>>>>>>>>>>>>>> where the
> > > >>>>>>>>>>>>>>> user has created a branch but wants to continue
> > processing
> > > and
> > > >>>> not
> > > >>>>>>>>>>>>>>> necessarily use a terminal node on the branched stream
> > > >>>> immediately?
> > > >>>>>>>>>>>>>>> For example, using today's logic as is if we had
> > something
> > > like
> > > >>>>>>>>>>>>>>> this:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> KStream<String, String>[] branches =
> > > >>>>>>>>>>>>>>> originalStream.branch(predicate1,
> > > >>>>>>>>>>>>>>> predicate2);
> > > >>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> > > >>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Thanks!
> > > >>>>>>>>>>>>>>> Bill
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
> > > bbejeck@gmail.com
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> All,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP- 418.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Here's the original message:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Hello,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418. Please
> > take
> > > a
> > > >>>> look
> > > >>>>>>>> at
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> KIP-418:
> > > >>>>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> > > >>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> > > >>>> https://issues.apache.org/jira/browse/KAFKA-5488
> > > >>>>>>>>>>>>>>>> PR#6164: https://github.com/apache/kafka/pull/6164
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Ivan Ponomarev
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >
> > >
> > >
> >
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Michael Drogalis <mi...@confluent.io>.
I’d like to propose a different way of thinking about this. To me, there
are three problems with the existing branch signature:

1. If you use it the way most people do, Java raises unsafe type warnings.
2. The way in which you use the stream branches is positionally coupled to
the ordering of the conditionals.
3. It is brittle to extend existing branch calls with additional code paths.

Using associative constructs instead of relying on ordered constructs would
be a stronger approach. Consider a signature that instead looks like this:

Map<String, KStream<K,V>> KStream#branch(SortedMap<String, Predicate<?
super K,? super V>>);

Branches are given names in a map, and as a result, the API returns a
mapping of names to streams. The ordering of the conditionals is maintained
because it’s a sorted map. Insert order determines the order of evaluation.

This solves problem 1 because there are no more varargs. It solves problem
2 because you no longer lean on ordering to access the branch you’re
interested in. It solves problem 3 because you can introduce another
conditional by simply attaching another name to the structure, rather than
messing with the existing indices.

One of the drawbacks is that creating the map inline is historically
awkward in Java. I know it’s an anti-pattern to use voluminously, but
double brace initialization would clean up the aesthetics.

On Tue, Apr 30, 2019 at 9:10 AM John Roesler <jo...@confluent.io> wrote:

> Hi Ivan,
>
> Thanks for the update.
>
> FWIW, I agree with Matthias that the current "start branching" operator is
> confusing when named the same way as the actual branches. "Split" seems
> like a good name. Alternatively, we can do without a "start branching"
> operator at all, and just do:
>
> stream
>       .branch(Predicate)
>       .branch(Predicate)
>       .defaultBranch();
>
> Tentatively, I think that this branching operation should be terminal. That
> way, we don't create ambiguity about how to use it. That is, `branch`
> should return `KBranchedStream`, while `defaultBranch` is `void`, to
> enforce that it comes last, and that there is only one definition of the
> default branch. Potentially, we should log a warning if there's no default,
> and additionally log a warning (or throw an exception) if a record falls
> though with no default.
>
> Thoughts?
>
> Thanks,
> -John
>
> On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > Thanks for updating the KIP and your answers.
> >
> >
> > >  this is to make the name similar to String#split
> > >> that also returns an array, right?
> >
> > The intend was to avoid name duplication. The return type should _not_
> > be an array.
> >
> > The current proposal is
> >
> > stream.branch()
> >       .branch(Predicate)
> >       .branch(Predicate)
> >       .defaultBranch();
> >
> > IMHO, this reads a little odd, because the first `branch()` does not
> > take any parameters and has different semantics than the later
> > `branch()` calls. Note, that from the code snippet above, it's hidden
> > that the first call is `KStream#branch()` while the others are
> > `KBranchedStream#branch()` what makes reading the code harder.
> >
> > Because I suggested to rename `addBranch()` -> `branch()`, I though it
> > might be better to also rename `KStream#branch()` to avoid the naming
> > overlap that seems to be confusing. The following reads much cleaner to
> me:
> >
> > stream.split()
> >       .branch(Predicate)
> >       .branch(Predicate)
> >       .defaultBranch();
> >
> > Maybe there is a better alternative to `split()` though to avoid the
> > naming overlap.
> >
> >
> > > 'default' is, however, a reserved word, so unfortunately we cannot have
> > a method with such name :-)
> >
> > Bummer. Didn't consider this. Maybe we can still come up with a short
> name?
> >
> >
> > Can you add the interface `KBranchedStream` to the KIP with all it's
> > methods? It will be part of public API and should be contained in the
> > KIP. For example, it's unclear atm, what the return type of
> > `defaultBranch()` is.
> >
> >
> > You did not comment on the idea to add a `KBranchedStream#get(int index)
> > -> KStream` method to get the individually branched-KStreams. Would be
> > nice to get your feedback about it. It seems you suggest that users
> > would need to write custom utility code otherwise, to access them. We
> > should discuss the pros and cons of both approaches. It feels
> > "incomplete" to me atm, if the API has no built-in support to get the
> > branched-KStreams directly.
> >
> >
> >
> > -Matthias
> >
> >
> > On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> > > Hi all!
> > >
> > > I have updated the KIP-418 according to the new vision.
> > >
> > > Matthias, thanks for your comment!
> > >
> > >> Renaming KStream#branch() -> #split()
> > >
> > > I can see your point: this is to make the name similar to String#split
> > > that also returns an array, right? But is it worth the loss of
> backwards
> > > compatibility? We can have overloaded branch() as well without
> affecting
> > > the existing code. Maybe the old array-based `branch` method should be
> > > deprecated, but this is a subject for discussion.
> > >
> > >> Renaming KBranchedStream#addBranch() -> BranchingKStream#branch(),
> > > KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > >
> > > Totally agree with 'addBranch->branch' rename. 'default' is, however, a
> > > reserved word, so unfortunately we cannot have a method with such name
> > :-)
> > >
> > >> defaultBranch() does take an `Predicate` as argument, but I think that
> > > is not required?
> > >
> > > Absolutely! I think that was just copy-paste error or something.
> > >
> > > Dear colleagues,
> > >
> > > please revise the new version of the KIP and Paul's PR
> > > (https://github.com/apache/kafka/pull/6512)
> > >
> > > Any new suggestions/objections?
> > >
> > > Regards,
> > >
> > > Ivan
> > >
> > >
> > > 11.04.2019 11:47, Matthias J. Sax пишет:
> > >> Thanks for driving the discussion of this KIP. It seems that everybody
> > >> agrees that the current branch() method using arrays is not optimal.
> > >>
> > >> I had a quick look into the PR and I like the overall proposal. There
> > >> are some minor things we need to consider. I would recommend the
> > >> following renaming:
> > >>
> > >> KStream#branch() -> #split()
> > >> KBranchedStream#addBranch() -> BranchingKStream#branch()
> > >> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> > >>
> > >> It's just a suggestion to get slightly shorter method names.
> > >>
> > >> In the current PR, defaultBranch() does take an `Predicate` as
> argument,
> > >> but I think that is not required?
> > >>
> > >> Also, we should consider KIP-307, that was recently accepted and is
> > >> currently implemented:
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> > >>
> > >> Ie, we should add overloads that accepted a `Named` parameter.
> > >>
> > >>
> > >> For the issue that the created `KStream` object are in different
> scopes:
> > >> could we extend `KBranchedStream` with a `get(int index)` method that
> > >> returns the corresponding "branched" result `KStream` object? Maybe,
> the
> > >> second argument of `addBranch()` should not be a `Consumer<KStream>`
> but
> > >> a `Function<KStream,KStream>` and `get()` could return whatever the
> > >> `Function` returns?
> > >>
> > >>
> > >> Finally, I would also suggest to update the KIP with the current
> > >> proposal. That makes it easier to review.
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>
> > >>
> > >> On 3/31/19 12:22 PM, Paul Whalen wrote:
> > >>> Ivan,
> > >>>
> > >>> I'm a bit of a novice here as well, but I think it makes sense for
> you
> > to
> > >>> revise the KIP and continue the discussion.  Obviously we'll need
> some
> > >>> buy-in from committers that have actual binding votes on whether the
> > KIP
> > >>> could be adopted.  It would be great to hear if they think this is a
> > good
> > >>> idea overall.  I'm not sure if that happens just by starting a vote,
> > or if
> > >>> there is generally some indication of interest beforehand.
> > >>>
> > >>> That being said, I'll continue the discussion a bit: assuming we do
> > move
> > >>> forward the solution of "stream.branch() returns KBranchedStream", do
> > we
> > >>> deprecate "stream.branch(...) returns KStream[]"?  I would favor
> > >>> deprecating, since having two mutually exclusive APIs that accomplish
> > the
> > >>> same thing is confusing, especially when they're fairly similar
> > anyway.  We
> > >>> just need to be sure we're not making something impossible/difficult
> > that
> > >>> is currently possible/easy.
> > >>>
> > >>> Regarding my PR - I think the general structure would work, it's
> just a
> > >>> little sloppy overall in terms of naming and clarity. In particular,
> > >>> passing in the "predicates" and "children" lists which get modified
> in
> > >>> KBranchedStream but read from all the way KStreamLazyBranch is a bit
> > >>> complicated to follow.
> > >>>
> > >>> Thanks,
> > >>> Paul
> > >>>
> > >>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <ip...@mail.ru>
> > wrote:
> > >>>
> > >>>> Hi Paul!
> > >>>>
> > >>>> I read your code carefully and now I am fully convinced: your
> proposal
> > >>>> looks better and should work. We just have to document the crucial
> > fact
> > >>>> that KStream consumers are invoked as they're added. And then it's
> all
> > >>>> going to be very nice.
> > >>>>
> > >>>> What shall we do now? I should re-write the KIP and resume the
> > >>>> discussion here, right?
> > >>>>
> > >>>> Why are you telling that your PR 'should not be even a starting
> point
> > if
> > >>>> we go in this direction'? To me it looks like a good starting point.
> > But
> > >>>> as a novice in this project I might miss some important details.
> > >>>>
> > >>>> Regards,
> > >>>>
> > >>>> Ivan
> > >>>>
> > >>>>
> > >>>> 28.03.2019 17:38, Paul Whalen пишет:
> > >>>>> Ivan,
> > >>>>>
> > >>>>> Maybe I’m missing the point, but I believe the stream.branch()
> > solution
> > >>>> supports this. The couponIssuer::set* consumers will be invoked as
> > they’re
> > >>>> added, not during streamsBuilder.build(). So the user still ought to
> > be
> > >>>> able to call couponIssuer.coupons() afterward and depend on the
> > branched
> > >>>> streams having been set.
> > >>>>> The issue I mean to point out is that it is hard to access the
> > branched
> > >>>> streams in the same scope as the original stream (that is, not
> inside
> > the
> > >>>> couponIssuer), which is a problem with both proposed solutions. It
> > can be
> > >>>> worked around though.
> > >>>>> [Also, great to hear additional interest in 401, I’m excited to
> hear
> > >>>> your thoughts!]
> > >>>>> Paul
> > >>>>>
> > >>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <ip...@mail.ru>
> > wrote:
> > >>>>>>
> > >>>>>> Hi Paul!
> > >>>>>>
> > >>>>>> The idea to postpone the wiring of branches to the
> > >>>> streamsBuilder.build() also looked great for me at first glance, but
> > ---
> > >>>>>>> the newly branched streams are not available in the same scope as
> > each
> > >>>> other.  That is, if we wanted to merge them back together again I
> > don't see
> > >>>> a way to do that.
> > >>>>>> You just took the words right out of my mouth, I was just going to
> > >>>> write in details about this issue.
> > >>>>>> Consider the example from Bill's book, p. 101: say we need to
> > identify
> > >>>> customers who have bought coffee and made a purchase in the
> > electronics
> > >>>> store to give them coupons.
> > >>>>>> This is the code I usually write under these circumstances using
> my
> > >>>> 'brancher' class:
> > >>>>>> @Setter
> > >>>>>> class CouponIssuer{
> > >>>>>>    private KStream<....> coffePurchases;
> > >>>>>>    private KStream<....> electronicsPurchases;
> > >>>>>>
> > >>>>>>    KStream<...> coupons(){
> > >>>>>>        return
> > coffePurchases.join(electronicsPurchases...)...whatever
> > >>>>>>
> > >>>>>>        /*In the real world the code here can be complex, so
> > creation of
> > >>>> a separate CouponIssuer class is fully justified, in order to
> separate
> > >>>> classes' responsibilities.*/
> > >>>>>>   }
> > >>>>>> }
> > >>>>>>
> > >>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> > >>>>>>
> > >>>>>> new KafkaStreamsBrancher<....>()
> > >>>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
> > >>>>>>      .branch(predicate2, couponIssuer::setElectronicsPurchases)
> > >>>>>>      .onTopOf(transactionStream);
> > >>>>>>
> > >>>>>> /*Alas, this won't work if we're going to wire up everything
> later,
> > >>>> without the terminal operation!!!*/
> > >>>>>> couponIssuer.coupons()...
> > >>>>>>
> > >>>>>> Does this make sense?  In order to properly initialize the
> > CouponIssuer
> > >>>> we need the terminal operation to be called before
> > streamsBuilder.build()
> > >>>> is called.
> > >>>>>> [BTW Paul, I just found out that your KIP-401 is essentially the
> > next
> > >>>> KIP I was going to write here. I have some thoughts based on my
> > experience,
> > >>>> so I will join the discussion on KIP-401 soon.]
> > >>>>>> Regards,
> > >>>>>>
> > >>>>>> Ivan
> > >>>>>>
> > >>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> > >>>>>>> Ivan,
> > >>>>>>> I tried to make a very rough proof of concept of a fluent API
> based
> > >>>> off of
> > >>>>>>> KStream here (https://github.com/apache/kafka/pull/6512), and I
> > think
> > >>>> I
> > >>>>>>> succeeded at removing both cons.
> > >>>>>>>     - Compatibility: I was incorrect earlier about compatibility
> > >>>> issues,
> > >>>>>>>     there aren't any direct ones.  I was unaware that Java is
> smart
> > >>>> enough to
> > >>>>>>>     distinguish between a branch(varargs...) returning one thing
> > and
> > >>>> branch()
> > >>>>>>>     with no arguments returning another thing.
> > >>>>>>>     - Requiring a terminal method: We don't actually need it.  We
> > can
> > >>>> just
> > >>>>>>>     build up the branches in the KBranchedStream who shares its
> > state
> > >>>> with the
> > >>>>>>>     ProcessorSupplier that will actually do the branching.  It's
> > not
> > >>>> terribly
> > >>>>>>>     pretty in its current form, but I think it demonstrates its
> > >>>> feasibility.
> > >>>>>>> To be clear, I don't think that pull request should be final or
> > even a
> > >>>>>>> starting point if we go in this direction, I just wanted to see
> how
> > >>>>>>> challenging it would be to get the API working.
> > >>>>>>> I will say though, that I'm not sure the existing solution could
> be
> > >>>>>>> deprecated in favor of this, which I had originally suggested
> was a
> > >>>>>>> possibility.  The reason is that the newly branched streams are
> not
> > >>>>>>> available in the same scope as each other.  That is, if we wanted
> > to
> > >>>> merge
> > >>>>>>> them back together again I don't see a way to do that.  The KIP
> > >>>> proposal
> > >>>>>>> has the same issue, though - all this means is that for either
> > >>>> solution,
> > >>>>>>> deprecating the existing branch(...) is not on the table.
> > >>>>>>> Thanks,
> > >>>>>>> Paul
> > >>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> > iponomarev@mail.ru>
> > >>>> wrote:
> > >>>>>>>> OK, let me summarize what we have discussed up to this point.
> > >>>>>>>>
> > >>>>>>>> First, it seems that it's commonly agreed that branch API needs
> > >>>>>>>> improvement. Motivation is given in the KIP.
> > >>>>>>>>
> > >>>>>>>> There are two potential ways to do it:
> > >>>>>>>>
> > >>>>>>>> 1. (as origianlly proposed)
> > >>>>>>>>
> > >>>>>>>> new KafkaStreamsBrancher<..>()
> > >>>>>>>>     .branch(predicate1, ks ->..)
> > >>>>>>>>     .branch(predicate2, ks->..)
> > >>>>>>>>     .defaultBranch(ks->..) //optional
> > >>>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns its
> > argument
> > >>>>>>>>
> > >>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't make
> sense
> > >>>> until
> > >>>>>>>> all the necessary ingredients are provided.
> > >>>>>>>>
> > >>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
> > contrasts the
> > >>>>>>>> fluency of other KStream methods.
> > >>>>>>>>
> > >>>>>>>> 2. (as Paul proposes)
> > >>>>>>>>
> > >>>>>>>> stream
> > >>>>>>>>     .branch(predicate1, ks ->...)
> > >>>>>>>>     .branch(predicate2, ks->...)
> > >>>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both
> > defaultBranch(..)
> > >>>> and
> > >>>>>>>> noDefault() return void
> > >>>>>>>>
> > >>>>>>>> PROS: Generally follows the way KStreams interface is defined.
> > >>>>>>>>
> > >>>>>>>> CONS: We need to define two terminal methods
> (defaultBranch(ks->)
> > and
> > >>>>>>>> noDefault()). And for a user it is very easy to miss the fact
> > that one
> > >>>>>>>> of the terminal methods should be called. If these methods are
> not
> > >>>>>>>> called, we can throw an exception in runtime.
> > >>>>>>>>
> > >>>>>>>> Colleagues, what are your thoughts? Can we do better?
> > >>>>>>>>
> > >>>>>>>> Regards,
> > >>>>>>>>
> > >>>>>>>> Ivan
> > >>>>>>>>
> > >>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> > >>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> > >>>>>>>>>> Paul,
> > >>>>>>>>>>
> > >>>>>>>>>> I see your point when you are talking about
> > >>>>>>>>>> stream..branch..branch...default..
> > >>>>>>>>>>
> > >>>>>>>>>> Still, I believe that this cannot not be implemented the easy
> > way.
> > >>>>>>>>>> Maybe we all should think further.
> > >>>>>>>>>>
> > >>>>>>>>>> Let me comment on two of your ideas.
> > >>>>>>>>>>
> > >>>>>>>>>>> user could specify a terminal method that assumes nothing
> will
> > >>>> reach
> > >>>>>>>>>>> the default branch,
> > >>>>>>>>>> throwing an exception if such a case occurs.
> > >>>>>>>>>>
> > >>>>>>>>>> 1) OK, apparently this should not be the only option besides
> > >>>>>>>>>> `default`, because there are scenarios when we want to just
> > silently
> > >>>>>>>>>> drop the messages that didn't match any predicate. 2) Throwing
> > an
> > >>>>>>>>>> exception in the middle of data flow processing looks like a
> bad
> > >>>> idea.
> > >>>>>>>>>> In stream processing paradigm, I would prefer to emit a
> special
> > >>>>>>>>>> message to a dedicated stream. This is exactly where `default`
> > can
> > >>>> be
> > >>>>>>>>>> used.
> > >>>>>>>>>>
> > >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to
> > track
> > >>>>>>>>>>> dangling
> > >>>>>>>>>> branches that haven't been terminated and raise a clear error
> > >>>> before it
> > >>>>>>>>>> becomes an issue.
> > >>>>>>>>>>
> > >>>>>>>>>> You mean a runtime exception, when the program is compiled and
> > run?
> > >>>>>>>>>> Well,  I'd prefer an API that simply won't compile if used
> > >>>>>>>>>> incorrectly. Can we build such an API as a method chain
> starting
> > >>>> from
> > >>>>>>>>>> KStream object? There is a huge cost difference between
> runtime
> > and
> > >>>>>>>>>> compile-time errors. Even if a failure uncovers instantly on
> > unit
> > >>>>>>>>>> tests, it costs more for the project than a compilation
> failure.
> > >>>>>>>>>>
> > >>>>>>>>>> Regards,
> > >>>>>>>>>>
> > >>>>>>>>>> Ivan
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> > >>>>>>>>>>> Ivan,
> > >>>>>>>>>>>
> > >>>>>>>>>>> Good point about the terminal operation being required.  But
> is
> > >>>> that
> > >>>>>>>>>>> really
> > >>>>>>>>>>> such a bad thing?  If the user doesn't want a defaultBranch
> > they
> > >>>> can
> > >>>>>>>>>>> call
> > >>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
> > easily.  In
> > >>>>>>>>>>> fact I
> > >>>>>>>>>>> think it creates an opportunity for a nicer API - a user
> could
> > >>>> specify
> > >>>>>>>> a
> > >>>>>>>>>>> terminal method that assumes nothing will reach the default
> > branch,
> > >>>>>>>>>>> throwing an exception if such a case occurs.  That seems like
> > an
> > >>>>>>>>>>> improvement over the current branch() API, which allows for
> the
> > >>>> more
> > >>>>>>>>>>> subtle
> > >>>>>>>>>>> behavior of records unexpectedly getting dropped.
> > >>>>>>>>>>>
> > >>>>>>>>>>> The need for a terminal operation certainly has to be well
> > >>>>>>>>>>> documented, but
> > >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to
> > track
> > >>>>>>>>>>> dangling
> > >>>>>>>>>>> branches that haven't been terminated and raise a clear error
> > >>>> before it
> > >>>>>>>>>>> becomes an issue.  Especially now that there is a "build
> step"
> > >>>> where
> > >>>>>>>> the
> > >>>>>>>>>>> topology is actually wired up, when StreamsBuilder.build() is
> > >>>> called.
> > >>>>>>>>>>> Regarding onTopOf() returning its argument, I agree that it's
> > >>>>>>>>>>> critical to
> > >>>>>>>>>>> allow users to do other operations on the input stream.  With
> > the
> > >>>>>>>> fluent
> > >>>>>>>>>>> solution, it ought to work the same way all other operations
> > do -
> > >>>> if
> > >>>>>>>> you
> > >>>>>>>>>>> want to process off the original KStream multiple times, you
> > just
> > >>>>>>>>>>> need the
> > >>>>>>>>>>> stream as a variable so you can call as many operations on it
> > as
> > >>>> you
> > >>>>>>>>>>> desire.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thoughts?
> > >>>>>>>>>>>
> > >>>>>>>>>>> Best,
> > >>>>>>>>>>> Paul
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> > iponomarev@mail.ru
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Hello Paul,
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> I afraid this won't work because we do not always need the
> > >>>>>>>>>>>> defaultBranch. And without a terminal operation we don't
> know
> > >>>> when to
> > >>>>>>>>>>>> finalize and build the 'branch switch'.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we can do
> > >>>> something
> > >>>>>>>>>>>> more with the original branch after branching.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> I understand your point that the need of special object
> > >>>> construction
> > >>>>>>>>>>>> contrasts the fluency of most KStream methods. But here we
> > have a
> > >>>>>>>>>>>> special case: we build the switch to split the flow, so I
> > think
> > >>>> this
> > >>>>>>>> is
> > >>>>>>>>>>>> still idiomatic.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> > >>>>>>>>>>>>> Ivan,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I think it's a great idea to improve this API, but I find
> the
> > >>>>>>>>>>>>> onTopOff()
> > >>>>>>>>>>>>> mechanism a little confusing since it contrasts the fluency
> > of
> > >>>> other
> > >>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call a
> > method on
> > >>>> the
> > >>>>>>>>>>>> stream
> > >>>>>>>>>>>>> so it still reads top to bottom if the branch cases are
> > defined
> > >>>>>>>>>>>>> fluently.
> > >>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very nice
> > and the
> > >>>>>>>>>>>>> right
> > >>>>>>>>>>>> way
> > >>>>>>>>>>>>> to do things, but what if we flipped around how we specify
> > the
> > >>>> source
> > >>>>>>>>>>>>> stream.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Like:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> stream.branch()
> > >>>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
> > >>>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
> > >>>>>>>>>>>>>            .defaultBranch(this::handleDefault);
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Where branch() returns a KBranchedStreams or
> KStreamBrancher
> > or
> > >>>>>>>>>>>> something,
> > >>>>>>>>>>>>> which is added to by addBranch() and terminated by
> > >>>> defaultBranch()
> > >>>>>>>>>>>>> (which
> > >>>>>>>>>>>>> returns void).  This is obviously incompatible with the
> > current
> > >>>>>>>>>>>>> API, so
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>> new stream.branch() would have to have a different name,
> but
> > that
> > >>>>>>>>>>>>> seems
> > >>>>>>>>>>>>> like a fairly small problem - we could call it something
> like
> > >>>>>>>>>>>>> branched()
> > >>>>>>>>>>>> or
> > >>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It seems
> > like it
> > >>>>>>>>>>>>> does to
> > >>>>>>>>>>>>> me, allowing for clear in-line branching while also
> allowing
> > you
> > >>>> to
> > >>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams if
> > desired.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>> Paul
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> > >>>>>>>>>>>> <ip...@mail.ru.invalid>
> > >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Hi Bill,
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thank you for your reply!
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> This is how I usually do it:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
> > >>>>>>>>>>>>>>            ks.filter(....).mapValues(...)
> > >>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
> > >>>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
> > >>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> ......
> > >>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> > >>>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
> > >>>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
> > >>>>>>>>>>>>>>       .onTopOf(....)
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Ivan
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> > >>>>>>>>>>>>>>> Hi Ivan,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks for the KIP.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher takes a
> > Consumer
> > >>>> as a
> > >>>>>>>>>>>>>> second
> > >>>>>>>>>>>>>>> argument which returns nothing, and the example in the
> KIP
> > >>>> shows
> > >>>>>>>>>>>>>>> each
> > >>>>>>>>>>>>>>> stream from the branch using a terminal node
> > (KafkaStreams#to()
> > >>>>>>>>>>>>>>> in this
> > >>>>>>>>>>>>>>> case).
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Maybe I've missed something, but how would we handle the
> > case
> > >>>>>>>>>>>>>>> where the
> > >>>>>>>>>>>>>>> user has created a branch but wants to continue
> processing
> > and
> > >>>> not
> > >>>>>>>>>>>>>>> necessarily use a terminal node on the branched stream
> > >>>> immediately?
> > >>>>>>>>>>>>>>> For example, using today's logic as is if we had
> something
> > like
> > >>>>>>>>>>>>>>> this:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> KStream<String, String>[] branches =
> > >>>>>>>>>>>>>>> originalStream.branch(predicate1,
> > >>>>>>>>>>>>>>> predicate2);
> > >>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> > >>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks!
> > >>>>>>>>>>>>>>> Bill
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
> > bbejeck@gmail.com
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> All,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP- 418.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Here's the original message:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Hello,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418. Please
> take
> > a
> > >>>> look
> > >>>>>>>> at
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> KIP-418:
> > >>>>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> > >>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> > >>>> https://issues.apache.org/jira/browse/KAFKA-5488
> > >>>>>>>>>>>>>>>> PR#6164: https://github.com/apache/kafka/pull/6164
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Ivan Ponomarev
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >
> >
> >
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by John Roesler <jo...@confluent.io>.
Hi Ivan,

Thanks for the update.

FWIW, I agree with Matthias that the current "start branching" operator is
confusing when named the same way as the actual branches. "Split" seems
like a good name. Alternatively, we can do without a "start branching"
operator at all, and just do:

stream
      .branch(Predicate)
      .branch(Predicate)
      .defaultBranch();

Tentatively, I think that this branching operation should be terminal. That
way, we don't create ambiguity about how to use it. That is, `branch`
should return `KBranchedStream`, while `defaultBranch` is `void`, to
enforce that it comes last, and that there is only one definition of the
default branch. Potentially, we should log a warning if there's no default,
and additionally log a warning (or throw an exception) if a record falls
though with no default.

Thoughts?

Thanks,
-John

On Fri, Apr 26, 2019 at 3:40 AM Matthias J. Sax <ma...@confluent.io>
wrote:

> Thanks for updating the KIP and your answers.
>
>
> >  this is to make the name similar to String#split
> >> that also returns an array, right?
>
> The intend was to avoid name duplication. The return type should _not_
> be an array.
>
> The current proposal is
>
> stream.branch()
>       .branch(Predicate)
>       .branch(Predicate)
>       .defaultBranch();
>
> IMHO, this reads a little odd, because the first `branch()` does not
> take any parameters and has different semantics than the later
> `branch()` calls. Note, that from the code snippet above, it's hidden
> that the first call is `KStream#branch()` while the others are
> `KBranchedStream#branch()` what makes reading the code harder.
>
> Because I suggested to rename `addBranch()` -> `branch()`, I though it
> might be better to also rename `KStream#branch()` to avoid the naming
> overlap that seems to be confusing. The following reads much cleaner to me:
>
> stream.split()
>       .branch(Predicate)
>       .branch(Predicate)
>       .defaultBranch();
>
> Maybe there is a better alternative to `split()` though to avoid the
> naming overlap.
>
>
> > 'default' is, however, a reserved word, so unfortunately we cannot have
> a method with such name :-)
>
> Bummer. Didn't consider this. Maybe we can still come up with a short name?
>
>
> Can you add the interface `KBranchedStream` to the KIP with all it's
> methods? It will be part of public API and should be contained in the
> KIP. For example, it's unclear atm, what the return type of
> `defaultBranch()` is.
>
>
> You did not comment on the idea to add a `KBranchedStream#get(int index)
> -> KStream` method to get the individually branched-KStreams. Would be
> nice to get your feedback about it. It seems you suggest that users
> would need to write custom utility code otherwise, to access them. We
> should discuss the pros and cons of both approaches. It feels
> "incomplete" to me atm, if the API has no built-in support to get the
> branched-KStreams directly.
>
>
>
> -Matthias
>
>
> On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> > Hi all!
> >
> > I have updated the KIP-418 according to the new vision.
> >
> > Matthias, thanks for your comment!
> >
> >> Renaming KStream#branch() -> #split()
> >
> > I can see your point: this is to make the name similar to String#split
> > that also returns an array, right? But is it worth the loss of backwards
> > compatibility? We can have overloaded branch() as well without affecting
> > the existing code. Maybe the old array-based `branch` method should be
> > deprecated, but this is a subject for discussion.
> >
> >> Renaming KBranchedStream#addBranch() -> BranchingKStream#branch(),
> > KBranchedStream#defaultBranch() -> BranchingKStream#default()
> >
> > Totally agree with 'addBranch->branch' rename. 'default' is, however, a
> > reserved word, so unfortunately we cannot have a method with such name
> :-)
> >
> >> defaultBranch() does take an `Predicate` as argument, but I think that
> > is not required?
> >
> > Absolutely! I think that was just copy-paste error or something.
> >
> > Dear colleagues,
> >
> > please revise the new version of the KIP and Paul's PR
> > (https://github.com/apache/kafka/pull/6512)
> >
> > Any new suggestions/objections?
> >
> > Regards,
> >
> > Ivan
> >
> >
> > 11.04.2019 11:47, Matthias J. Sax пишет:
> >> Thanks for driving the discussion of this KIP. It seems that everybody
> >> agrees that the current branch() method using arrays is not optimal.
> >>
> >> I had a quick look into the PR and I like the overall proposal. There
> >> are some minor things we need to consider. I would recommend the
> >> following renaming:
> >>
> >> KStream#branch() -> #split()
> >> KBranchedStream#addBranch() -> BranchingKStream#branch()
> >> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> >>
> >> It's just a suggestion to get slightly shorter method names.
> >>
> >> In the current PR, defaultBranch() does take an `Predicate` as argument,
> >> but I think that is not required?
> >>
> >> Also, we should consider KIP-307, that was recently accepted and is
> >> currently implemented:
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
> >>
> >> Ie, we should add overloads that accepted a `Named` parameter.
> >>
> >>
> >> For the issue that the created `KStream` object are in different scopes:
> >> could we extend `KBranchedStream` with a `get(int index)` method that
> >> returns the corresponding "branched" result `KStream` object? Maybe, the
> >> second argument of `addBranch()` should not be a `Consumer<KStream>` but
> >> a `Function<KStream,KStream>` and `get()` could return whatever the
> >> `Function` returns?
> >>
> >>
> >> Finally, I would also suggest to update the KIP with the current
> >> proposal. That makes it easier to review.
> >>
> >>
> >> -Matthias
> >>
> >>
> >>
> >> On 3/31/19 12:22 PM, Paul Whalen wrote:
> >>> Ivan,
> >>>
> >>> I'm a bit of a novice here as well, but I think it makes sense for you
> to
> >>> revise the KIP and continue the discussion.  Obviously we'll need some
> >>> buy-in from committers that have actual binding votes on whether the
> KIP
> >>> could be adopted.  It would be great to hear if they think this is a
> good
> >>> idea overall.  I'm not sure if that happens just by starting a vote,
> or if
> >>> there is generally some indication of interest beforehand.
> >>>
> >>> That being said, I'll continue the discussion a bit: assuming we do
> move
> >>> forward the solution of "stream.branch() returns KBranchedStream", do
> we
> >>> deprecate "stream.branch(...) returns KStream[]"?  I would favor
> >>> deprecating, since having two mutually exclusive APIs that accomplish
> the
> >>> same thing is confusing, especially when they're fairly similar
> anyway.  We
> >>> just need to be sure we're not making something impossible/difficult
> that
> >>> is currently possible/easy.
> >>>
> >>> Regarding my PR - I think the general structure would work, it's just a
> >>> little sloppy overall in terms of naming and clarity. In particular,
> >>> passing in the "predicates" and "children" lists which get modified in
> >>> KBranchedStream but read from all the way KStreamLazyBranch is a bit
> >>> complicated to follow.
> >>>
> >>> Thanks,
> >>> Paul
> >>>
> >>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <ip...@mail.ru>
> wrote:
> >>>
> >>>> Hi Paul!
> >>>>
> >>>> I read your code carefully and now I am fully convinced: your proposal
> >>>> looks better and should work. We just have to document the crucial
> fact
> >>>> that KStream consumers are invoked as they're added. And then it's all
> >>>> going to be very nice.
> >>>>
> >>>> What shall we do now? I should re-write the KIP and resume the
> >>>> discussion here, right?
> >>>>
> >>>> Why are you telling that your PR 'should not be even a starting point
> if
> >>>> we go in this direction'? To me it looks like a good starting point.
> But
> >>>> as a novice in this project I might miss some important details.
> >>>>
> >>>> Regards,
> >>>>
> >>>> Ivan
> >>>>
> >>>>
> >>>> 28.03.2019 17:38, Paul Whalen пишет:
> >>>>> Ivan,
> >>>>>
> >>>>> Maybe I’m missing the point, but I believe the stream.branch()
> solution
> >>>> supports this. The couponIssuer::set* consumers will be invoked as
> they’re
> >>>> added, not during streamsBuilder.build(). So the user still ought to
> be
> >>>> able to call couponIssuer.coupons() afterward and depend on the
> branched
> >>>> streams having been set.
> >>>>> The issue I mean to point out is that it is hard to access the
> branched
> >>>> streams in the same scope as the original stream (that is, not inside
> the
> >>>> couponIssuer), which is a problem with both proposed solutions. It
> can be
> >>>> worked around though.
> >>>>> [Also, great to hear additional interest in 401, I’m excited to hear
> >>>> your thoughts!]
> >>>>> Paul
> >>>>>
> >>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <ip...@mail.ru>
> wrote:
> >>>>>>
> >>>>>> Hi Paul!
> >>>>>>
> >>>>>> The idea to postpone the wiring of branches to the
> >>>> streamsBuilder.build() also looked great for me at first glance, but
> ---
> >>>>>>> the newly branched streams are not available in the same scope as
> each
> >>>> other.  That is, if we wanted to merge them back together again I
> don't see
> >>>> a way to do that.
> >>>>>> You just took the words right out of my mouth, I was just going to
> >>>> write in details about this issue.
> >>>>>> Consider the example from Bill's book, p. 101: say we need to
> identify
> >>>> customers who have bought coffee and made a purchase in the
> electronics
> >>>> store to give them coupons.
> >>>>>> This is the code I usually write under these circumstances using my
> >>>> 'brancher' class:
> >>>>>> @Setter
> >>>>>> class CouponIssuer{
> >>>>>>    private KStream<....> coffePurchases;
> >>>>>>    private KStream<....> electronicsPurchases;
> >>>>>>
> >>>>>>    KStream<...> coupons(){
> >>>>>>        return
> coffePurchases.join(electronicsPurchases...)...whatever
> >>>>>>
> >>>>>>        /*In the real world the code here can be complex, so
> creation of
> >>>> a separate CouponIssuer class is fully justified, in order to separate
> >>>> classes' responsibilities.*/
> >>>>>>   }
> >>>>>> }
> >>>>>>
> >>>>>> CouponIssuer couponIssuer = new CouponIssuer();
> >>>>>>
> >>>>>> new KafkaStreamsBrancher<....>()
> >>>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
> >>>>>>      .branch(predicate2, couponIssuer::setElectronicsPurchases)
> >>>>>>      .onTopOf(transactionStream);
> >>>>>>
> >>>>>> /*Alas, this won't work if we're going to wire up everything later,
> >>>> without the terminal operation!!!*/
> >>>>>> couponIssuer.coupons()...
> >>>>>>
> >>>>>> Does this make sense?  In order to properly initialize the
> CouponIssuer
> >>>> we need the terminal operation to be called before
> streamsBuilder.build()
> >>>> is called.
> >>>>>> [BTW Paul, I just found out that your KIP-401 is essentially the
> next
> >>>> KIP I was going to write here. I have some thoughts based on my
> experience,
> >>>> so I will join the discussion on KIP-401 soon.]
> >>>>>> Regards,
> >>>>>>
> >>>>>> Ivan
> >>>>>>
> >>>>>> 28.03.2019 6:29, Paul Whalen пишет:
> >>>>>>> Ivan,
> >>>>>>> I tried to make a very rough proof of concept of a fluent API based
> >>>> off of
> >>>>>>> KStream here (https://github.com/apache/kafka/pull/6512), and I
> think
> >>>> I
> >>>>>>> succeeded at removing both cons.
> >>>>>>>     - Compatibility: I was incorrect earlier about compatibility
> >>>> issues,
> >>>>>>>     there aren't any direct ones.  I was unaware that Java is smart
> >>>> enough to
> >>>>>>>     distinguish between a branch(varargs...) returning one thing
> and
> >>>> branch()
> >>>>>>>     with no arguments returning another thing.
> >>>>>>>     - Requiring a terminal method: We don't actually need it.  We
> can
> >>>> just
> >>>>>>>     build up the branches in the KBranchedStream who shares its
> state
> >>>> with the
> >>>>>>>     ProcessorSupplier that will actually do the branching.  It's
> not
> >>>> terribly
> >>>>>>>     pretty in its current form, but I think it demonstrates its
> >>>> feasibility.
> >>>>>>> To be clear, I don't think that pull request should be final or
> even a
> >>>>>>> starting point if we go in this direction, I just wanted to see how
> >>>>>>> challenging it would be to get the API working.
> >>>>>>> I will say though, that I'm not sure the existing solution could be
> >>>>>>> deprecated in favor of this, which I had originally suggested was a
> >>>>>>> possibility.  The reason is that the newly branched streams are not
> >>>>>>> available in the same scope as each other.  That is, if we wanted
> to
> >>>> merge
> >>>>>>> them back together again I don't see a way to do that.  The KIP
> >>>> proposal
> >>>>>>> has the same issue, though - all this means is that for either
> >>>> solution,
> >>>>>>> deprecating the existing branch(...) is not on the table.
> >>>>>>> Thanks,
> >>>>>>> Paul
> >>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <
> iponomarev@mail.ru>
> >>>> wrote:
> >>>>>>>> OK, let me summarize what we have discussed up to this point.
> >>>>>>>>
> >>>>>>>> First, it seems that it's commonly agreed that branch API needs
> >>>>>>>> improvement. Motivation is given in the KIP.
> >>>>>>>>
> >>>>>>>> There are two potential ways to do it:
> >>>>>>>>
> >>>>>>>> 1. (as origianlly proposed)
> >>>>>>>>
> >>>>>>>> new KafkaStreamsBrancher<..>()
> >>>>>>>>     .branch(predicate1, ks ->..)
> >>>>>>>>     .branch(predicate2, ks->..)
> >>>>>>>>     .defaultBranch(ks->..) //optional
> >>>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns its
> argument
> >>>>>>>>
> >>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't make sense
> >>>> until
> >>>>>>>> all the necessary ingredients are provided.
> >>>>>>>>
> >>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance
> contrasts the
> >>>>>>>> fluency of other KStream methods.
> >>>>>>>>
> >>>>>>>> 2. (as Paul proposes)
> >>>>>>>>
> >>>>>>>> stream
> >>>>>>>>     .branch(predicate1, ks ->...)
> >>>>>>>>     .branch(predicate2, ks->...)
> >>>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both
> defaultBranch(..)
> >>>> and
> >>>>>>>> noDefault() return void
> >>>>>>>>
> >>>>>>>> PROS: Generally follows the way KStreams interface is defined.
> >>>>>>>>
> >>>>>>>> CONS: We need to define two terminal methods (defaultBranch(ks->)
> and
> >>>>>>>> noDefault()). And for a user it is very easy to miss the fact
> that one
> >>>>>>>> of the terminal methods should be called. If these methods are not
> >>>>>>>> called, we can throw an exception in runtime.
> >>>>>>>>
> >>>>>>>> Colleagues, what are your thoughts? Can we do better?
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>>
> >>>>>>>> Ivan
> >>>>>>>>
> >>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
> >>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
> >>>>>>>>>> Paul,
> >>>>>>>>>>
> >>>>>>>>>> I see your point when you are talking about
> >>>>>>>>>> stream..branch..branch...default..
> >>>>>>>>>>
> >>>>>>>>>> Still, I believe that this cannot not be implemented the easy
> way.
> >>>>>>>>>> Maybe we all should think further.
> >>>>>>>>>>
> >>>>>>>>>> Let me comment on two of your ideas.
> >>>>>>>>>>
> >>>>>>>>>>> user could specify a terminal method that assumes nothing will
> >>>> reach
> >>>>>>>>>>> the default branch,
> >>>>>>>>>> throwing an exception if such a case occurs.
> >>>>>>>>>>
> >>>>>>>>>> 1) OK, apparently this should not be the only option besides
> >>>>>>>>>> `default`, because there are scenarios when we want to just
> silently
> >>>>>>>>>> drop the messages that didn't match any predicate. 2) Throwing
> an
> >>>>>>>>>> exception in the middle of data flow processing looks like a bad
> >>>> idea.
> >>>>>>>>>> In stream processing paradigm, I would prefer to emit a special
> >>>>>>>>>> message to a dedicated stream. This is exactly where `default`
> can
> >>>> be
> >>>>>>>>>> used.
> >>>>>>>>>>
> >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to
> track
> >>>>>>>>>>> dangling
> >>>>>>>>>> branches that haven't been terminated and raise a clear error
> >>>> before it
> >>>>>>>>>> becomes an issue.
> >>>>>>>>>>
> >>>>>>>>>> You mean a runtime exception, when the program is compiled and
> run?
> >>>>>>>>>> Well,  I'd prefer an API that simply won't compile if used
> >>>>>>>>>> incorrectly. Can we build such an API as a method chain starting
> >>>> from
> >>>>>>>>>> KStream object? There is a huge cost difference between runtime
> and
> >>>>>>>>>> compile-time errors. Even if a failure uncovers instantly on
> unit
> >>>>>>>>>> tests, it costs more for the project than a compilation failure.
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>>
> >>>>>>>>>> Ivan
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
> >>>>>>>>>>> Ivan,
> >>>>>>>>>>>
> >>>>>>>>>>> Good point about the terminal operation being required.  But is
> >>>> that
> >>>>>>>>>>> really
> >>>>>>>>>>> such a bad thing?  If the user doesn't want a defaultBranch
> they
> >>>> can
> >>>>>>>>>>> call
> >>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as
> easily.  In
> >>>>>>>>>>> fact I
> >>>>>>>>>>> think it creates an opportunity for a nicer API - a user could
> >>>> specify
> >>>>>>>> a
> >>>>>>>>>>> terminal method that assumes nothing will reach the default
> branch,
> >>>>>>>>>>> throwing an exception if such a case occurs.  That seems like
> an
> >>>>>>>>>>> improvement over the current branch() API, which allows for the
> >>>> more
> >>>>>>>>>>> subtle
> >>>>>>>>>>> behavior of records unexpectedly getting dropped.
> >>>>>>>>>>>
> >>>>>>>>>>> The need for a terminal operation certainly has to be well
> >>>>>>>>>>> documented, but
> >>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to
> track
> >>>>>>>>>>> dangling
> >>>>>>>>>>> branches that haven't been terminated and raise a clear error
> >>>> before it
> >>>>>>>>>>> becomes an issue.  Especially now that there is a "build step"
> >>>> where
> >>>>>>>> the
> >>>>>>>>>>> topology is actually wired up, when StreamsBuilder.build() is
> >>>> called.
> >>>>>>>>>>> Regarding onTopOf() returning its argument, I agree that it's
> >>>>>>>>>>> critical to
> >>>>>>>>>>> allow users to do other operations on the input stream.  With
> the
> >>>>>>>> fluent
> >>>>>>>>>>> solution, it ought to work the same way all other operations
> do -
> >>>> if
> >>>>>>>> you
> >>>>>>>>>>> want to process off the original KStream multiple times, you
> just
> >>>>>>>>>>> need the
> >>>>>>>>>>> stream as a variable so you can call as many operations on it
> as
> >>>> you
> >>>>>>>>>>> desire.
> >>>>>>>>>>>
> >>>>>>>>>>> Thoughts?
> >>>>>>>>>>>
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Paul
> >>>>>>>>>>>
> >>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <
> iponomarev@mail.ru
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hello Paul,
> >>>>>>>>>>>>
> >>>>>>>>>>>> I afraid this won't work because we do not always need the
> >>>>>>>>>>>> defaultBranch. And without a terminal operation we don't know
> >>>> when to
> >>>>>>>>>>>> finalize and build the 'branch switch'.
> >>>>>>>>>>>>
> >>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we can do
> >>>> something
> >>>>>>>>>>>> more with the original branch after branching.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I understand your point that the need of special object
> >>>> construction
> >>>>>>>>>>>> contrasts the fluency of most KStream methods. But here we
> have a
> >>>>>>>>>>>> special case: we build the switch to split the flow, so I
> think
> >>>> this
> >>>>>>>> is
> >>>>>>>>>>>> still idiomatic.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Ivan
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
> >>>>>>>>>>>>> Ivan,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I think it's a great idea to improve this API, but I find the
> >>>>>>>>>>>>> onTopOff()
> >>>>>>>>>>>>> mechanism a little confusing since it contrasts the fluency
> of
> >>>> other
> >>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call a
> method on
> >>>> the
> >>>>>>>>>>>> stream
> >>>>>>>>>>>>> so it still reads top to bottom if the branch cases are
> defined
> >>>>>>>>>>>>> fluently.
> >>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very nice
> and the
> >>>>>>>>>>>>> right
> >>>>>>>>>>>> way
> >>>>>>>>>>>>> to do things, but what if we flipped around how we specify
> the
> >>>> source
> >>>>>>>>>>>>> stream.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Like:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> stream.branch()
> >>>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
> >>>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
> >>>>>>>>>>>>>            .defaultBranch(this::handleDefault);
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Where branch() returns a KBranchedStreams or KStreamBrancher
> or
> >>>>>>>>>>>> something,
> >>>>>>>>>>>>> which is added to by addBranch() and terminated by
> >>>> defaultBranch()
> >>>>>>>>>>>>> (which
> >>>>>>>>>>>>> returns void).  This is obviously incompatible with the
> current
> >>>>>>>>>>>>> API, so
> >>>>>>>>>>>> the
> >>>>>>>>>>>>> new stream.branch() would have to have a different name, but
> that
> >>>>>>>>>>>>> seems
> >>>>>>>>>>>>> like a fairly small problem - we could call it something like
> >>>>>>>>>>>>> branched()
> >>>>>>>>>>>> or
> >>>>>>>>>>>>> branchedStreams() and deprecate the old API.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It seems
> like it
> >>>>>>>>>>>>> does to
> >>>>>>>>>>>>> me, allowing for clear in-line branching while also allowing
> you
> >>>> to
> >>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams if
> desired.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>> Paul
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
> >>>>>>>>>>>> <ip...@mail.ru.invalid>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Bill,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thank you for your reply!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> This is how I usually do it:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
> >>>>>>>>>>>>>>            ks.filter(....).mapValues(...)
> >>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
> >>>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
> >>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> ......
> >>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
> >>>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
> >>>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
> >>>>>>>>>>>>>>       .onTopOf(....)
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Ivan
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
> >>>>>>>>>>>>>>> Hi Ivan,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks for the KIP.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher takes a
> Consumer
> >>>> as a
> >>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>> argument which returns nothing, and the example in the KIP
> >>>> shows
> >>>>>>>>>>>>>>> each
> >>>>>>>>>>>>>>> stream from the branch using a terminal node
> (KafkaStreams#to()
> >>>>>>>>>>>>>>> in this
> >>>>>>>>>>>>>>> case).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Maybe I've missed something, but how would we handle the
> case
> >>>>>>>>>>>>>>> where the
> >>>>>>>>>>>>>>> user has created a branch but wants to continue processing
> and
> >>>> not
> >>>>>>>>>>>>>>> necessarily use a terminal node on the branched stream
> >>>> immediately?
> >>>>>>>>>>>>>>> For example, using today's logic as is if we had something
> like
> >>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> KStream<String, String>[] branches =
> >>>>>>>>>>>>>>> originalStream.branch(predicate1,
> >>>>>>>>>>>>>>> predicate2);
> >>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
> >>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks!
> >>>>>>>>>>>>>>> Bill
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <
> bbejeck@gmail.com
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> All,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP- 418.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Here's the original message:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hello,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418. Please take
> a
> >>>> look
> >>>>>>>> at
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> KIP-418:
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
> >>>>>>>>>>>>>>>> JIRA KAFKA-5488:
> >>>> https://issues.apache.org/jira/browse/KAFKA-5488
> >>>>>>>>>>>>>>>> PR#6164: https://github.com/apache/kafka/pull/6164
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Ivan Ponomarev
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >
>
>

Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks for updating the KIP and your answers.


>  this is to make the name similar to String#split
>> that also returns an array, right?

The intend was to avoid name duplication. The return type should _not_
be an array.

The current proposal is

stream.branch()
      .branch(Predicate)
      .branch(Predicate)
      .defaultBranch();

IMHO, this reads a little odd, because the first `branch()` does not
take any parameters and has different semantics than the later
`branch()` calls. Note, that from the code snippet above, it's hidden
that the first call is `KStream#branch()` while the others are
`KBranchedStream#branch()` what makes reading the code harder.

Because I suggested to rename `addBranch()` -> `branch()`, I though it
might be better to also rename `KStream#branch()` to avoid the naming
overlap that seems to be confusing. The following reads much cleaner to me:

stream.split()
      .branch(Predicate)
      .branch(Predicate)
      .defaultBranch();

Maybe there is a better alternative to `split()` though to avoid the
naming overlap.


> 'default' is, however, a reserved word, so unfortunately we cannot have a method with such name :-)

Bummer. Didn't consider this. Maybe we can still come up with a short name?


Can you add the interface `KBranchedStream` to the KIP with all it's
methods? It will be part of public API and should be contained in the
KIP. For example, it's unclear atm, what the return type of
`defaultBranch()` is.


You did not comment on the idea to add a `KBranchedStream#get(int index)
-> KStream` method to get the individually branched-KStreams. Would be
nice to get your feedback about it. It seems you suggest that users
would need to write custom utility code otherwise, to access them. We
should discuss the pros and cons of both approaches. It feels
"incomplete" to me atm, if the API has no built-in support to get the
branched-KStreams directly.



-Matthias


On 4/13/19 2:13 AM, Ivan Ponomarev wrote:
> Hi all!
> 
> I have updated the KIP-418 according to the new vision.
> 
> Matthias, thanks for your comment!
> 
>> Renaming KStream#branch() -> #split()
> 
> I can see your point: this is to make the name similar to String#split
> that also returns an array, right? But is it worth the loss of backwards
> compatibility? We can have overloaded branch() as well without affecting
> the existing code. Maybe the old array-based `branch` method should be
> deprecated, but this is a subject for discussion.
> 
>> Renaming KBranchedStream#addBranch() -> BranchingKStream#branch(),
> KBranchedStream#defaultBranch() -> BranchingKStream#default()
> 
> Totally agree with 'addBranch->branch' rename. 'default' is, however, a
> reserved word, so unfortunately we cannot have a method with such name :-)
> 
>> defaultBranch() does take an `Predicate` as argument, but I think that
> is not required?
> 
> Absolutely! I think that was just copy-paste error or something.
> 
> Dear colleagues,
> 
> please revise the new version of the KIP and Paul's PR
> (https://github.com/apache/kafka/pull/6512)
> 
> Any new suggestions/objections?
> 
> Regards,
> 
> Ivan
> 
> 
> 11.04.2019 11:47, Matthias J. Sax пишет:
>> Thanks for driving the discussion of this KIP. It seems that everybody
>> agrees that the current branch() method using arrays is not optimal.
>>
>> I had a quick look into the PR and I like the overall proposal. There
>> are some minor things we need to consider. I would recommend the
>> following renaming:
>>
>> KStream#branch() -> #split()
>> KBranchedStream#addBranch() -> BranchingKStream#branch()
>> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>>
>> It's just a suggestion to get slightly shorter method names.
>>
>> In the current PR, defaultBranch() does take an `Predicate` as argument,
>> but I think that is not required?
>>
>> Also, we should consider KIP-307, that was recently accepted and is
>> currently implemented:
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>>
>> Ie, we should add overloads that accepted a `Named` parameter.
>>
>>
>> For the issue that the created `KStream` object are in different scopes:
>> could we extend `KBranchedStream` with a `get(int index)` method that
>> returns the corresponding "branched" result `KStream` object? Maybe, the
>> second argument of `addBranch()` should not be a `Consumer<KStream>` but
>> a `Function<KStream,KStream>` and `get()` could return whatever the
>> `Function` returns?
>>
>>
>> Finally, I would also suggest to update the KIP with the current
>> proposal. That makes it easier to review.
>>
>>
>> -Matthias
>>
>>
>>
>> On 3/31/19 12:22 PM, Paul Whalen wrote:
>>> Ivan,
>>>
>>> I'm a bit of a novice here as well, but I think it makes sense for you to
>>> revise the KIP and continue the discussion.  Obviously we'll need some
>>> buy-in from committers that have actual binding votes on whether the KIP
>>> could be adopted.  It would be great to hear if they think this is a good
>>> idea overall.  I'm not sure if that happens just by starting a vote, or if
>>> there is generally some indication of interest beforehand.
>>>
>>> That being said, I'll continue the discussion a bit: assuming we do move
>>> forward the solution of "stream.branch() returns KBranchedStream", do we
>>> deprecate "stream.branch(...) returns KStream[]"?  I would favor
>>> deprecating, since having two mutually exclusive APIs that accomplish the
>>> same thing is confusing, especially when they're fairly similar anyway.  We
>>> just need to be sure we're not making something impossible/difficult that
>>> is currently possible/easy.
>>>
>>> Regarding my PR - I think the general structure would work, it's just a
>>> little sloppy overall in terms of naming and clarity. In particular,
>>> passing in the "predicates" and "children" lists which get modified in
>>> KBranchedStream but read from all the way KStreamLazyBranch is a bit
>>> complicated to follow.
>>>
>>> Thanks,
>>> Paul
>>>
>>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <ip...@mail.ru> wrote:
>>>
>>>> Hi Paul!
>>>>
>>>> I read your code carefully and now I am fully convinced: your proposal
>>>> looks better and should work. We just have to document the crucial fact
>>>> that KStream consumers are invoked as they're added. And then it's all
>>>> going to be very nice.
>>>>
>>>> What shall we do now? I should re-write the KIP and resume the
>>>> discussion here, right?
>>>>
>>>> Why are you telling that your PR 'should not be even a starting point if
>>>> we go in this direction'? To me it looks like a good starting point. But
>>>> as a novice in this project I might miss some important details.
>>>>
>>>> Regards,
>>>>
>>>> Ivan
>>>>
>>>>
>>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>>> Ivan,
>>>>>
>>>>> Maybe I’m missing the point, but I believe the stream.branch() solution
>>>> supports this. The couponIssuer::set* consumers will be invoked as they’re
>>>> added, not during streamsBuilder.build(). So the user still ought to be
>>>> able to call couponIssuer.coupons() afterward and depend on the branched
>>>> streams having been set.
>>>>> The issue I mean to point out is that it is hard to access the branched
>>>> streams in the same scope as the original stream (that is, not inside the
>>>> couponIssuer), which is a problem with both proposed solutions. It can be
>>>> worked around though.
>>>>> [Also, great to hear additional interest in 401, I’m excited to hear
>>>> your thoughts!]
>>>>> Paul
>>>>>
>>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <ip...@mail.ru> wrote:
>>>>>>
>>>>>> Hi Paul!
>>>>>>
>>>>>> The idea to postpone the wiring of branches to the
>>>> streamsBuilder.build() also looked great for me at first glance, but ---
>>>>>>> the newly branched streams are not available in the same scope as each
>>>> other.  That is, if we wanted to merge them back together again I don't see
>>>> a way to do that.
>>>>>> You just took the words right out of my mouth, I was just going to
>>>> write in details about this issue.
>>>>>> Consider the example from Bill's book, p. 101: say we need to identify
>>>> customers who have bought coffee and made a purchase in the electronics
>>>> store to give them coupons.
>>>>>> This is the code I usually write under these circumstances using my
>>>> 'brancher' class:
>>>>>> @Setter
>>>>>> class CouponIssuer{
>>>>>>    private KStream<....> coffePurchases;
>>>>>>    private KStream<....> electronicsPurchases;
>>>>>>
>>>>>>    KStream<...> coupons(){
>>>>>>        return coffePurchases.join(electronicsPurchases...)...whatever
>>>>>>
>>>>>>        /*In the real world the code here can be complex, so creation of
>>>> a separate CouponIssuer class is fully justified, in order to separate
>>>> classes' responsibilities.*/
>>>>>>   }
>>>>>> }
>>>>>>
>>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>>
>>>>>> new KafkaStreamsBrancher<....>()
>>>>>>      .branch(predicate1, couponIssuer::setCoffePurchases)
>>>>>>      .branch(predicate2, couponIssuer::setElectronicsPurchases)
>>>>>>      .onTopOf(transactionStream);
>>>>>>
>>>>>> /*Alas, this won't work if we're going to wire up everything later,
>>>> without the terminal operation!!!*/
>>>>>> couponIssuer.coupons()...
>>>>>>
>>>>>> Does this make sense?  In order to properly initialize the CouponIssuer
>>>> we need the terminal operation to be called before streamsBuilder.build()
>>>> is called.
>>>>>> [BTW Paul, I just found out that your KIP-401 is essentially the next
>>>> KIP I was going to write here. I have some thoughts based on my experience,
>>>> so I will join the discussion on KIP-401 soon.]
>>>>>> Regards,
>>>>>>
>>>>>> Ivan
>>>>>>
>>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>>> Ivan,
>>>>>>> I tried to make a very rough proof of concept of a fluent API based
>>>> off of
>>>>>>> KStream here (https://github.com/apache/kafka/pull/6512), and I think
>>>> I
>>>>>>> succeeded at removing both cons.
>>>>>>>     - Compatibility: I was incorrect earlier about compatibility
>>>> issues,
>>>>>>>     there aren't any direct ones.  I was unaware that Java is smart
>>>> enough to
>>>>>>>     distinguish between a branch(varargs...) returning one thing and
>>>> branch()
>>>>>>>     with no arguments returning another thing.
>>>>>>>     - Requiring a terminal method: We don't actually need it.  We can
>>>> just
>>>>>>>     build up the branches in the KBranchedStream who shares its state
>>>> with the
>>>>>>>     ProcessorSupplier that will actually do the branching.  It's not
>>>> terribly
>>>>>>>     pretty in its current form, but I think it demonstrates its
>>>> feasibility.
>>>>>>> To be clear, I don't think that pull request should be final or even a
>>>>>>> starting point if we go in this direction, I just wanted to see how
>>>>>>> challenging it would be to get the API working.
>>>>>>> I will say though, that I'm not sure the existing solution could be
>>>>>>> deprecated in favor of this, which I had originally suggested was a
>>>>>>> possibility.  The reason is that the newly branched streams are not
>>>>>>> available in the same scope as each other.  That is, if we wanted to
>>>> merge
>>>>>>> them back together again I don't see a way to do that.  The KIP
>>>> proposal
>>>>>>> has the same issue, though - all this means is that for either
>>>> solution,
>>>>>>> deprecating the existing branch(...) is not on the table.
>>>>>>> Thanks,
>>>>>>> Paul
>>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <ip...@mail.ru>
>>>> wrote:
>>>>>>>> OK, let me summarize what we have discussed up to this point.
>>>>>>>>
>>>>>>>> First, it seems that it's commonly agreed that branch API needs
>>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>>
>>>>>>>> There are two potential ways to do it:
>>>>>>>>
>>>>>>>> 1. (as origianlly proposed)
>>>>>>>>
>>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>>     .branch(predicate1, ks ->..)
>>>>>>>>     .branch(predicate2, ks->..)
>>>>>>>>     .defaultBranch(ks->..) //optional
>>>>>>>>     .onTopOf(stream).mapValues(...).... //onTopOf returns its argument
>>>>>>>>
>>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't make sense
>>>> until
>>>>>>>> all the necessary ingredients are provided.
>>>>>>>>
>>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance contrasts the
>>>>>>>> fluency of other KStream methods.
>>>>>>>>
>>>>>>>> 2. (as Paul proposes)
>>>>>>>>
>>>>>>>> stream
>>>>>>>>     .branch(predicate1, ks ->...)
>>>>>>>>     .branch(predicate2, ks->...)
>>>>>>>>     .defaultBranch(ks->...) //or noDefault(). Both defaultBranch(..)
>>>> and
>>>>>>>> noDefault() return void
>>>>>>>>
>>>>>>>> PROS: Generally follows the way KStreams interface is defined.
>>>>>>>>
>>>>>>>> CONS: We need to define two terminal methods (defaultBranch(ks->) and
>>>>>>>> noDefault()). And for a user it is very easy to miss the fact that one
>>>>>>>> of the terminal methods should be called. If these methods are not
>>>>>>>> called, we can throw an exception in runtime.
>>>>>>>>
>>>>>>>> Colleagues, what are your thoughts? Can we do better?
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Ivan
>>>>>>>>
>>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>>> Paul,
>>>>>>>>>>
>>>>>>>>>> I see your point when you are talking about
>>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>>
>>>>>>>>>> Still, I believe that this cannot not be implemented the easy way.
>>>>>>>>>> Maybe we all should think further.
>>>>>>>>>>
>>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>>
>>>>>>>>>>> user could specify a terminal method that assumes nothing will
>>>> reach
>>>>>>>>>>> the default branch,
>>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>>>>
>>>>>>>>>> 1) OK, apparently this should not be the only option besides
>>>>>>>>>> `default`, because there are scenarios when we want to just silently
>>>>>>>>>> drop the messages that didn't match any predicate. 2) Throwing an
>>>>>>>>>> exception in the middle of data flow processing looks like a bad
>>>> idea.
>>>>>>>>>> In stream processing paradigm, I would prefer to emit a special
>>>>>>>>>> message to a dedicated stream. This is exactly where `default` can
>>>> be
>>>>>>>>>> used.
>>>>>>>>>>
>>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to track
>>>>>>>>>>> dangling
>>>>>>>>>> branches that haven't been terminated and raise a clear error
>>>> before it
>>>>>>>>>> becomes an issue.
>>>>>>>>>>
>>>>>>>>>> You mean a runtime exception, when the program is compiled and run?
>>>>>>>>>> Well,  I'd prefer an API that simply won't compile if used
>>>>>>>>>> incorrectly. Can we build such an API as a method chain starting
>>>> from
>>>>>>>>>> KStream object? There is a huge cost difference between runtime and
>>>>>>>>>> compile-time errors. Even if a failure uncovers instantly on unit
>>>>>>>>>> tests, it costs more for the project than a compilation failure.
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>>
>>>>>>>>>> Ivan
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>>> Ivan,
>>>>>>>>>>>
>>>>>>>>>>> Good point about the terminal operation being required.  But is
>>>> that
>>>>>>>>>>> really
>>>>>>>>>>> such a bad thing?  If the user doesn't want a defaultBranch they
>>>> can
>>>>>>>>>>> call
>>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as easily.  In
>>>>>>>>>>> fact I
>>>>>>>>>>> think it creates an opportunity for a nicer API - a user could
>>>> specify
>>>>>>>> a
>>>>>>>>>>> terminal method that assumes nothing will reach the default branch,
>>>>>>>>>>> throwing an exception if such a case occurs.  That seems like an
>>>>>>>>>>> improvement over the current branch() API, which allows for the
>>>> more
>>>>>>>>>>> subtle
>>>>>>>>>>> behavior of records unexpectedly getting dropped.
>>>>>>>>>>>
>>>>>>>>>>> The need for a terminal operation certainly has to be well
>>>>>>>>>>> documented, but
>>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to track
>>>>>>>>>>> dangling
>>>>>>>>>>> branches that haven't been terminated and raise a clear error
>>>> before it
>>>>>>>>>>> becomes an issue.  Especially now that there is a "build step"
>>>> where
>>>>>>>> the
>>>>>>>>>>> topology is actually wired up, when StreamsBuilder.build() is
>>>> called.
>>>>>>>>>>> Regarding onTopOf() returning its argument, I agree that it's
>>>>>>>>>>> critical to
>>>>>>>>>>> allow users to do other operations on the input stream.  With the
>>>>>>>> fluent
>>>>>>>>>>> solution, it ought to work the same way all other operations do -
>>>> if
>>>>>>>> you
>>>>>>>>>>> want to process off the original KStream multiple times, you just
>>>>>>>>>>> need the
>>>>>>>>>>> stream as a variable so you can call as many operations on it as
>>>> you
>>>>>>>>>>> desire.
>>>>>>>>>>>
>>>>>>>>>>> Thoughts?
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Paul
>>>>>>>>>>>
>>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <iponomarev@mail.ru
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>>
>>>>>>>>>>>> I afraid this won't work because we do not always need the
>>>>>>>>>>>> defaultBranch. And without a terminal operation we don't know
>>>> when to
>>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>>>>
>>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we can do
>>>> something
>>>>>>>>>>>> more with the original branch after branching.
>>>>>>>>>>>>
>>>>>>>>>>>> I understand your point that the need of special object
>>>> construction
>>>>>>>>>>>> contrasts the fluency of most KStream methods. But here we have a
>>>>>>>>>>>> special case: we build the switch to split the flow, so I think
>>>> this
>>>>>>>> is
>>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Ivan
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I think it's a great idea to improve this API, but I find the
>>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>>> mechanism a little confusing since it contrasts the fluency of
>>>> other
>>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call a method on
>>>> the
>>>>>>>>>>>> stream
>>>>>>>>>>>>> so it still reads top to bottom if the branch cases are defined
>>>>>>>>>>>>> fluently.
>>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very nice and the
>>>>>>>>>>>>> right
>>>>>>>>>>>> way
>>>>>>>>>>>>> to do things, but what if we flipped around how we specify the
>>>> source
>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Like:
>>>>>>>>>>>>>
>>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>>            .addBranch(predicate1, this::handle1)
>>>>>>>>>>>>>            .addBranch(predicate2, this::handle2)
>>>>>>>>>>>>>            .defaultBranch(this::handleDefault);
>>>>>>>>>>>>>
>>>>>>>>>>>>> Where branch() returns a KBranchedStreams or KStreamBrancher or
>>>>>>>>>>>> something,
>>>>>>>>>>>>> which is added to by addBranch() and terminated by
>>>> defaultBranch()
>>>>>>>>>>>>> (which
>>>>>>>>>>>>> returns void).  This is obviously incompatible with the current
>>>>>>>>>>>>> API, so
>>>>>>>>>>>> the
>>>>>>>>>>>>> new stream.branch() would have to have a different name, but that
>>>>>>>>>>>>> seems
>>>>>>>>>>>>> like a fairly small problem - we could call it something like
>>>>>>>>>>>>> branched()
>>>>>>>>>>>> or
>>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It seems like it
>>>>>>>>>>>>> does to
>>>>>>>>>>>>> me, allowing for clear in-line branching while also allowing you
>>>> to
>>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams if desired.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Paul
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
>>>>>>>>>>>>>>            ks.filter(....).mapValues(...)
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
>>>>>>>>>>>>>>            ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> ......
>>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>>>>>>>>       .addBranch(predicate1, this::handleFirstCase)
>>>>>>>>>>>>>>       .addBranch(predicate2, this::handleSecondCase)
>>>>>>>>>>>>>>       .onTopOf(....)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher takes a Consumer
>>>> as a
>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>> argument which returns nothing, and the example in the KIP
>>>> shows
>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>> stream from the branch using a terminal node (KafkaStreams#to()
>>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Maybe I've missed something, but how would we handle the case
>>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>>> user has created a branch but wants to continue processing and
>>>> not
>>>>>>>>>>>>>>> necessarily use a terminal node on the branched stream
>>>> immediately?
>>>>>>>>>>>>>>> For example, using today's logic as is if we had something like
>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <bbejeck@gmail.com
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP- 418.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418. Please take a
>>>> look
>>>>>>>> at
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> KIP-418:
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>>> PR#6164: https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
> 


Re: [DISCUSSION] KIP-418: A method-chaining way to branch KStream

Posted by Ivan Ponomarev <ip...@mail.ru.INVALID>.
Hi all!

I have updated the KIP-418 according to the new vision.

Matthias, thanks for your comment!

> Renaming KStream#branch() -> #split()

I can see your point: this is to make the name similar to String#split 
that also returns an array, right? But is it worth the loss of backwards 
compatibility? We can have overloaded branch() as well without affecting 
the existing code. Maybe the old array-based `branch` method should be 
deprecated, but this is a subject for discussion.

 > Renaming KBranchedStream#addBranch() -> BranchingKStream#branch(), 
KBranchedStream#defaultBranch() -> BranchingKStream#default()

Totally agree with 'addBranch->branch' rename. 'default' is, however, a 
reserved word, so unfortunately we cannot have a method with such name :-)

 > defaultBranch() does take an `Predicate` as argument, but I think 
that is not required?

Absolutely! I think that was just copy-paste error or something.

Dear colleagues,

please revise the new version of the KIP and Paul's PR 
(https://github.com/apache/kafka/pull/6512)

Any new suggestions/objections?

Regards,

Ivan


11.04.2019 11:47, Matthias J. Sax пишет:
> Thanks for driving the discussion of this KIP. It seems that everybody
> agrees that the current branch() method using arrays is not optimal.
>
> I had a quick look into the PR and I like the overall proposal. There
> are some minor things we need to consider. I would recommend the
> following renaming:
>
> KStream#branch() -> #split()
> KBranchedStream#addBranch() -> BranchingKStream#branch()
> KBranchedStream#defaultBranch() -> BranchingKStream#default()
>
> It's just a suggestion to get slightly shorter method names.
>
> In the current PR, defaultBranch() does take an `Predicate` as argument,
> but I think that is not required?
>
> Also, we should consider KIP-307, that was recently accepted and is
> currently implemented:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-307%3A+Allow+to+define+custom+processor+names+with+KStreams+DSL
>
> Ie, we should add overloads that accepted a `Named` parameter.
>
>
> For the issue that the created `KStream` object are in different scopes:
> could we extend `KBranchedStream` with a `get(int index)` method that
> returns the corresponding "branched" result `KStream` object? Maybe, the
> second argument of `addBranch()` should not be a `Consumer<KStream>` but
> a `Function<KStream,KStream>` and `get()` could return whatever the
> `Function` returns?
>
>
> Finally, I would also suggest to update the KIP with the current
> proposal. That makes it easier to review.
>
>
> -Matthias
>
>
>
> On 3/31/19 12:22 PM, Paul Whalen wrote:
>> Ivan,
>>
>> I'm a bit of a novice here as well, but I think it makes sense for you to
>> revise the KIP and continue the discussion.  Obviously we'll need some
>> buy-in from committers that have actual binding votes on whether the KIP
>> could be adopted.  It would be great to hear if they think this is a good
>> idea overall.  I'm not sure if that happens just by starting a vote, or if
>> there is generally some indication of interest beforehand.
>>
>> That being said, I'll continue the discussion a bit: assuming we do move
>> forward the solution of "stream.branch() returns KBranchedStream", do we
>> deprecate "stream.branch(...) returns KStream[]"?  I would favor
>> deprecating, since having two mutually exclusive APIs that accomplish the
>> same thing is confusing, especially when they're fairly similar anyway.  We
>> just need to be sure we're not making something impossible/difficult that
>> is currently possible/easy.
>>
>> Regarding my PR - I think the general structure would work, it's just a
>> little sloppy overall in terms of naming and clarity. In particular,
>> passing in the "predicates" and "children" lists which get modified in
>> KBranchedStream but read from all the way KStreamLazyBranch is a bit
>> complicated to follow.
>>
>> Thanks,
>> Paul
>>
>> On Fri, Mar 29, 2019 at 5:37 AM Ivan Ponomarev <ip...@mail.ru> wrote:
>>
>>> Hi Paul!
>>>
>>> I read your code carefully and now I am fully convinced: your proposal
>>> looks better and should work. We just have to document the crucial fact
>>> that KStream consumers are invoked as they're added. And then it's all
>>> going to be very nice.
>>>
>>> What shall we do now? I should re-write the KIP and resume the
>>> discussion here, right?
>>>
>>> Why are you telling that your PR 'should not be even a starting point if
>>> we go in this direction'? To me it looks like a good starting point. But
>>> as a novice in this project I might miss some important details.
>>>
>>> Regards,
>>>
>>> Ivan
>>>
>>>
>>> 28.03.2019 17:38, Paul Whalen пишет:
>>>> Ivan,
>>>>
>>>> Maybe I’m missing the point, but I believe the stream.branch() solution
>>> supports this. The couponIssuer::set* consumers will be invoked as they’re
>>> added, not during streamsBuilder.build(). So the user still ought to be
>>> able to call couponIssuer.coupons() afterward and depend on the branched
>>> streams having been set.
>>>> The issue I mean to point out is that it is hard to access the branched
>>> streams in the same scope as the original stream (that is, not inside the
>>> couponIssuer), which is a problem with both proposed solutions. It can be
>>> worked around though.
>>>> [Also, great to hear additional interest in 401, I’m excited to hear
>>> your thoughts!]
>>>> Paul
>>>>
>>>>> On Mar 28, 2019, at 4:00 AM, Ivan Ponomarev <ip...@mail.ru> wrote:
>>>>>
>>>>> Hi Paul!
>>>>>
>>>>> The idea to postpone the wiring of branches to the
>>> streamsBuilder.build() also looked great for me at first glance, but ---
>>>>>> the newly branched streams are not available in the same scope as each
>>> other.  That is, if we wanted to merge them back together again I don't see
>>> a way to do that.
>>>>> You just took the words right out of my mouth, I was just going to
>>> write in details about this issue.
>>>>> Consider the example from Bill's book, p. 101: say we need to identify
>>> customers who have bought coffee and made a purchase in the electronics
>>> store to give them coupons.
>>>>> This is the code I usually write under these circumstances using my
>>> 'brancher' class:
>>>>> @Setter
>>>>> class CouponIssuer{
>>>>>     private KStream<....> coffePurchases;
>>>>>     private KStream<....> electronicsPurchases;
>>>>>
>>>>>     KStream<...> coupons(){
>>>>>         return coffePurchases.join(electronicsPurchases...)...whatever
>>>>>
>>>>>         /*In the real world the code here can be complex, so creation of
>>> a separate CouponIssuer class is fully justified, in order to separate
>>> classes' responsibilities.*/
>>>>>    }
>>>>> }
>>>>>
>>>>> CouponIssuer couponIssuer = new CouponIssuer();
>>>>>
>>>>> new KafkaStreamsBrancher<....>()
>>>>>       .branch(predicate1, couponIssuer::setCoffePurchases)
>>>>>       .branch(predicate2, couponIssuer::setElectronicsPurchases)
>>>>>       .onTopOf(transactionStream);
>>>>>
>>>>> /*Alas, this won't work if we're going to wire up everything later,
>>> without the terminal operation!!!*/
>>>>> couponIssuer.coupons()...
>>>>>
>>>>> Does this make sense?  In order to properly initialize the CouponIssuer
>>> we need the terminal operation to be called before streamsBuilder.build()
>>> is called.
>>>>>
>>>>> [BTW Paul, I just found out that your KIP-401 is essentially the next
>>> KIP I was going to write here. I have some thoughts based on my experience,
>>> so I will join the discussion on KIP-401 soon.]
>>>>> Regards,
>>>>>
>>>>> Ivan
>>>>>
>>>>> 28.03.2019 6:29, Paul Whalen пишет:
>>>>>> Ivan,
>>>>>> I tried to make a very rough proof of concept of a fluent API based
>>> off of
>>>>>> KStream here (https://github.com/apache/kafka/pull/6512), and I think
>>> I
>>>>>> succeeded at removing both cons.
>>>>>>      - Compatibility: I was incorrect earlier about compatibility
>>> issues,
>>>>>>      there aren't any direct ones.  I was unaware that Java is smart
>>> enough to
>>>>>>      distinguish between a branch(varargs...) returning one thing and
>>> branch()
>>>>>>      with no arguments returning another thing.
>>>>>>      - Requiring a terminal method: We don't actually need it.  We can
>>> just
>>>>>>      build up the branches in the KBranchedStream who shares its state
>>> with the
>>>>>>      ProcessorSupplier that will actually do the branching.  It's not
>>> terribly
>>>>>>      pretty in its current form, but I think it demonstrates its
>>> feasibility.
>>>>>> To be clear, I don't think that pull request should be final or even a
>>>>>> starting point if we go in this direction, I just wanted to see how
>>>>>> challenging it would be to get the API working.
>>>>>> I will say though, that I'm not sure the existing solution could be
>>>>>> deprecated in favor of this, which I had originally suggested was a
>>>>>> possibility.  The reason is that the newly branched streams are not
>>>>>> available in the same scope as each other.  That is, if we wanted to
>>> merge
>>>>>> them back together again I don't see a way to do that.  The KIP
>>> proposal
>>>>>> has the same issue, though - all this means is that for either
>>> solution,
>>>>>> deprecating the existing branch(...) is not on the table.
>>>>>> Thanks,
>>>>>> Paul
>>>>>>> On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev <ip...@mail.ru>
>>> wrote:
>>>>>>> OK, let me summarize what we have discussed up to this point.
>>>>>>>
>>>>>>> First, it seems that it's commonly agreed that branch API needs
>>>>>>> improvement. Motivation is given in the KIP.
>>>>>>>
>>>>>>> There are two potential ways to do it:
>>>>>>>
>>>>>>> 1. (as origianlly proposed)
>>>>>>>
>>>>>>> new KafkaStreamsBrancher<..>()
>>>>>>>      .branch(predicate1, ks ->..)
>>>>>>>      .branch(predicate2, ks->..)
>>>>>>>      .defaultBranch(ks->..) //optional
>>>>>>>      .onTopOf(stream).mapValues(...).... //onTopOf returns its argument
>>>>>>>
>>>>>>> PROS: 1) Fully backwards compatible. 2) The code won't make sense
>>> until
>>>>>>> all the necessary ingredients are provided.
>>>>>>>
>>>>>>> CONS: The need to create a KafkaStreamsBrancher instance contrasts the
>>>>>>> fluency of other KStream methods.
>>>>>>>
>>>>>>> 2. (as Paul proposes)
>>>>>>>
>>>>>>> stream
>>>>>>>      .branch(predicate1, ks ->...)
>>>>>>>      .branch(predicate2, ks->...)
>>>>>>>      .defaultBranch(ks->...) //or noDefault(). Both defaultBranch(..)
>>> and
>>>>>>> noDefault() return void
>>>>>>>
>>>>>>> PROS: Generally follows the way KStreams interface is defined.
>>>>>>>
>>>>>>> CONS: We need to define two terminal methods (defaultBranch(ks->) and
>>>>>>> noDefault()). And for a user it is very easy to miss the fact that one
>>>>>>> of the terminal methods should be called. If these methods are not
>>>>>>> called, we can throw an exception in runtime.
>>>>>>>
>>>>>>> Colleagues, what are your thoughts? Can we do better?
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> Ivan
>>>>>>>
>>>>>>> 27.03.2019 18:46, Ivan Ponomarev пишет:
>>>>>>>>
>>>>>>>> 25.03.2019 17:43, Ivan Ponomarev пишет:
>>>>>>>>> Paul,
>>>>>>>>>
>>>>>>>>> I see your point when you are talking about
>>>>>>>>> stream..branch..branch...default..
>>>>>>>>>
>>>>>>>>> Still, I believe that this cannot not be implemented the easy way.
>>>>>>>>> Maybe we all should think further.
>>>>>>>>>
>>>>>>>>> Let me comment on two of your ideas.
>>>>>>>>>
>>>>>>>>>> user could specify a terminal method that assumes nothing will
>>> reach
>>>>>>>>>> the default branch,
>>>>>>>>> throwing an exception if such a case occurs.
>>>>>>>>>
>>>>>>>>> 1) OK, apparently this should not be the only option besides
>>>>>>>>> `default`, because there are scenarios when we want to just silently
>>>>>>>>> drop the messages that didn't match any predicate. 2) Throwing an
>>>>>>>>> exception in the middle of data flow processing looks like a bad
>>> idea.
>>>>>>>>> In stream processing paradigm, I would prefer to emit a special
>>>>>>>>> message to a dedicated stream. This is exactly where `default` can
>>> be
>>>>>>>>> used.
>>>>>>>>>
>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to track
>>>>>>>>>> dangling
>>>>>>>>> branches that haven't been terminated and raise a clear error
>>> before it
>>>>>>>>> becomes an issue.
>>>>>>>>>
>>>>>>>>> You mean a runtime exception, when the program is compiled and run?
>>>>>>>>> Well,  I'd prefer an API that simply won't compile if used
>>>>>>>>> incorrectly. Can we build such an API as a method chain starting
>>> from
>>>>>>>>> KStream object? There is a huge cost difference between runtime and
>>>>>>>>> compile-time errors. Even if a failure uncovers instantly on unit
>>>>>>>>> tests, it costs more for the project than a compilation failure.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Ivan
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 25.03.2019 0:38, Paul Whalen пишет:
>>>>>>>>>> Ivan,
>>>>>>>>>>
>>>>>>>>>> Good point about the terminal operation being required.  But is
>>> that
>>>>>>>>>> really
>>>>>>>>>> such a bad thing?  If the user doesn't want a defaultBranch they
>>> can
>>>>>>>>>> call
>>>>>>>>>> some other terminal method (noDefaultBranch()?) just as easily.  In
>>>>>>>>>> fact I
>>>>>>>>>> think it creates an opportunity for a nicer API - a user could
>>> specify
>>>>>>> a
>>>>>>>>>> terminal method that assumes nothing will reach the default branch,
>>>>>>>>>> throwing an exception if such a case occurs.  That seems like an
>>>>>>>>>> improvement over the current branch() API, which allows for the
>>> more
>>>>>>>>>> subtle
>>>>>>>>>> behavior of records unexpectedly getting dropped.
>>>>>>>>>>
>>>>>>>>>> The need for a terminal operation certainly has to be well
>>>>>>>>>> documented, but
>>>>>>>>>> it would be fairly easily for the InternalTopologyBuilder to track
>>>>>>>>>> dangling
>>>>>>>>>> branches that haven't been terminated and raise a clear error
>>> before it
>>>>>>>>>> becomes an issue.  Especially now that there is a "build step"
>>> where
>>>>>>> the
>>>>>>>>>> topology is actually wired up, when StreamsBuilder.build() is
>>> called.
>>>>>>>>>> Regarding onTopOf() returning its argument, I agree that it's
>>>>>>>>>> critical to
>>>>>>>>>> allow users to do other operations on the input stream.  With the
>>>>>>> fluent
>>>>>>>>>> solution, it ought to work the same way all other operations do -
>>> if
>>>>>>> you
>>>>>>>>>> want to process off the original KStream multiple times, you just
>>>>>>>>>> need the
>>>>>>>>>> stream as a variable so you can call as many operations on it as
>>> you
>>>>>>>>>> desire.
>>>>>>>>>>
>>>>>>>>>> Thoughts?
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Paul
>>>>>>>>>>
>>>>>>>>>> On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev <iponomarev@mail.ru
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hello Paul,
>>>>>>>>>>>
>>>>>>>>>>> I afraid this won't work because we do not always need the
>>>>>>>>>>> defaultBranch. And without a terminal operation we don't know
>>> when to
>>>>>>>>>>> finalize and build the 'branch switch'.
>>>>>>>>>>>
>>>>>>>>>>> In my proposal, onTopOf returns its argument, so we can do
>>> something
>>>>>>>>>>> more with the original branch after branching.
>>>>>>>>>>>
>>>>>>>>>>> I understand your point that the need of special object
>>> construction
>>>>>>>>>>> contrasts the fluency of most KStream methods. But here we have a
>>>>>>>>>>> special case: we build the switch to split the flow, so I think
>>> this
>>>>>>> is
>>>>>>>>>>> still idiomatic.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>>
>>>>>>>>>>> Ivan
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 24.03.2019 4:02, Paul Whalen пишет:
>>>>>>>>>>>> Ivan,
>>>>>>>>>>>>
>>>>>>>>>>>> I think it's a great idea to improve this API, but I find the
>>>>>>>>>>>> onTopOff()
>>>>>>>>>>>> mechanism a little confusing since it contrasts the fluency of
>>> other
>>>>>>>>>>>> KStream method calls.  Ideally I'd like to just call a method on
>>> the
>>>>>>>>>>> stream
>>>>>>>>>>>> so it still reads top to bottom if the branch cases are defined
>>>>>>>>>>>> fluently.
>>>>>>>>>>>> I think the addBranch(predicate, handleCase) is very nice and the
>>>>>>>>>>>> right
>>>>>>>>>>> way
>>>>>>>>>>>> to do things, but what if we flipped around how we specify the
>>> source
>>>>>>>>>>>> stream.
>>>>>>>>>>>>
>>>>>>>>>>>> Like:
>>>>>>>>>>>>
>>>>>>>>>>>> stream.branch()
>>>>>>>>>>>>             .addBranch(predicate1, this::handle1)
>>>>>>>>>>>>             .addBranch(predicate2, this::handle2)
>>>>>>>>>>>>             .defaultBranch(this::handleDefault);
>>>>>>>>>>>>
>>>>>>>>>>>> Where branch() returns a KBranchedStreams or KStreamBrancher or
>>>>>>>>>>> something,
>>>>>>>>>>>> which is added to by addBranch() and terminated by
>>> defaultBranch()
>>>>>>>>>>>> (which
>>>>>>>>>>>> returns void).  This is obviously incompatible with the current
>>>>>>>>>>>> API, so
>>>>>>>>>>> the
>>>>>>>>>>>> new stream.branch() would have to have a different name, but that
>>>>>>>>>>>> seems
>>>>>>>>>>>> like a fairly small problem - we could call it something like
>>>>>>>>>>>> branched()
>>>>>>>>>>> or
>>>>>>>>>>>> branchedStreams() and deprecate the old API.
>>>>>>>>>>>>
>>>>>>>>>>>> Does this satisfy the motivations of your KIP?  It seems like it
>>>>>>>>>>>> does to
>>>>>>>>>>>> me, allowing for clear in-line branching while also allowing you
>>> to
>>>>>>>>>>>> dynamically build of branches off of KBranchedStreams if desired.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Paul
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev
>>>>>>>>>>> <ip...@mail.ru.invalid>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Bill,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thank you for your reply!
>>>>>>>>>>>>>
>>>>>>>>>>>>> This is how I usually do it:
>>>>>>>>>>>>>
>>>>>>>>>>>>> void handleFirstCase(KStream<String, String> ks){
>>>>>>>>>>>>>             ks.filter(....).mapValues(...)
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> void handleSecondCase(KStream<String, String> ks){
>>>>>>>>>>>>>             ks.selectKey(...).groupByKey()...
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> ......
>>>>>>>>>>>>> new KafkaStreamsBrancher<String, String>()
>>>>>>>>>>>>>        .addBranch(predicate1, this::handleFirstCase)
>>>>>>>>>>>>>        .addBranch(predicate2, this::handleSecondCase)
>>>>>>>>>>>>>        .onTopOf(....)
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ivan
>>>>>>>>>>>>>
>>>>>>>>>>>>> 22.03.2019 1:34, Bill Bejeck пишет:
>>>>>>>>>>>>>> Hi Ivan,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I have one question, the KafkaStreamsBrancher takes a Consumer
>>> as a
>>>>>>>>>>>>> second
>>>>>>>>>>>>>> argument which returns nothing, and the example in the KIP
>>> shows
>>>>>>>>>>>>>> each
>>>>>>>>>>>>>> stream from the branch using a terminal node (KafkaStreams#to()
>>>>>>>>>>>>>> in this
>>>>>>>>>>>>>> case).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Maybe I've missed something, but how would we handle the case
>>>>>>>>>>>>>> where the
>>>>>>>>>>>>>> user has created a branch but wants to continue processing and
>>> not
>>>>>>>>>>>>>> necessarily use a terminal node on the branched stream
>>> immediately?
>>>>>>>>>>>>>> For example, using today's logic as is if we had something like
>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> KStream<String, String>[] branches =
>>>>>>>>>>>>>> originalStream.branch(predicate1,
>>>>>>>>>>>>>> predicate2);
>>>>>>>>>>>>>> branches[0].filter(....).mapValues(...)..
>>>>>>>>>>>>>> branches[1].selectKey(...).groupByKey().....
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks!
>>>>>>>>>>>>>> Bill
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Mar 21, 2019 at 6:15 PM Bill Bejeck <bbejeck@gmail.com
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> All,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'd like to jump-start the discussion for KIP- 418.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Here's the original message:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'd like to start a discussion about KIP-418. Please take a
>>> look
>>>>>>> at
>>>>>>>>>>> the
>>>>>>>>>>>>>>> KIP if you can, I would appreciate any feedback :)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> KIP-418:
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream
>>>>>>>>>>>>>>> JIRA KAFKA-5488:
>>> https://issues.apache.org/jira/browse/KAFKA-5488
>>>>>>>>>>>>>>> PR#6164: https://github.com/apache/kafka/pull/6164
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ivan Ponomarev
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>