Re: [DISCUSS] KIP-1027 Add MockFixedKeyProcessorContext

2024-05-20 Thread Matthias J. Sax
would extend `FixedKeyRecord`? -- Or maybe an even better solution? I could not think of something else so far. Thoughts? On 5/3/24 9:46 AM, Matthias J. Sax wrote: Please also update the KIP. To get a wiki account created, please request it via a commet on this ticket: https

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-05-16 Thread Matthias J. Sax
of a more obvious metric name. Nit: the KIP says it's a store-level metric, but I think it would be good to say explicitly that it's recorded with DEBUG level only? Yes, I've already updated the KIP with this information in the table. Regards, Nick On Sun, 31 Mar 2024 at 10:53, Matthias J. S

Re: [VOTE] KIP-989: RocksDB Iterator Metrics

2024-05-14 Thread Matthias J. Sax
+1 (binding) On 5/14/24 9:19 AM, Lucas Brutschy wrote: Hi Nick! Thanks for the KIP. +1 (binding) On Tue, May 14, 2024 at 5:16 PM Nick Telford wrote: Hi everyone, I'd like to call a vote on the Kafka Streams KIP-989: RocksDB Iterator Metrics:

Re: [VOTE] KIP-1033: Add Kafka Streams exception handler for exceptions occurring during processing

2024-05-13 Thread Matthias J. Sax
+1 (binding) On 5/13/24 5:54 PM, Sophie Blee-Goldman wrote: Thanks for the KIP guys! +1 (binding) On Mon, May 13, 2024 at 6:02 AM Bill Bejeck wrote: Thanks for the KIP, this will be a great addition! +1(binding) -Bill On Fri, May 3, 2024 at 4:48 AM Bruno Cadonna wrote: Hi Damien,

Re: Creating kafka wiki id

2024-05-09 Thread Matthias J. Sax
Self-service to create an account is currently not working. Please reply on https://issues.apache.org/jira/browse/INFRA-25451 to request a wiki account. I'll update the wiki page for now until the issue is resolved. -Matthias On 5/7/24 8:25 AM, 黃竣陽 wrote: Hello, I want to create a KIP, but

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-05-09 Thread Matthias J. Sax
ProductionExceptionHandler... Just explaining my reasoning -- in the end I leave it up to the KIP authors and anyone who will actually be using this feature in their applications :) On Tue, May 7, 2024 at 8:35 PM Matthias J. Sax wrote: @Loic, yes, what you describe is exactly what I had i

Re: [DISCUSS] KIP-1038: Add Custom Error Handler to Producer

2024-05-07 Thread Matthias J. Sax
rRecord as the input parameter of the handle() method as well. - increased the response types to 3 to have fail and two types of continue. - The default behaviour is having no custom handler, having the corresponding config parameter set to null. Therefore, the KIP provides no default

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-05-07 Thread Matthias J. Sax
@Loic, yes, what you describe is exactly what I had in mind. @Sophie, can you elaborate a little bit? First of all, I agree that it makes sense to maintain the two separate callbacks for the ProductionExceptionHandler, since one of them is specifically for serialization exceptions while the

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-05-03 Thread Matthias J. Sax
On 5/3/24 6:29 PM, Matthias J. Sax wrote: 101: Yes, but what I am saying is, that we don't need to flush the .position file to disk periodically, but only maintain it in main memory, and only write it to disk on close() to preserve it across restarts. This way, it would never be ahead

Re: [DISCUSS] KIP-1027 Add MockFixedKeyProcessorContext

2024-05-03 Thread Matthias J. Sax
, May 1, 2024 at 11:00 PM Matthias J. Sax wrote: Any updates on this KIP? On 3/28/24 4:11 AM, Matthias J. Sax wrote: It seems that `MockRecordMetadata` is a private class, and thus not part of the public API. If there are any changes required, we don't need to discuss on the KIP

Re: [DISCUSS] KIP-924: customizable task assignment for Streams

2024-05-03 Thread Matthias J. Sax
:43 AM, Matthias J. Sax wrote: Thanks Sophie. My bad. You are of course right about `TaskAssignment` and the StreamsPartitionAssignor's responsibitliy to map tasks of a instance to consumers. When I wrote my reply, I forgot about this detail. Seems you did not add `UNKNOWN_TASK_ID` error yet as pr

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-05-03 Thread Matthias J. Sax
eams/state/internals/RocksDBStore.java#L397 On 5/1/24 3:00 AM, Matthias J. Sax wrote: Thanks Bruno. 101: I think I understand this better now. But just want to make sure I do. What do you mean by "they can diverge" and "Recovering after a failure might load inconsistent offse

Re: [VOTE] KIP-1036: Extend RecordDeserializationException exception

2024-05-03 Thread Matthias J. Sax
+1 (binding) On 5/3/24 8:52 AM, Federico Valeri wrote: Hi Fred, this is a useful addition. +1 non binding Thanks On Fri, May 3, 2024 at 4:11 PM Andrew Schofield wrote: Hi Fred, Thanks for the KIP. It’s turned out nice and elegant I think. Definitely a worthwhile improvement. +1

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-05-03 Thread Matthias J. Sax
What about (106) to unify both exiting callback methods of `ProductionExceptionHandler` into a single one, instead of adding two new ones? Damien's last reply about it was: I will think about unifying, I do agree it would be cleaner. There was not follow up on this question, and the KIP

Re: [VOTE] KIP-924: customizable task assignment for Streams

2024-05-02 Thread Matthias J. Sax
I left one more nit on the discuss thread. But overall LGTM. +1 (binding) Thanks Rohan and Sophie for driving this KIP. -Matthias On 4/29/24 2:07 PM, Sophie Blee-Goldman wrote: +1 (binding) thanks for driving this KIP! On Tue, Apr 16, 2024 at 1:46 PM Rohan Desai wrote:

Re: [DISCUSS] KIP-924: customizable task assignment for Streams

2024-05-02 Thread Matthias J. Sax
have a lower level indention? It caught me for a sec until I realized there are just two categories. * In TaskAssignmentUtils , why not let those util functions return `TaskAssignment` (to me it feels more consistent with the user APIs), but instead return a Map? Guozhang On Tue, Apr 30, 2024

Re: [DISCUSS] KIP-1027 Add MockFixedKeyProcessorContext

2024-05-01 Thread Matthias J. Sax
Any updates on this KIP? On 3/28/24 4:11 AM, Matthias J. Sax wrote: It seems that `MockRecordMetadata` is a private class, and thus not part of the public API. If there are any changes required, we don't need to discuss on the KIP. For `CapturedPunctuator` and `CapturedForward` it's

[jira] [Resolved] (KAFKA-16644) FK join emits duplicate tombstone on left-side delete

2024-04-30 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16644. - Resolution: Duplicate > FK join emits duplicate tombstone on left-side del

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-04-30 Thread Matthias J. Sax
Thanks for the update. I am wondering if we should use `ReadOnlyHeaders` instead of `ImmutableHeaders` as interface name? Also, the returned `Header` interface is technically not immutable either, because `Header#key()` returns a mutable byte-array... Would we need a `ReadOnlyHeader`

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-30 Thread Matthias J. Sax
that we should try to support downgrades without wipes. At least Nick should state in the KIP why we do not support it. Best, Bruno On 4/23/24 8:13 AM, Matthias J. Sax wrote: Thanks for splitting out this KIP. The discussion shows, that it is a complex beast by itself, so worth to discuss by

Re: [DISCUSS] KIP-924: customizable task assignment for Streams

2024-04-30 Thread Matthias J. Sax
I like the idea of error codes. Not sure if the name are ideal? UNKNOWN_PROCESS_ID makes sense, but the other two seems a little bit difficult to understand? Should we be very descriptive (and also try to avoid coupling it to the threading model -- important for the first error code): -

[jira] [Created] (KAFKA-16644) FK join emit duplicate tombstone on left-side delete

2024-04-29 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16644: --- Summary: FK join emit duplicate tombstone on left-side delete Key: KAFKA-16644 URL: https://issues.apache.org/jira/browse/KAFKA-16644 Project: Kafka

Wiki account sign-up

2024-04-29 Thread Matthias J. Sax
Hi, as many of you know, there is a issue with creating new wiki accounts right now: https://issues.apache.org/jira/browse/INFRA-25451 A fix might take some more time. In the meantime, please reply to the INFRA ticket with your email address, and accounts will be created manually in the

Re: Assistance Needed with Creating Wiki ID for Kafka Contribution

2024-04-29 Thread Matthias J. Sax
It's a known issue and INFRA is working on a solution: https://issues.apache.org/jira/browse/INFRA-25451 In the mean time, users can be added manually (cf the ticket, and reply there to get added). -Matthias On 3/28/24 5:10 AM, Prashant Jagtap wrote: Hi, I hope this email finds you well.

Re: Confluence edit access

2024-04-26 Thread Matthias J. Sax
Thanks. You should be all set. -Matthias On 4/25/24 10:49 PM, Claude Warren wrote: My Confluence ID is "claude" On Thu, Apr 25, 2024 at 8:40 PM Matthias J. Sax wrote: What's your wiki ID? We can grant write access on our side if you have already an account. -Matthias On 4/25/

Re: [DISCUSS] KIP-924: customizable task assignment for Streams

2024-04-25 Thread Matthias J. Sax
believe that we shouldn't call certain kinds of assignments "invalid" (such as a task not being assigned to anyone). For the clearly-invalid assignment cases, I'd err on the side of not holding users hands too much for now, but again: would be happy to defer if anyone has another suggestion and/or s

Re: Confluence edit access

2024-04-25 Thread Matthias J. Sax
What's your wiki ID? We can grant write access on our side if you have already an account. -Matthias On 4/25/24 4:06 AM, Claude Warren wrote: I would like to get edit access to the Kafka confluence so that I can work on KIP-936. Can someone here do that or do I need to go through Infra?

Re: [DISCUSS] KIP-924: customizable task assignment for Streams

2024-04-24 Thread Matthias J. Sax
aying -- but I'm now realizing that I completely misinterpreted what your concern was. Sorry for the long-winded and ultimately irrelevant answer. I'm completely fine with having the return type be a simple Set with additional info such as TaskId in the AssignedTask class (and I see Rohan already made this

Re: [ANNOUNCE] New committer: Igor Soarez

2024-04-24 Thread Matthias J. Sax
Congrats! On 4/24/24 2:29 PM, Bill Bejeck wrote: Congrats Igor! -Bill On Wed, Apr 24, 2024 at 2:37 PM Tom Bentley wrote: Congratulations Igor! On Thu, 25 Apr 2024 at 6:27 AM, Chia-Ping Tsai wrote: Congratulations, Igor! you are one of the best Kafka developers!!! Mickael Maison 於

Re: [DISCUSS] KIP-1038: Add Custom Error Handler to Producer

2024-04-23 Thread Matthias J. Sax
et to null. Therefore, the KIP provides no default implementation of the interface. - We follow the interface solution as described in the Rejected Alternetives section. Cheers, Alieh On Thu, Apr 18, 2024 at 8:11 PM Matthias J. Sax wrote: Thanks for the KIP Alieh! It addresses an important case

Re: [DISCUSS] KIP-1035: StateStore managed changelog offsets

2024-04-23 Thread Matthias J. Sax
Thanks for splitting out this KIP. The discussion shows, that it is a complex beast by itself, so worth to discuss by its own. Couple of question / comment: 100 `StateStore#commit()`: The JavaDoc says "must not be called by users" -- I would propose to put a guard in place for this, by

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-04-22 Thread Matthias J. Sax
gaps in a separate KIP. Best, Bruno On 4/11/24 12:06 AM, Matthias J. Sax wrote: Thanks for the KIP. Great discussion. I am not sure if I understand the proposal from Bruno to hand in the processor node id? Isn't this internal (could not even find it quickly). We do have a processor name, r

Re: [DISCUSS] KIP-924: customizable task assignment for Streams

2024-04-19 Thread Matthias J. Sax
, Matthias J. Sax wrote: Great KIP. I have some minor comments/questions: 100 The KIP says: "In the future, additional plugins can use the same partition.assignor  prefix". What does this mean? 101 (nit) The KIP says: "Note that the thread-level assignment will remain an

Re: [DISCUSS] KIP-924: customizable task assignment for Streams

2024-04-19 Thread Matthias J. Sax
Great KIP. I have some minor comments/questions: 100 The KIP says: "In the future, additional plugins can use the same partition.assignor prefix". What does this mean? 101 (nit) The KIP says: "Note that the thread-level assignment will remain an un-configurable internal implementation

[jira] [Resolved] (KAFKA-16486) Integrate metric measurability changes in metrics collector

2024-04-19 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16486?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16486. - Fix Version/s: 3.8.0 Resolution: Done > Integrate metric measurability chan

[jira] [Created] (KAFKA-16586) Test TaskAssignorConvergenceTest failing

2024-04-18 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16586: --- Summary: Test TaskAssignorConvergenceTest failing Key: KAFKA-16586 URL: https://issues.apache.org/jira/browse/KAFKA-16586 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-16280) Expose method to determine Metric Measurability

2024-04-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16280. - Resolution: Done > Expose method to determine Metric Measurabil

Re: [DISCUSS] KIP-1038: Add Custom Error Handler to Producer

2024-04-18 Thread Matthias J. Sax
Thanks for the KIP Alieh! It addresses an important case for error handling. I agree that using this handler would be an expert API, as mentioned by a few people. But I don't think it would be a reason to not add it. It's always a tricky tradeoff what to expose to users and to avoid foot guns,

Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread Matthias J. Sax
Andrew, thanks for the details about Consumer internals. That's super useful for this discussion! -- And it confirms my understanding. I don't think we want to use ConsumerRecord type thought, because for a DLQ the handler wants to write the message into some DLQ topic, and thus needs the key

Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread Matthias J. Sax
Hi, I am actually not sure if using `Record` is really the right thing? While `Record` is technically public API, it does not seem to be intended to be exposed to end users? But I guess my main question is really about what metadata we really want to add to `RecordDeserializationException`?

Re: [ANNOUNCE] New Kafka PMC Member: Greg Harris

2024-04-18 Thread Matthias J. Sax
Congrats Greg! On 4/15/24 10:44 AM, Hector Geraldino (BLOOMBERG/ 919 3RD A) wrote: Congrats! Well deserved From: dev@kafka.apache.org At: 04/13/24 14:42:22 UTC-4:00To: dev@kafka.apache.org Subject: [ANNOUNCE] New Kafka PMC Member: Greg Harris Hi all, Greg Harris has been a Kafka committer

[jira] [Created] (KAFKA-16575) Automatically remove KTable aggregation result when group becomes empty

2024-04-17 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16575: --- Summary: Automatically remove KTable aggregation result when group becomes empty Key: KAFKA-16575 URL: https://issues.apache.org/jira/browse/KAFKA-16575

Re: [DISCUSS] KIP-1033: Add Kafka Streams exception handler for exceptions occuring during processing

2024-04-10 Thread Matthias J. Sax
Thanks for the KIP. Great discussion. I am not sure if I understand the proposal from Bruno to hand in the processor node id? Isn't this internal (could not even find it quickly). We do have a processor name, right? Or do I mix up something? Another question is about `ProcessingContext` --

Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-10 Thread Matthias J. Sax
Thanks for the KIP Fred. Couple of nits: it's not clear from the "Public API" section what is new and what is existing API w/o going back to the code. For existing methods which are not changed, it's also best to actually omit them. -- It would also be best to only put the interface itself

[jira] [Created] (KAFKA-16508) Infinte loop if output topic does not exisit

2024-04-10 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16508: --- Summary: Infinte loop if output topic does not exisit Key: KAFKA-16508 URL: https://issues.apache.org/jira/browse/KAFKA-16508 Project: Kafka Issue

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-04-02 Thread Matthias J. Sax
specifies the names of source processors. So I am fine with not having an overload with a Named parameter. Best, Bruno On 3/31/24 11:30 AM, Matthias J. Sax wrote: Two more follow up thoughts: (1) I am still not a big fan of the boolean parameter we introduce. Did you consider to use

Re: [VOTE] KIP-1020 Move `window.size.ms` and `windowed.inner.serde.class` from `StreamsConfig` to TimeWindowedDe/Serializer class

2024-04-02 Thread Matthias J. Sax
+1 (binding) -Matthias On 4/1/24 7:44 PM, Lucia Cerchie wrote: Hello everyone, I'd like to call a vote on KIP-1020 . It has been under discussion since Feb 15, and has received edits to the KIP and approval by

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-03-31 Thread Matthias J. Sax
tation yet -- it just strikes me as easiest to do at the top level of the store hierarchy rather than at the bottom. My gut instinct may very well be wrong, but that's what it's saying On Thu, Mar 7, 2024 at 10:43 AM Matthias J. Sax wrote: Seems I am late to this party. Can we pick this up again

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-31 Thread Matthias J. Sax
to open a vote and hopefully we can get it voted on soon! best, Walker On Thu, Mar 28, 2024 at 5:55 AM Matthias J. Sax wrote: Hey, looking into the API, I am wondering why we would need to add an overload talking a `Named` parameter? StreamsBuilder.addGlobalStore() (and .addGlobalTable

Re: [DISCUSS] KIP-1027 Add MockFixedKeyProcessorContext

2024-03-28 Thread Matthias J. Sax
you think! Regards, Shashwat Pandey On Mon, Mar 11, 2024 at 10:09 PM Matthias J. Sax wrote: Thanks for the KIP Shashwat. Closing this testing gap is great! It did come up a few time already... One question: why do you propose to `extend MockProcessorContext`? Given how the actual runti

Re: [DISCUSS] KIP-1020: Deprecate `window.size.ms` in `StreamsConfig`

2024-03-28 Thread Matthias J. Sax
90982804 On Sun, Mar 17, 2024 at 7:09 PM Sophie Blee-Goldman < sop...@responsive.dev wrote: Sounds good! @Lucia when you have a moment can you update the KIP with the new proposal, including the details that Matthias pointed out in his last response? After that's done I think you can go ahead

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-28 Thread Matthias J. Sax
Hey, looking into the API, I am wondering why we would need to add an overload talking a `Named` parameter? StreamsBuilder.addGlobalStore() (and .addGlobalTable()) already takes a `Consumed` parameter that allows to set a name. 2. I do not understand what you mean with "maximum

Re: [ANNOUNCE] New committer: Christo Lolov

2024-03-27 Thread Matthias J. Sax
Congrats! On 3/26/24 9:39 PM, Christo Lolov wrote: Thank you everyone! It wouldn't have been possible without quite a lot of reviews and extremely helpful inputs from you and the rest of the community! I am looking forward to working more closely with you going forward :) On Tue, 26 Mar 2024

Re: [DISCUSS] KIP-1020: Deprecate `window.size.ms` in `StreamsConfig`

2024-03-16 Thread Matthias J. Sax
t to go with these configs? On Thu, Mar 14, 2024 at 4:58 PM Matthias J. Sax wrote: By "don't add them" do you just mean we would not have any actual variables defined anywhere for these configs (eg WINDOW_SIZE_MS) and simply document -- somewhere -- that one can use the string "window.s

Re: [DISCUSS] KIP-1020: Deprecate `window.size.ms` in `StreamsConfig`

2024-03-14 Thread Matthias J. Sax
configs defined in classes you don't even need for the console client, like the Serde form Just my two cents -- happy to hear other opinions on this On Mon, Mar 11, 2024 at 6:58 PM Matthias J. Sax wrote: Yes, it's used inside `TimeWindowedSerializer` and actually also inside `TimeWindowDeseria

Re: [DISCUSS] Personal branches under apache/kafka

2024-03-13 Thread Matthias J. Sax
+1 Should be fine to just delete all of them (as long as nobody raised objections). Not sure if we could enable some protection GitHub side that disallow to push into non-existing branches and thus avoids accidental branch creation? -Matthias On 3/13/24 11:39 AM, Josep Prat wrote: Hi

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-13 Thread Matthias J. Sax
ate before loading it, defaulting to identity. This would provide the ability to do efficient, non-deserializing transformations like => On Thu, Mar 7, 2024 at 7:19 PM Matthias J. Sax wrote: @Bruno: (1), I think you are spot for the ts-extractor: on the restore code path, we only support rec

Re: [DISCUSS] KIP-990: Capability to SUSPEND Tasks on DeserializationException

2024-03-13 Thread Matthias J. Sax
l like these concerns are all essentially "true", but to me they just seem like implementation or design decisions and none of them strike them as posing an unsolvable problem for this feature. But maybe I'm just lacking in imagination... Thoughts? On Fri, Mar 8, 2024 at 5:30 PM Matt

[jira] [Resolved] (KAFKA-16357) Kafka Client JAR manifest breaks javac linting

2024-03-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16357?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16357. - Resolution: Duplicate > Kafka Client JAR manifest breaks javac lint

Re: [DISCUSS] KIP-1027 Add MockFixedKeyProcessorContext

2024-03-11 Thread Matthias J. Sax
Thanks for the KIP Shashwat. Closing this testing gap is great! It did come up a few time already... One question: why do you propose to `extend MockProcessorContext`? Given how the actual runtime context classes are setup, it seems that the regular context and fixed-key-context are distinct,

Re: [DISCUSS] KIP-1020: Deprecate `window.size.ms` in `StreamsConfig`

2024-03-11 Thread Matthias J. Sax
nally stick them into the serde class, rather than the serializer and/or deserializer, but I could be convinced either way) On Fri, Mar 1, 2024 at 3:00 PM Matthias J. Sax wrote: One more thought after I did some more digging on the related PR. Should we do the same thing for `windowed.inner.serd

[jira] [Resolved] (KAFKA-16360) Release plan of 3.x kafka releases.

2024-03-11 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-16360?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-16360. - Resolution: Invalid Please don't use Jira to ask questions. Jira tickets are for bug

[jira] [Created] (KAFKA-16366) Refactor KTable source optimization

2024-03-11 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16366: --- Summary: Refactor KTable source optimization Key: KAFKA-16366 URL: https://issues.apache.org/jira/browse/KAFKA-16366 Project: Kafka Issue Type

Re: [DISCUSS] KIP-955: Add stream-table join on foreign key

2024-03-08 Thread Matthias J. Sax
es are aggregated per stream event and not per table update. Disclaimer: I did not test both solutions side by side for performance. For now I am just using design observations for performance/scalability projections. Any additions to pros/cons? Any other solution alternatives? Regards, Ig

Re: [DISCUSS] KIP-990: Capability to SUSPEND Tasks on DeserializationException

2024-03-08 Thread Matthias J. Sax
Hey Nick, I am sorry that I have to say that I am not a fan of this KIP. I see way too many food-guns and complications that can be introduced. I am also not sure if I understand the motivation. You say, CONTINUE and FAIL is not good enough, but don't describe in detail why? If we

[jira] [Resolved] (KAFKA-15576) Add 3.6.0 to broker/client and streams upgrade/compatibility tests

2024-03-07 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-15576. - Resolution: Fixed > Add 3.6.0 to broker/client and streams upgrade/compatibility te

Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-03-07 Thread Matthias J. Sax
Seems I am late to this party. Can we pick this up again aiming for 3.8 release? I think it would be a great addition. Few comments: - I think it does make sense to report `iterator-duration-avg` and `iterator-duration-max` for all *closed* iterators -- it just seems to be a useful metric

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-07 Thread Matthias J. Sax
g like: GlobalStoreParameters.globalStore().withKeySerde(keySerde).disableReprocessOnRestore() Just an idea, open for discussion. 3. Could you please go over the KIP and correct typos and other mistakes in the KIP? Best, Bruno On 3/2/24 1:43 AM, Matthias J. Sax wrote: Thanks for the KIP Walker.

[jira] [Created] (KAFKA-16350) StateUpdated does not init transaction after canceling task close action

2024-03-06 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16350: --- Summary: StateUpdated does not init transaction after canceling task close action Key: KAFKA-16350 URL: https://issues.apache.org/jira/browse/KAFKA-16350

[jira] [Resolved] (KAFKA-15417) JoinWindow does not seem to work properly with a KStream - KStream - LeftJoin()

2024-03-05 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15417?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-15417. - Fix Version/s: 3.8.0 Resolution: Fixed > JoinWindow does not seem to w

[jira] [Resolved] (KAFKA-14747) FK join should record discarded subscription responses

2024-03-04 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-14747. - Fix Version/s: 3.8.0 Resolution: Fixed > FK join should record discar

[jira] [Resolved] (KAFKA-10603) Re-design KStream.process() and K*.transform*() operations

2024-03-04 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10603?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-10603. - Resolution: Fixed > Re-design KStream.process() and K*.transform*() operati

[jira] [Created] (KAFKA-16339) Remove Deprecated "transformer" methods and classes

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16339: --- Summary: Remove Deprecated "transformer" methods and classes Key: KAFKA-16339 URL: https://issues.apache.org/jira/browse/KAFKA-16339 Proj

[jira] [Created] (KAFKA-16338) Removed Deprecated configs from StreamsConfig

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16338: --- Summary: Removed Deprecated configs from StreamsConfig Key: KAFKA-16338 URL: https://issues.apache.org/jira/browse/KAFKA-16338 Project: Kafka Issue

[jira] [Created] (KAFKA-16337) Remove Deprecates APIs of Kafka Streams in 5.0

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16337: --- Summary: Remove Deprecates APIs of Kafka Streams in 5.0 Key: KAFKA-16337 URL: https://issues.apache.org/jira/browse/KAFKA-16337 Project: Kafka Issue

[jira] [Created] (KAFKA-16336) Remove Deprecated metric standby-process-ratio

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16336: --- Summary: Remove Deprecated metric standby-process-ratio Key: KAFKA-16336 URL: https://issues.apache.org/jira/browse/KAFKA-16336 Project: Kafka Issue

[jira] [Created] (KAFKA-16335) Remove Deprecated method on StreamPartitioner

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16335: --- Summary: Remove Deprecated method on StreamPartitioner Key: KAFKA-16335 URL: https://issues.apache.org/jira/browse/KAFKA-16335 Project: Kafka Issue

[jira] [Created] (KAFKA-16334) Remove Deprecated command line option from reset tool

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16334: --- Summary: Remove Deprecated command line option from reset tool Key: KAFKA-16334 URL: https://issues.apache.org/jira/browse/KAFKA-16334 Project: Kafka

[jira] [Created] (KAFKA-16333) Removed Deprecated methods KTable#join

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16333: --- Summary: Removed Deprecated methods KTable#join Key: KAFKA-16333 URL: https://issues.apache.org/jira/browse/KAFKA-16333 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-16332) Remove Deprecated builder methods for Time/Session/Join/SlidingWindows

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16332: --- Summary: Remove Deprecated builder methods for Time/Session/Join/SlidingWindows Key: KAFKA-16332 URL: https://issues.apache.org/jira/browse/KAFKA-16332 Project

[jira] [Created] (KAFKA-16331) Remove Deprecated EOSv1

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16331: --- Summary: Remove Deprecated EOSv1 Key: KAFKA-16331 URL: https://issues.apache.org/jira/browse/KAFKA-16331 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-16330) Remove Deprecated methods/variables from TaskId

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16330: --- Summary: Remove Deprecated methods/variables from TaskId Key: KAFKA-16330 URL: https://issues.apache.org/jira/browse/KAFKA-16330 Project: Kafka Issue

[jira] [Created] (KAFKA-16329) Remove Deprecated Task/ThreadMetadata classes and related methods

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16329: --- Summary: Remove Deprecated Task/ThreadMetadata classes and related methods Key: KAFKA-16329 URL: https://issues.apache.org/jira/browse/KAFKA-16329 Project

[jira] [Resolved] (KAFKA-12831) Remove Deprecated method StateStore#init

2024-03-04 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-12831. - Resolution: Fixed > Remove Deprecated method StateStore#i

[jira] [Resolved] (KAFKA-12825) Remove Deprecated method StreamsBuilder#addGlobalStore

2024-03-04 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12825?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-12825. - Resolution: Fixed > Remove Deprecated method StreamsBuilder#addGlobalSt

[jira] [Created] (KAFKA-16328) Remove deprecated config StreamsConfig#retries

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16328: --- Summary: Remove deprecated config StreamsConfig#retries Key: KAFKA-16328 URL: https://issues.apache.org/jira/browse/KAFKA-16328 Project: Kafka Issue

[jira] [Created] (KAFKA-16327) Remove Deprecated variable StreamsConfig#TOPOLOGY_OPTIMIZATION

2024-03-04 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16327: --- Summary: Remove Deprecated variable StreamsConfig#TOPOLOGY_OPTIMIZATION Key: KAFKA-16327 URL: https://issues.apache.org/jira/browse/KAFKA-16327 Project: Kafka

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-01 Thread Matthias J. Sax
Thanks for the KIP Walker. Fixing this issue, and providing users some flexibility to opt-in/out on "restore reprocessing" is certainly a good improvement. From an API design POV, I like the idea to not require passing in a ProcessorSupplier to begin with. Given the current implementation of

Re: [DISCUSS] KIP-1020: Deprecate `window.size.ms` in `StreamsConfig`

2024-03-01 Thread Matthias J. Sax
? -Matthias On 2/21/24 2:41 PM, Matthias J. Sax wrote: Thanks for the KIP. Sounds like a nice cleanup. window.size.ms  is not a true KafkaStreams config, and results in an error when set from a KStreams application What error? Given that the configs is used by `TimeWindowedDeserializer` I am

Re: Join request

2024-02-24 Thread Matthias J. Sax
To subscribe, please follow instructions from the webpage https://kafka.apache.org/contact -Matthias On 2/23/24 1:15 AM, kashi mori wrote: Hi, please add my email to the mailin list

[jira] [Resolved] (KAFKA-12549) Allow state stores to opt-in transactional support

2024-02-22 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12549?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-12549. - Resolution: Duplicate Closing this ticket in favor of K14412. > Allow state sto

Re: [DISCUSS] KIP-844: Transactional State Stores

2024-02-22 Thread Matthias J. Sax
To close the loop on this thread. KIP-892 was accepted and is currently implemented. Thus I'll go a head and mark this KIP a discarded. Thanks a lot Alex for spending so much time on this very important feature! Without your ground work, we would not have KIP-892 and your contributions are

Re: [DISCUSS] KIP-1020: Deprecate `window.size.ms` in `StreamsConfig`

2024-02-21 Thread Matthias J. Sax
Thanks for the KIP. Sounds like a nice cleanup. window.size.ms is not a true KafkaStreams config, and results in an error when set from a KStreams application What error? Given that the configs is used by `TimeWindowedDeserializer` I am wondering if we should additionally add public

[jira] [Created] (KAFKA-16295) Align RocksDB and in-memory store inti() sequence

2024-02-21 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16295: --- Summary: Align RocksDB and in-memory store inti() sequence Key: KAFKA-16295 URL: https://issues.apache.org/jira/browse/KAFKA-16295 Project: Kafka

Re: [VOTE] KIP-1019: Expose method to determine Metric Measurability

2024-02-20 Thread Matthias J. Sax
+1 (binding) On 2/20/24 2:55 AM, Manikumar wrote: +1 (binding). Thanks for the KIP. Manikumar On Tue, Feb 20, 2024 at 2:31 PM Andrew Schofield < andrew_schofield_j...@outlook.com> wrote: Hi Apoorv, Thanks for the KIP. +1 (non-binding) Thanks, Andrew On 19 Feb 2024, at 22:31, Apoorv

[jira] [Created] (KAFKA-16279) Avoid leaking abstractions of `StateStore`

2024-02-19 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16279: --- Summary: Avoid leaking abstractions of `StateStore` Key: KAFKA-16279 URL: https://issues.apache.org/jira/browse/KAFKA-16279 Project: Kafka Issue Type

Re: [Discuss] KIP-1019: Expose method to determine Metric Measurability

2024-02-16 Thread Matthias J. Sax
Thanks for the KIP. Seems there is not much we need to discuss about it. Feel free to start a VOTE. -Matthias On 2/15/24 7:24 AM, Manikumar wrote: LGTM, Thanks for the KIP. On Thu, Feb 15, 2024 at 8:50 PM Doğuşcan Namal wrote: LGTM thanks for the KIP. +1(non-binding) On Wed, 14 Feb 2024

[jira] [Created] (KAFKA-16263) Add Kafka Streams docs about available listeners/callback

2024-02-15 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16263: --- Summary: Add Kafka Streams docs about available listeners/callback Key: KAFKA-16263 URL: https://issues.apache.org/jira/browse/KAFKA-16263 Project: Kafka

[jira] [Created] (KAFKA-16262) Add IQv2 to Kafka Streams documentation

2024-02-15 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-16262: --- Summary: Add IQv2 to Kafka Streams documentation Key: KAFKA-16262 URL: https://issues.apache.org/jira/browse/KAFKA-16262 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-14957) Default value for state.dir is confusing

2024-02-10 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14957?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-14957. - Fix Version/s: 3.8.0 Resolution: Fixed > Default value for state.

  1   2   3   4   5   6   7   8   9   10   >