Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-04-22 Thread Matthias J. Sax
Nick, how should we proceed? On 3/29/22 9:21 PM, Chris Egerton wrote: Hi all, The Named-first variant seems pretty appealing: merge(Named named, KStream... streams) It doesn't align with the existing merge methods, but it does at least follow precedent set by the (now-deprecated) branch me

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-29 Thread Chris Egerton
Hi all, The Named-first variant seems pretty appealing: merge(Named named, KStream... streams) It doesn't align with the existing merge methods, but it does at least follow precedent set by the (now-deprecated) branch method [1]. A Collection-based alternative seems slightly less appealing, b

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-29 Thread Matthias J. Sax
My understand was, that the original proposal was to have: merge(KStream stream); merge(KStream... streams); Maybe I misunderstood. I am not really a fan of merge(KStream stream, KStream... streams); because it seem to break the `Collection` pattern. If I have a collection of KStreams

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-29 Thread Nick Telford
Yeah, the Named parameter makes it a little trickier. My suggestion would be to add an additional overload that looks like: KStream merged(KStream first, Named named, KStream rest); It's not ideal having the Named parameter split the other parameters; we could alternatively move the Named paramet

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-28 Thread Chris Egerton
Hi all, Java permits the overload. Simple test class to demonstrate: ``` public class Test { private final String field; public Test(String field) { this.field = field; } public Test merge(Test that) { return new Test("Single-arg merge: " + this.field + ", " + th

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-28 Thread Matthias J. Sax
I think Java does not allow to have both overloads, because it would result in ambiguity? If you call `s1.merge(s2)` it's unclear which method you want to call. -Matthias On 3/28/22 7:20 AM, Nick Telford wrote: Hi Matthias, How about instead of changing the signature of the existing method

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-28 Thread Nick Telford
Hi Matthias, How about instead of changing the signature of the existing method to variadic, we simply add a new overload which takes variadic args: KStream merge(KStream first, KStream... rest); That way, we maintain both source *and* binary compatibility for the existing method, and we can enf

Re: [DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-03-24 Thread Matthias J. Sax
Thanks for proposing this KIP. I feel a little bit torn by the idea. In general, we try to keep the surface area small, and only add APIs that delivery (significant) value. It seems the current proposal is more or less about syntactic sugar, what can still be valuable, but I am not really sur

[DISCUSS] KIP-819: Merge multiple KStreams in one operation

2022-01-31 Thread Nick Telford
Hi everyone, I'd like to discuss KIP 819: https://cwiki.apache.org/confluence/display/KAFKA/KIP-819%3A+Merge+multiple+KStreams+in+one+operation This is a simple KIP that adds/modifies the KStream#merge API to enable many streams to be merged in a single graph node. Regards, Nick Telford