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

2023-07-31 Thread Shay Lin
Hi all, It's been a few days, if there is no further comments or questions I'd like to call for a vote. There is an existing VOTE thread if you search for KIP-759. Thank you, Shay On Wed, Jul 26, 2023 at 7:30 PM Shay Lin wrote: > Very good catch, Matthias. I updated the KIP to state that the n

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

2023-07-26 Thread Shay Lin
Very good catch, Matthias. I updated the KIP to state that the new DSLOperation will return a new, mutated KStream. Thank you, Shay On Wed, Jul 26, 2023 at 6:13 PM Matthias J. Sax wrote: > One last question. What should happen for the following case: > > KStream myStream = build.stream(...).map

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

2023-07-26 Thread Matthias J. Sax
One last question. What should happen for the following case: KStream myStream = build.stream(...).map(...); myStream.markAsPartiitoned().groupByKey().aggregate(...); myStream.join(...) The question is about the "fan-out" pattern. `myStream`, which is marked for partitioning, is fed into two do

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

2023-07-26 Thread Shay Lin
Hi John, Thanks for your reply. I updated the KIP to reflect the changes we discussed in the thread today. #1 is duly noted, I learned from the examples Sophie sent earlier! =) In the new version, I also talked about why IQ and joins will not work with the interface and talked about the mitigatio

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

2023-07-26 Thread John Roesler
Hello Shay, Thanks for the KIP! I just took a look in preparation to vote, and there are two small-ish things that I'd like to fix first. Apologies if this stuff has already come up in the discussion thread; I only skimmed it. 1. The KIP only mentions the name of the method instead of providin

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

2023-07-24 Thread Sophie Blee-Goldman
Thanks Shay! You and Matthias have convinced me, I'm happy with the current proposal. I think once you make the minor updates to the KIP document this will be ready for voting again. Cheers, Sophie On Mon, Jul 24, 2023 at 8:26 AM Shay Lin wrote: > Hi Sophie and Matthias, thanks for your comment

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

2023-07-24 Thread Shay Lin
Hi Sophie and Matthias, thanks for your comments and replies. 1. Scope of change: KStreams only or KStreams/KTable I took some time to digest your points, looking through how KStreams triggers repartitions today. I noticed that `repartitionRequired`is a flag in KStreamImpl etc and not in KTableImp

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

2023-07-21 Thread Matthias J. Sax
I agree that it could easily be misused. There is a few Jira tickets for cases when people want to "cancel" a repartition step. I would hope those tickets are linked to the KIP (if not, we should do this, and maybe even c&p those cases as motivation into the KIP itself)? It's always a tricky q

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

2023-07-21 Thread Sophie Blee-Goldman
I guess I felt a bit uneasy about how this could be used/abused while reading the KIP, but if we truly believe this is an advanced feature, I'm fine with the way things currently are. It doesn't feel like the best API, but it does seem to be the best *possible* API given the way things are. W.r.t

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

2023-07-21 Thread Matthias J. Sax
Some thought about the API question. A. kstream.groupBy(...).aggregate(...) This can be re-writtten as kstream.selectKey(...) .markAsRepartitioned() .groupByKey() .aggregate() Given that `markAsRepartitoned` is an advanced feature, I think it would be ok? B. ktable

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

2023-07-15 Thread Sophie Blee-Goldman
Hey Shay, while I don't have any specific concerns about the new public API in this KIP, I'd like to better understand how this feature will work before I vote. We should document the behavior of this new operator clearly in the KIP as well -- you don't necessarily need to write the complete javado

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

2023-07-10 Thread Shay Lin
Hi all, It's been a few days so I went ahead with editing the KIP, the main change is on the method name https://cwiki.apache.org/confluence/display/KAFKA/KIP-759%3A+Unneeded+repartition+canceling. I will follow up with a VOTE separately. Best, Shay On Thu, Jun 29, 2023 at 4:52 PM Matthias J. Sa

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

2023-06-29 Thread Matthias J. Sax
Shay, thanks for picking up this KIP. It's a pity that the discussion stalled for such a long time. As expressed previously, I am happy with the name `markAsPartitioned()` and also believe it's ok to just document the impact and leave it to the user to do the right thing. If we really get

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

2023-06-28 Thread Shay Lin
Hi all, Great discussion thread. May I take this KIP up? If it’s alright my plan is to update the KIP with the operator `markAsPartitioned()`. As you have discussed and pointed out, there are implications to downstream joins or aggregation operations. Still, the operator is intended for advanced

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

2022-05-22 Thread Levani Kokhreidze
Hi all, Since there was no activity around this KIP, I’ll pick it up in coming weeks and continue the discussion. Best, Levani > On 27. Apr 2022, at 22:50, Matthias J. Sax wrote: > > Let's wait a couple of days to give Ivan a chance to reply. If he does not > reply, feel free to pick it up.

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

2022-04-27 Thread Matthias J. Sax
Let's wait a couple of days to give Ivan a chance to reply. If he does not reply, feel free to pick it up. -Matthias On 4/26/22 3:58 AM, Levani Kokhreidze wrote: Hi, Sorry, maybe I am jumping the gun here, but if by any chance this KIP becomes dormant, I'd be interested in picking it up. L

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

2022-04-26 Thread Levani Kokhreidze
Hi, Sorry, maybe I am jumping the gun here, but if by any chance this KIP becomes dormant, I'd be interested in picking it up. Levani > On 23. Apr 2022, at 02:43, Matthias J. Sax wrote: > > Ivan, > > are you still interested in this KIP? I think it would be a good addition. > > > -Matthias

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

2022-04-22 Thread Matthias J. Sax
Ivan, are you still interested in this KIP? I think it would be a good addition. -Matthias On 8/16/21 5:30 PM, Matthias J. Sax wrote: Your point about the IQ problem is an interesting one. I missed the point that the "new key" would be a "superkey", and thus, it should always be possible to c

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

2021-08-16 Thread Matthias J. Sax
Your point about the IQ problem is an interesting one. I missed the point that the "new key" would be a "superkey", and thus, it should always be possible to compute the original key from the superkey. (As a matter of fact, for windowed-table the windowed-key is also a superkey...) I am not sure i

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

2021-08-09 Thread Ivan Ponomarev
Hi Matthias and Sophie! ==1. markAsPartitioned vs extending `selectKey(..)` etc. with a config.== I don't have a strong opinion here, both Sophie's and Matthias' points look convincing for me. I think we should estimate the following: what is the probability that we will ever need to extend

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

2021-08-08 Thread Matthias J. Sax
Hi, I originally had a similar thought about `markAsPartitioned()` vs extending `selectKey()` et al. with a config. While I agree that it might be conceptually cleaner to use a config object, I did not propose it as the API impact (deprecating stuff and adding new stuff) is quite big... If we thin

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

2021-08-07 Thread Sophie Blee-Goldman
Before I dive in to the question of IQ and the approaches you proposed, can you just elaborate on the problem itself? By definition, the `markAsPartitioned` flag means that a repartition would be a no-op, ie that the stream (and its partitioning) would be the same whether or not a repartition is in

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

2021-08-07 Thread Ivan Ponomarev
Hi Sophie, thanks for your reply! So your proposal is: 1). For each key-changing operation, deprecate the existing overloads that accept a Named, and replace them with overloads that take an operator-specific config object. 2). Add `markAsPartitioned` flag to these configs. IMO, this looks much

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

2021-08-06 Thread Sophie Blee-Goldman
Hey Ivan I completely agree that adding it as a config to Grouped/Joined/etc isn't much better, I was just listing it for completeness, and that I would prefer to make it a configuration of the key-changing operation itself -- that's what I meant by a better alternative might be to introduce this

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

2021-08-06 Thread Ivan Ponomarev
Hi Matthias, Concerning the naming: I like `markAsPartitioned`, because it describes what this operation is actually doing! Hi Sophie, I see the concern about poor code cohesion. We declare key mapping in one place of code, then later in another place we say "markAsPartitioned()". When we c

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

2021-08-03 Thread Sophie Blee-Goldman
Do we really need a whole DSL operator for this? I think the original name for this operator -- `cancelRepartition()` -- is itself a sign that this is not an operation on the stream itself but rather a command/request to whichever operator would have otherwise triggered this repartition. What abou

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

2021-08-03 Thread Matthias J. Sax
Thanks for the KIP Ivan! I think it's a good feature to give advanced users more control, and allow them to build more efficient application. Not sure if I like the proposed named though (the good old "naming things" discussion :)) Did you consider alternatives? What about - markAsPartitioned(

[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 trivia