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 work
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 ver
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 propos
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 canno
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 pri
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
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
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, wit
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 fo
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 discu
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 m
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
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 g
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
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 requ
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
wrote:
> Interesting discussion.
>
> I am wondering, if we cannot unify the advantage of both approaches:
>
>
>
> KStream#split() -> KBranchedStream
>
> // branch is
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)
-> KBranchedStream
// assign a name to the branch and
// return the
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 prese
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 sam
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
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`.
It makes the code e
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 t
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 wrote:
> Also +1 on the issues/goals as Michael outlined them, I think th
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
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),
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
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
.br
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)
.br
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
compatibi
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:
KS
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
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 KI
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 branch
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 d
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
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, k
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 re
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 spec
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 branch
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.
Hi Bill,
Thank you for your reply!
This is how I usually do it:
void handleFirstCase(KStream ks){
ks.filter().mapValues(...)
}
void handleSecondCase(KStream ks){
ks.selectKey(...).groupByKey()...
}
..
new KafkaStreamsBrancher()
.addBranch(predicate1, this::handleFir
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
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+metho
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/bro
44 matches
Mail list logo