Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-08-09 Thread Ivan Ponomarev
Could it be that the idea to apply a DISTINCT-aggregation is for a different use-case than to remove duplicate messages from a KStream? -Matthias On 8/6/21 4:12 AM, Ivan Ponomarev wrote:    - Why restrict de-duplication for the key only? Should we not also consider the value (or make it someh

Re: [DISCUSS] KIP-759: Unneeded repartition canceling

2021-08-09 Thread Ivan Ponomarev
ey result to assert that they do indeed match. Imo this is probably overkill, just putting it out there. On Sat, Aug 7, 2021 at 1:42 PM Ivan Ponomarev wrote: Hi Sophie, thanks for your reply! So your proposal is: 1). For each key-changing operation, deprecate the existing overloads that acce

Re: [DISCUSS] KIP-759: Unneeded repartition canceling

2021-08-07 Thread Ivan Ponomarev
that same config object. WDYT? By the way, the above idea (introducing a single config object to wrap all operator properties) was also raised by John Roesler a while back. Let's hope he hasn't changed his mind since then :) On Fri, Aug 6, 2021 at 3:01 AM Ivan Ponomarev wrote: Hi Matthi

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-08-06 Thread Ivan Ponomarev
t to keep the first record base on offset order. Wondering why? While I agree that deduplication for overlapping window is questionable, I am still wondering if you plan to disallow it (by adding a runtime check and throwing an exception), or not? On 8/1/21 6:42 AM, Ivan Ponomarev wrote: Hi Bruno,

Re: [DISCUSS] KIP-759: Unneeded repartition canceling

2021-08-06 Thread Ivan Ponomarev
sider alternatives? What about - markAsPartitioned() - markAsKeyed() - skipRepartition() Not sure if there are other idea on a good name? -Matthias On 6/24/21 7:45 AM, Ivan Ponomarev wrote: Hello, I'd like to start a discussion for KIP-759: https://cwiki.apache.org/confluence/dis

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-08-01 Thread Ivan Ponomarev
lized values seems to be way to do it. 2. Thanks for the confirmation 3. I continue to be satisfied to let you all hash it out. Thanks, -John On Tue, 2021-07-20 at 11:42 +0300, Ivan Ponomarev wrote: Hi all, 1. Actually I always thought about the serialized byte array only -- at least this is wha

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-07-26 Thread Ivan Ponomarev
u all hash it out. Thanks, -John On Tue, 2021-07-20 at 11:42 +0300, Ivan Ponomarev wrote: Hi all, 1. Actually I always thought about the serialized byte array only -- at least this is what local stores depend upon, and what Kafka itself depends upon when doing log compaction. I can imagine a case

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-07-20 Thread Ivan Ponomarev
, 2021, at 10:18, Ivan Ponomarev wrote: Hello everyone, I would like to remind you about KIP-655 and KIP-759 just in case they got lost in your inbox. Now the initial proposal is split into two independent and smaller ones, so it must be easier to review them. Of course, if you have time. Regards

[VOTE] KIP-655: Windowed "Distinct" Operation for Kafka Streams

2021-07-12 Thread Ivan Ponomarev
Hello all! I'd like to start the vote for KIP-655 which proposes the zero-arg distict() method to be added to TimeWindowedKStream and SessionWindowedKStream. https://cwiki.apache.org/confluence/display/KAFKA/KIP-655%3A+Windowed+Distinct+Operation+for+Kafka+Streams+API Regards, Ivan

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-07-11 Thread Ivan Ponomarev
you are now only proposing the zero-arg distict() method to be added to TimeWindowedKStream and SessionWindowedKStream, right? I’m in favor of this proposal. Thanks, John On Sat, Jul 10, 2021, at 10:18, Ivan Ponomarev wrote: Hello everyone, I would like to remind you about KIP-655 and KIP-759 j

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-07-10 Thread Ivan Ponomarev
We can avoid adding distinct() to the sliding window interface, but hopping windows are just a different parameterization of epoch-aligned windows. It seems we can’t do much about that except document the issue. Thanks, John On Wed, May 26, 2021, at 10:14, Ivan Ponomarev wrote: Hi John! I thin

[DISCUSS] KIP-759: Unneeded repartition canceling

2021-06-24 Thread Ivan Ponomarev
Hello, I'd like to start a discussion for KIP-759: https://cwiki.apache.org/confluence/display/KAFKA/KIP-759%3A+Unneeded+repartition+canceling This is an offshoot of the discussion of KIP-655 for a `distinct` operator, which turned out to be a separate proposal. The proposal is quite

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-06-24 Thread Ivan Ponomarev
void adding distinct() to the sliding window interface, but hopping windows are just a different parameterization of epoch-aligned windows. It seems we can’t do much about that except document the issue. Thanks, John On Wed, May 26, 2021, at 10:14, Ivan Ponomarev wrote: Hi John! I think that your propos

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-05-26 Thread Ivan Ponomarev
istinct()`. All the rest of the KStream operations would also benefit. What do you think? Thanks, John On Sun, May 23, 2021, at 08:09, Ivan Ponomarev wrote: Hello everyone, let me revive the discussion for KIP-655. Now I have some time again and I'm eager to finalize this. Based on what was

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2021-05-23 Thread Ivan Ponomarev
riate for the Streams API. WDYT? On Mon, Sep 14, 2020 at 10:04 AM Ivan Ponomarev wrote: Hi Matthias, Thanks for your review! It made me think deeper, and indeed I understood that I was missing some important details. To simplify, let me explain my particular use case first so I can refer to it

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

2021-01-23 Thread Ivan Ponomarev
instead I have updated the KIP. Does anyone want to object these changes? Regards, Ivan 06.07.2020 21:43, Matthias J. Sax пишет: I am late, but I am also +1 (binding). -Matthias On 7/6/20 2:16 AM, Ivan Ponomarev wrote: Wow! So excited to hear that! Thanks for your collaboration, now

[jira] [Resolved] (KAFKA-12230) Some Kafka TopologyTestDriver-based unit tests can't be run on Windows file system

2021-01-21 Thread Ivan Ponomarev (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12230?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ivan Ponomarev resolved KAFKA-12230. Resolution: Duplicate Duplicate of KAFKA-12190 > Some Kafka TopologyTestDriver-ba

[jira] [Created] (KAFKA-12230) Some Kafka TopologyTestDriver-based unit tests can't be run on Windows file system

2021-01-21 Thread Ivan Ponomarev (Jira)
Ivan Ponomarev created KAFKA-12230: -- Summary: Some Kafka TopologyTestDriver-based unit tests can't be run on Windows file system Key: KAFKA-12230 URL: https://issues.apache.org/jira/browse/KAFKA-12230

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2020-09-14 Thread Ivan Ponomarev
l. (5) Even if it might be an implementation detail (and maybe the KIP itself does not need to mention it), can you give a high level overview how you intent to implement it (that would be easier to grog, compared to reading the PR). -Matthias On 8/23/20 4:29 PM, Ivan Ponomarev wrote: Sorry

Re: [DISCUSS] KIP-655: Windowed "Distinct" Operation for KStream

2020-08-23 Thread Ivan Ponomarev
Sorry, I forgot to add [DISCUSS] tag to the topic 24.08.2020 2:27, Ivan Ponomarev пишет: Hello, I'd like to start a discussion for KIP-655. KIP-655: https://cwiki.apache.org/confluence/display/KAFKA/KIP-655%3A+Windowed+Distinct+Operation+for+Kafka+Streams+API I also opened a proof

KIP-655: Windowed "Distinct" Operation for KStream

2020-08-23 Thread Ivan Ponomarev
/9210 Regards, Ivan Ponomarev

[jira] [Created] (KAFKA-10369) Introduce Distinct operation in KStream

2020-08-06 Thread Ivan Ponomarev (Jira)
Ivan Ponomarev created KAFKA-10369: -- Summary: Introduce Distinct operation in KStream Key: KAFKA-10369 URL: https://issues.apache.org/jira/browse/KAFKA-10369 Project: Kafka Issue Type

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

2020-07-06 Thread Ivan Ponomarev
.jo...@gmail.com> wrote: This will be a great addition. Thanks Ivan! +1 (non-binding) On Fri, Jun 26, 2020 at 7:07 PM John Roesler wrote: Thanks, Ivan! I’m +1 (binding) -John On Thu, May 28, 2020, at 17:24, Ivan Ponomarev wrote: Hello all! I'd like to start the vote for KIP-418 which pr

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

2020-05-28 Thread Ivan Ponomarev
Hello all! I'd like to start the vote for KIP-418 which proposes deprecation of current `branch` method and provides a method-chaining based API for branching. https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream Regards, Ivan

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

2020-05-22 Thread Ivan Ponomarev
ast, 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, and of course a K is not the same as "? extends K&quo

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

2020-05-21 Thread Ivan Ponomarev
d 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 usual

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

2020-05-15 Thread Ivan Ponomarev
aybe `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 eve

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

2020-05-03 Thread Ivan Ponomarev
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"

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

2020-04-22 Thread Ivan Ponomarev
able 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 ever

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

2020-04-17 Thread Ivan Ponomarev
y 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 th

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

2019-05-02 Thread Ivan Ponomarev
-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 ge

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

2019-04-13 Thread Ivan Ponomarev
ll 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 wro

Re: [DISCUSS] KIP-401 TransformerSupplier/ProcessorSupplier enhancements

2019-03-30 Thread Ivan Ponomarev
Hi all! I was about to write another KIP, but found out that KIP-401 addresses exactly the problem I faced. So let me jump into your discussion and ask you to assess another idea. I fully agree with the KIP-401's motivation part. E. g in my project I had to invent a wrapper class that hides

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

2019-03-29 Thread Ivan Ponomarev
, 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 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

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

2019-03-28 Thread Ivan Ponomarev
ecating the existing branch(...) is not on the table. Thanks, Paul On Wed, Mar 27, 2019 at 12:08 PM Ivan Ponomarev 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

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

2019-03-27 Thread Ivan Ponomarev
on 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

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

2019-03-25 Thread Ivan Ponomarev
operations on it as you desire. Thoughts? Best, Paul On Sun, Mar 24, 2019 at 2:02 PM Ivan Ponomarev 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 m

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

2019-03-24 Thread Ivan Ponomarev
off of KBranchedStreams if desired. Thanks, Paul On Sat, Mar 23, 2019 at 4:28 PM Ivan Ponomarev wrote: 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

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

2019-03-23 Thread Ivan Ponomarev
+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: [ANNOUNCE] New Committer: Bill Bejeck

2019-02-14 Thread Ivan Ponomarev
Congratulations, Bill! Your 'Kafka Streams in Action' is a great book. These months it is always travelling with me in my backpack with my laptop )) Regards, Ivan 14.02.2019 3:56, Guozhang Wang пишет: Hello all, The PMC of Apache Kafka is happy to announce that we've added Bill Bejeck as

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

2019-01-18 Thread Ivan Ponomarev
/browse/KAFKA-5488 PR#6164: https://github.com/apache/kafka/pull/6164 Regards, Ivan Ponomarev