Re: [DISCUSS] KIP-1124: Providing a clear Kafka Client upgrade path

2025-01-15 Thread Sophie Blee-Goldman
Thanks for the KIP. I'm all for documenting the upgrade path like this. Just want to chime in on the Streams side of things, since there is (at least) one nuance which is probably worth accounting for: Because of the introduction of cooperative rebalancing in 2.4 which was enabled by default for K

[jira] [Resolved] (KAFKA-14419) Failed SyncGroup leading to partitions lost due to processing during rebalances

2025-01-13 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-14419. Assignee: A. Sophie Blee-Goldman Resolution: Duplicate > Fai

Re: [DISCUSS] KIP-1092: Extend Consumer#close with an option to leave the group or not

2025-01-08 Thread Sophie Blee-Goldman
tl;dr: 1) I tend to agree we should keep the existing behavior, but what this means is actually different and more complicated than just "if leaves group, then invokes leave callback" 2) Personally I think we should actually *always* invoke this callback, for every case Longer version: First, to

[jira] [Resolved] (KAFKA-18326) Cached stores may return deleted values

2025-01-07 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18326?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-18326. Fix Version/s: 4.0.0 3.9.1 3.8.2

[jira] [Resolved] (KAFKA-18026) Allow custom processor wrapping

2024-12-17 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18026?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-18026. Resolution: Fixed > Allow custom processor wrapp

Re: [VOTE] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-11 Thread Sophie Blee-Goldman
+1 (binding) thanks Sebastien! -Sophie On Tue, Dec 10, 2024 at 9:26 AM Bill Bejeck wrote: > Hi Sébastien, > > Thanks for the KIP! > > +1(binding) > > Regards, > Bill > > On Fri, Dec 6, 2024 at 4:33 AM Bruno Cadonna wrote: > > > Hi Sébastien, > > > > +1 (binding) > > > > Many Thanks! > > Bruno

Re: [DISCUSS] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-12-11 Thread Sophie Blee-Goldman
t;>> > >>>>> > >>>>> > >>>>> > >>>>> On Tue, Nov 26, 2024 at 9:37 PM Sebastien Viale > >>>>> wrote: > >>>>>> > >>>>>> Thanks Bruno for your comments: > >>>>>> > >>>>>

[jira] [Resolved] (KAFKA-13713) Tech Debt: keep StreamThread and TopologyMetadata's view of the topology in sync

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13713?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13713. Resolution: Won't Fix > Tech Debt: keep StreamTh

[jira] [Resolved] (KAFKA-13282) Draft final NamedTopology API and publish a KIP

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13282. Resolution: Won't Fix > Draft final NamedTopology API and publi

[jira] [Resolved] (KAFKA-13645) Support the TopologyTestDriver with modular topologies

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13645. Resolution: Won't Fix > Support the TopologyTestDriver with

[jira] [Resolved] (KAFKA-13644) Support global state stores with modular topologies

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13644. Resolution: Won't Fix > Support global state stores with

[jira] [Resolved] (KAFKA-13283) Migrate experimental feature to public API

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13283. Resolution: Won't Fix > Migrate experimental feature to pu

[jira] [Resolved] (KAFKA-13643) Replace "NamedTopology" with "ModularTopology" in the codebase

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13643?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13643. Resolution: Won't Fix > Replace "NamedTopology" with &q

[jira] [Resolved] (KAFKA-13712) Make topology addition/removal atomic so we can roll back if request fails

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13712?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13712. Resolution: Won't Fix > Make topology addition/removal atomic s

[jira] [Resolved] (KAFKA-12648) Experiment with resilient isomorphic topologies

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12648?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-12648. Resolution: Won't Fix Deprecated named topologies in 4.0 wit

[jira] [Resolved] (KAFKA-13711) Fix bugs with input topic management to support pattern subscription fully

2024-12-11 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13711?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-13711. Resolution: Won't Fix > Fix bugs with input topic management to

[jira] [Created] (KAFKA-18196) Reuse window store for stream-stream self-join

2024-12-09 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18196: -- Summary: Reuse window store for stream-stream self-join Key: KAFKA-18196 URL: https://issues.apache.org/jira/browse/KAFKA-18196 Project: Kafka

[jira] [Created] (KAFKA-18195) Enter "incompatible" instead of leaving incompatible entires blank in Kafka Streams broker compatibility matrix

2024-12-09 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18195: -- Summary: Enter "incompatible" instead of leaving incompatible entires blank in Kafka Streams broker compatibility matrix Key: KAFKA-18195

Re: [VOTE]: KIP-1071: Streams Rebalance Protocol

2024-12-09 Thread Sophie Blee-Goldman
+1 (binding) Thank you guys for the deep discussion and for carefully considering all the details of this new protocol. Looking forward to seeing the KIP in action! -Sophie On Fri, Dec 6, 2024 at 5:52 AM Andrew Schofield < andrew_schofield_j...@outlook.com> wrote: > Hi Bruno and Lucas, > Thanks

[jira] [Resolved] (KAFKA-18067) Kafka Streams can leak Producer client under EOS

2024-12-09 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18067?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-18067. Resolution: Fixed > Kafka Streams can leak Producer client under

[jira] [Created] (KAFKA-18191) StreamJoined name is not used for processor names

2024-12-09 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18191: -- Summary: StreamJoined name is not used for processor names Key: KAFKA-18191 URL: https://issues.apache.org/jira/browse/KAFKA-18191 Project: Kafka

[jira] [Created] (KAFKA-18190) TopologyTestDriver time is not synchronized with test topics

2024-12-09 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18190: -- Summary: TopologyTestDriver time is not synchronized with test topics Key: KAFKA-18190 URL: https://issues.apache.org/jira/browse/KAFKA-18190

[jira] [Created] (KAFKA-18067) Kafka Streams can leak Producer client under EOS

2024-11-22 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18067: -- Summary: Kafka Streams can leak Producer client under EOS Key: KAFKA-18067 URL: https://issues.apache.org/jira/browse/KAFKA-18067 Project: Kafka

[jira] [Created] (KAFKA-18066) Misleading/mismatched StreamThread id in logging

2024-11-21 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18066: -- Summary: Misleading/mismatched StreamThread id in logging Key: KAFKA-18066 URL: https://issues.apache.org/jira/browse/KAFKA-18066 Project: Kafka

Re: [DISCUSS] KIP-1111: Enforcing Explicit Naming for Kafka Streams Internal Topics

2024-11-21 Thread Sophie Blee-Goldman
First off, thanks for the KIP! I think this is a great idea as it's super easy to miss naming one thing and end up with a topology that isn't upgradeable. A1. I actually had the same reaction as Almog to the name, I feel it's slightly clearer as a positive instead of a negative, though I think the

Re: [VOTE] KIP-1112: allow custom processor wrapping

2024-11-21 Thread Sophie Blee-Goldman
Lucas Brutschy > > wrote: > >> > >> Thanks! > >> > >> +1 (binding) > >> > >> Lucas > >> > >> Bill Bejeck schrieb am Di., 19. Nov. 2024, 00:06: > >> > >>> Thanks for the KIP, Sophie, seems like a us

Re: [DISCUSS] KIP-1112: allow custom processor wrapping

2024-11-20 Thread Sophie Blee-Goldman
and its two subtasks describe the proposed improvements. One covers what we can do without the KIP to help things ASAP, the other covers the API cleanup that will require a KIP On Wed, Nov 20, 2024 at 5:07 PM Sophie Blee-Goldman wrote: > Bruno -- I updated the javadocs to remove the wrapping requ

[jira] [Created] (KAFKA-18054) Automatically detect missed configs needed by a topology

2024-11-20 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18054: -- Summary: Automatically detect missed configs needed by a topology Key: KAFKA-18054 URL: https://issues.apache.org/jira/browse/KAFKA-18054 Project

[jira] [Created] (KAFKA-18055) Remove TopologyConfig and clean up Topology/StreamsBuilder config APIs

2024-11-20 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18055: -- Summary: Remove TopologyConfig and clean up Topology/StreamsBuilder config APIs Key: KAFKA-18055 URL: https://issues.apache.org/jira/browse/KAFKA-18055

[jira] [Created] (KAFKA-18053) Clean up TopologyConfig and API for supplying configs needed by the topology

2024-11-20 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18053: -- Summary: Clean up TopologyConfig and API for supplying configs needed by the topology Key: KAFKA-18053 URL: https://issues.apache.org/jira/browse/KAFKA-18053

Re: [DISCUSS] KIP-1112: allow custom processor wrapping

2024-11-20 Thread Sophie Blee-Goldman
ote: > > > Hi Sophie, > > > > Thanks for the details! > > > > Yeah, I think replacing the contract in the javadocs with a warning is > > better. > > > > Best, > > Bruno > > > > On 20.11.24 10:05, Sophie Blee-Goldman wrote: >

[jira] [Resolved] (KAFKA-18044) Test ticket for INFRA

2024-11-20 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-18044?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-18044. Resolution: Not A Problem > Test ticket for IN

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-11-20 Thread Sophie Blee-Goldman
whether the topology_epoch & topology match the > broker-side. All following heartbeats don't include the topology > anymore, but we do not need to "reverify" that the topology epoch & > topology are consistent, it is enough to compare epochs. > > Cheers, &g

Re: [DISCUSS] KIP-1112: allow custom processor wrapping

2024-11-20 Thread Sophie Blee-Goldman
> > > I'm a bit confused since the motivation promises a cleanup of > > topologyconfig/streams config, but this doesn't seem to be part of this > > kip. But on the whole, this change looks fine for me. > > > > Thanks for the KIP! > > > > Cheers, &

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-11-19 Thread Sophie Blee-Goldman
gt; > idea to specify it in the configuration. > > > > Cheers, > > Nick > > > > On Sat, 16 Nov 2024 at 17:59, Nick Telford > wrote: > > > > > Hey everyone, > > > > > > With respect to Bruno's proposal, could instances cache thei

Re: [DISCUSS] KIP-1112: allow custom processor wrapping

2024-11-19 Thread Sophie Blee-Goldman
7;m a bit confused since the motivation promises a cleanup of > topologyconfig/streams config, but this doesn't seem to be part of this > kip. But on the whole, this change looks fine for me. > > Thanks for the KIP! > > Cheers, > Lucas, > > Sophie Blee-Goldm

[jira] [Reopened] (KAFKA-10409) Refactor Kafka Streams RocksDb iterators

2024-11-19 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10409?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman reopened KAFKA-10409: > Refactor Kafka Streams RocksDb iterat

[jira] [Resolved] (KAFKA-10409) Refactor Kafka Streams RocksDb iterators

2024-11-19 Thread A. Sophie Blee-Goldman (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10409?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] A. Sophie Blee-Goldman resolved KAFKA-10409. Resolution: Incomplete > Refactor Kafka Streams RocksDb iterat

Re: [DISCUSS] KIP-1112: allow custom processor wrapping

2024-11-18 Thread Sophie Blee-Goldman
of configs. I have made this change in the KIP so please take a look and let me know if you have any concerns. Happy to discuss alternatives On Mon, Nov 18, 2024 at 3:27 PM Sophie Blee-Goldman wrote: > Thanks Almog! That makes sense to me, I've updated the KIP so that the > ProcessorW

Re: [DISCUSS] KIP-1112: allow custom processor wrapping

2024-11-18 Thread Sophie Blee-Goldman
s like application.id (e.g. for emitting custom metrics per > processor). > > - Almog > > On Fri, Nov 15, 2024 at 10:16 PM Sophie Blee-Goldman < > sop...@responsive.dev> > wrote: > > > Hey all, > > > > We have a short KIP we'd like to propose which w

[VOTE] KIP-1112: allow custom processor wrapping

2024-11-18 Thread Sophie Blee-Goldman
Since there have been no concerns about this proposal I'm going to go ahead and call for a vote on KIP-1112: allow custom processor wrapping. https://cwiki.apache.org/confluence/display/KAFKA/KIP-1112%3A+allow+custom+processor+wrapping Please direct any new feedback (if any) to the [DISCUSS] thre

[DISCUSS] KIP-1112: allow custom processor wrapping

2024-11-15 Thread Sophie Blee-Goldman
Hey all, We have a short KIP we'd like to propose which will allow injecting custom code modules around the processors of Kafka Streams applications, including DSL-built topologies. Please let us know if you have any thoughts or concerns https://cwiki.apache.org/confluence/display/KAFKA/KIP-111

[jira] [Created] (KAFKA-18026) Allow custom processor wrapping

2024-11-15 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-18026: -- Summary: Allow custom processor wrapping Key: KAFKA-18026 URL: https://issues.apache.org/jira/browse/KAFKA-18026 Project: Kafka Issue

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-11-15 Thread Sophie Blee-Goldman
rom a classic group. A consequence of the merge > >> is that the heartbeat now needs permissions to create and describe > >> topics which before only the initialize call had. But we think that is > >> not a big deal. A consumer of a Streams client will send the me

Re: [DISCUSS] Require KIPs to include "How to teach this section"

2024-11-06 Thread Sophie Blee-Goldman
It seems like the "Create KIP" button on the main KIP page actually does not point to this template? I was actually surprised to hear that we had a "Documentation Plan" section already because I have literally never seen it. I took a quick look at this button but as far as I can tell it's pointing

Re: [VOTE] KIP-1091: Improved Kafka Streams operator metrics

2024-11-06 Thread Sophie Blee-Goldman
+1 (binding) thanks Bill On Wed, Nov 6, 2024 at 4:44 PM Matthias J. Sax wrote: > +1 (binding) > > On 11/6/24 3:39 AM, Apoorv Mittal wrote: > > Hi Bill, > > Thanks for the KIP. > > > > +1 (non-binding) > > > > Regards, > > Apoorv Mittal > > > > > > On Wed, Nov 6, 2024 at 10:55 AM Lucas Brutschy

Re: [DISCUSS] KIP-1091: Improved Kafka Streams operator metrics

2024-10-28 Thread Sophie Blee-Goldman
Hey Bill, Thanks for the KIP! That all makes sense to me, just one minor note: while you mentioned the TRACE recording level in the Motivation section, it seems to be missing from the table in the Public Interfaces section. I assume this will also be included, presumably with a value of 2? Cheers

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-10-21 Thread Sophie Blee-Goldman
rceptor` object back. The > > >> fundamental problem is, that we cannot know if the object we get is an > > >> actually wrapper/intercpetor for the passed-in client, or if the > object > > >> is not wrapper at all, but a `new MyKafkaConsumer extends >

[jira] [Created] (KAFKA-17805) Deprecate named topologies

2024-10-15 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-17805: -- Summary: Deprecate named topologies Key: KAFKA-17805 URL: https://issues.apache.org/jira/browse/KAFKA-17805 Project: Kafka Issue Type

Re: [VOTE] KIP-1092: Extend Consumer#close with an option to leave the group or not

2024-10-10 Thread Sophie Blee-Goldman
+1 (binding) Thanks for this KIP! On Mon, Oct 7, 2024 at 9:22 AM Kirk True wrote: > Hi TengYao, > > +1 (non-binding) > > Thanks for all the work so far on this. > > Kirk > > On Mon, Oct 7, 2024, at 4:09 AM, TengYao Chi wrote: > > Hi Andrew, > > > > Thanks for reviewing and participating in the

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-10-04 Thread Sophie Blee-Goldman
If this KIP is > > going to immediately deprecate the old KafkaClientSupplier, then we need > to > > support implementing an interceptor while configured to use the current > > rebalancing protocol. > > I don't think we need to do this, but we could make it mutu

Re: [DISCUSS] KIP-1094 Add a new constructor method with nextOffsets to ConsumerRecords

2024-09-30 Thread Sophie Blee-Goldman
gt; andrew_schofield_j...@outlook.com> wrote: > > > @sophie I meant AK tests. But you make a fair point. There will not be > > that many instances I'm sure, so if the consensus is to deprecate the > > old constructor, that's fine with me. > > > >

Re: [DISCUSS] KIP-1092: Extend Consumer#close with an option to leave the group or not

2024-09-30 Thread Sophie Blee-Goldman
efault > > >>>>> value, and this default should align with the existing > > >> `Consumer#close()` > > >>>>> method, which internally calls the overloaded > > >> `Consumer#close(Duration)` > > >>>>> with a default

Re: [DISCUSS] KIP-1094 Add a new constructor method with nextOffsets to ConsumerRecords

2024-09-28 Thread Sophie Blee-Goldman
September 2024 09:09 > > To: dev@kafka.apache.org > > Subject: Re: [DISCUSS] KIP-1094 Add a new constructor method with > nextOffsets to ConsumerRecords > > > > Thank you, Bill and Sophie. > > > > @Bill: You are right. I updated the KIP. > > @Sophie

Re: [DISCUSS] KIP-1092: Extend Consumer#close with an option to leave the group or not

2024-09-28 Thread Sophie Blee-Goldman
s not necessary for the plain consumer case. > > >> > > > >> > However, for the KS case it's different. Because KS uses the > internal > > >> > config to disable sending leave group request for dynamic members, > we > > >> > lack a

Re: [DISCUSS] KIP-1092: Extend Consumer#close with an option to leave the group or not

2024-09-26 Thread Sophie Blee-Goldman
Thanks for the KIP! Quick request for readability, can you please include the exact APIs that you're proposing to add or change under the "Public Interfaces" section? The KIP should display the actual method signature and any applicable javadocs for new public APIs. You can look at other KIPs for

Re: [DISCUSS] KIP-1094 Add a new constructor method with nextOffsets to ConsumerRecords

2024-09-26 Thread Sophie Blee-Goldman
Should we deprecate the old constructor to make sure that all info gets passed in when creating a ConsumerRecords instance? On Thu, Sep 26, 2024 at 3:37 PM Bill Bejeck wrote: > Hi Alieh, > > Thanks for the KIP, it will be very useful to Kafka Streams. > I have one comment. In the "Proposed Chan

Re: [VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-09-25 Thread Sophie Blee-Goldman
+1 (binding) thanks for the KIP guys! On Mon, Sep 23, 2024 at 3:38 AM Sebastien Viale < sebastien.vi...@michelin.com> wrote: > Hi everyone, > > Just a quick reminder that the vote for KIP-1034 is still open. > Thank you all for your participation! > > Best regards, > Damien Sebastien and Loic >

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-09-10 Thread Sophie Blee-Goldman
all this, and still make a lot of deep > changes to Kafka Streams (as mechanisms like probing rebalances etc. > don't make sense in KIP-848). I think adapting KIP-848 to work with > Kafka Streams either way is a big effort. Piggybacking on consumer > group RPCs and extending the consume

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-09-09 Thread Sophie Blee-Goldman
our > example > > here <https://lists.apache.org/thread/l6dhq1rfl3xkq8g9wfqsvw89yjrgzbn8> > > confused me since it contracts with the above sentence. > > > > Thanks, > > Alieh > > > > On Fri, Sep 6, 2024 at 5:08 AM Sophie Blee-Goldman < > sop...@res

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-09-05 Thread Sophie Blee-Goldman
/KIP-848%3A+The+Next+Generation+of+the+Consumer+Rebalance+Protocol#KIP848:TheNextGenerationoftheConsumerRebalanceProtocol-AssignmentEpoch-Computethegroupassignment > > > > Its definition does not change at all, so we didn't describe it in > > detail in KIP-1071. > > > &g

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-09-05 Thread Sophie Blee-Goldman
Kafka Streams, but this is > true right now, too, so I don't think anything really changes. > > I guess, in the end, the new interface allows you to do everything you > did before, but we still change the API contract a little bit, as Kafka > Streams provides a client insta

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-09-03 Thread Sophie Blee-Goldman
Ah, my bad -- I thought I refreshed the page to get the latest version which is why I was a bit confused when I couldn't find anything about the new tools which I had previously seen in the KIP. Sorry for the confusion and unnecessary questions S1. > You could imagine calling the initialize RPC >

Re: [VOTE] KIP-1065: Add "retry" return-option to ProductionExceptionHandler

2024-08-28 Thread Sophie Blee-Goldman
Also : +1 (binding) On Wed, Aug 28, 2024 at 3:48 PM Sophie Blee-Goldman wrote: > Thanks guys. I filed https://issues.apache.org/jira/browse/KAFKA-17441 to > consider a followup KIP for adding RETRY to the other exception handlers. I > agree we can move on and vote this through > &

Re: [ANNOUNCE] New committer: Lianet Magrans

2024-08-28 Thread Sophie Blee-Goldman
Congrats Lianet! Well deserved On Wed, Aug 28, 2024 at 2:27 PM Jiashen Zhang wrote: > Congratulations Lianet! Awesome! > > On Wed, Aug 28, 2024 at 2:10 PM Andrew Schofield < > andrew_schofi...@live.com> > wrote: > > > Awesome news. Well done, Lianet. > > > > Andrew > > > > > On 28 Aug 2024, at 1

Re: [VOTE] KIP-1065: Add "retry" return-option to ProductionExceptionHandler

2024-08-28 Thread Sophie Blee-Goldman
Thanks guys. I filed https://issues.apache.org/jira/browse/KAFKA-17441 to consider a followup KIP for adding RETRY to the other exception handlers. I agree we can move on and vote this through On Thu, Aug 22, 2024 at 11:28 AM Matthias J. Sax wrote: > Hi, > > given that there was no further feedb

[jira] [Created] (KAFKA-17441) Add RETRY option to other exception handlers

2024-08-28 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-17441: -- Summary: Add RETRY option to other exception handlers Key: KAFKA-17441 URL: https://issues.apache.org/jira/browse/KAFKA-17441 Project: Kafka

Re: [DISCUSS] KIP-1071: Streams Rebalance Protocol

2024-08-28 Thread Sophie Blee-Goldman
Hey guys -- sorry I'm late to the party, I'm still going over some things and don't have everything I want to say ready just yet, but I figured that shouldn't stop me from starting with the questions/comments that are ready to go. So here's my first set of feedback: S1. Can you clarify which clien

Re: [VOTE] KIP-1070: Deprecate MockProcessorContext

2024-07-15 Thread Sophie Blee-Goldman
t; deprecated, it seems we should deprecate all of them, too? > > > > -Matthias > > > On 7/12/24 1:06 AM, Lucas Brutschy wrote: > > Sounds good to me! > > > > +1 (binding) > > > > On Fri, Jul 12, 2024 at 12:55 AM Bill Bejeck wrote: > >

Re: [VOTE] KIP-1070: Deprecate MockProcessorContext

2024-07-11 Thread Sophie Blee-Goldman
Makes sense to me, +1 (binding) On Thu, Jul 11, 2024 at 9:24 AM Matthias J. Sax wrote: > Hi, > > I want to propose a very small KIP. Skipping the DISCUSS step, and > calling for a VOTE directly. > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1070%3A+deprecate+MockProcessorContext >

Re: [DISCUSS] KIP-1065: Add "retry" return-option to ProductionExceptionHandler

2024-07-01 Thread Sophie Blee-Goldman
Thanks for the KIP -- definitely agree with this proposal, just have a few suggestions: 1. In the KIP, you mention We might also consider to not calling the handler when writing into > internal topics, as those must exist. Personally I would vote to consider all topics the same in this regard,

Re: [VOTE] KIP-1056 - Deprecate `default.` prefix for exception handler in StreamsConfig

2024-07-01 Thread Sophie Blee-Goldman
After reaching an agreement on the discussion thread, I'm giving this a +1 (binding) under the condition that we ship this change in 4.0 so as to minimize the impact on users. Thanks for the KIP! -Sophie On Thu, Jun 27, 2024 at 2:43 PM Sophie Blee-Goldman wrote: > I'll just not

Re: [DISCUSS] KIP-1056: Remove `default.` prefix for exception handler StreamsConfig

2024-07-01 Thread Sophie Blee-Goldman
I guess > users need to change a couple of things in their code when they switch > to 4.0, so those deprecations will carry (almost) no weight. > > What do you think? > > Best, > Bruno > > > On 6/15/24 8:58 AM, Sophie Blee-Goldman wrote: > > I think we should pause

Re: [VOTE] KIP-1056 - Deprecate `default.` prefix for exception handler in StreamsConfig

2024-06-27 Thread Sophie Blee-Goldman
I'll just note that I will personally be abstaining from this vote, but won't vote -1 and just want to defer to the rest of the community on this. I've stated my concerns in the discussion thread and will leave it at that -- if we hear from users who actively support this change and want it to happ

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

2024-06-19 Thread Sophie Blee-Goldman
+1 (binding) thanks Nick! On Thu, Jun 13, 2024 at 12:39 AM Bruno Cadonna wrote: > Thanks Nick! > > Great KIP! > > +1 (binding) > > Best, > Bruno > > On 6/13/24 2:31 AM, Matthias J. Sax wrote: > > Thanks Nick. > > > > +1 (binding) > > > > > > Looking forward to get this all merged! > > > > > > -M

Re: [DISCUSS] KIP-1056: Remove `default.` prefix for exception handler StreamsConfig

2024-06-14 Thread Sophie Blee-Goldman
I think we should pause for a minute and have an honest conversation about whether the benefits of making this change outweigh the negatives. Here's my quick roundup of the positives and negatives, feel free to add to this list if there's else you think should be considered but then let's evaluate

Re: [DISCUSS] Apache Kafka 3.9.0 release

2024-06-14 Thread Sophie Blee-Goldman
+1, thank you Colin Given the July freeze deadlines, I take it we are going with the "short 3.9.0 release" option and that the existence of this release will impact the 4.0.0 deadlines which will still follow the usual schedule -- in other words, this is an "additional release" outside of the regu

Re: [DISCUSS] Apache Kafka 3.8.0 release

2024-06-13 Thread Sophie Blee-Goldman
Hey all -- was just wondering where we currently stand Re: delaying 3. for the KRaft KIPs vs doing a 3.9 release I know we don't want to have to wait for a whole release cycle to ship these KRaft features, but delaying 3.8 up to month is also rather difficult to swallow. I just wanted to throw an

Re: [VOTE] KIP-1049: Add config log.summary.interval.ms to Kafka Streams

2024-06-11 Thread Sophie Blee-Goldman
+1 (binding) Thanks for the KIP! On Tue, Jun 11, 2024 at 5:37 AM jiang dou wrote: > HI > I would like to start a vote for KIP-1049: Add config > log.summary.interval.ms to Kafka Streams > > KIP: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1049%3A+Add+config+log.summary.interval.ms

Re: Build hanging

2024-06-07 Thread Sophie Blee-Goldman
-16916 to you. > > > Thanks, > > 2024年6月8日(土) 10:13 Apoorv Mittal : > > > Hi, > > Please find the fix for the issue: > > https://github.com/apache/kafka/pull/16249 > > > > Regards, > > Apoorv Mittal > > +44 7721681581 > > > >

Re: Build hanging

2024-06-07 Thread Sophie Blee-Goldman
gt; > > Luke > > > > > > > > On Sat, Jun 8, 2024 at 8:20 AM Haruki Okada > > wrote: > > > > > > > > > Hi > > > > > > > > > > I found that the hanging can be reproduced locally. > > > > >

Build hanging

2024-06-07 Thread Sophie Blee-Goldman
Seems like the build is currently broken -- specifically, a test is hanging and causing it to abort after 7+ hours. There are many examples in the current PRs, such as Timed out after almost 8 hours: 1. https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-16238/1/pipe

[jira] [Created] (KAFKA-16875) Replace ClientState with TaskAssignment when creating individual consumer Assignments

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16875: -- Summary: Replace ClientState with TaskAssignment when creating individual consumer Assignments Key: KAFKA-16875 URL: https://issues.apache.org/jira/browse

[jira] [Created] (KAFKA-16874) Remove old TaskAssignor interface

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16874: -- Summary: Remove old TaskAssignor interface Key: KAFKA-16874 URL: https://issues.apache.org/jira/browse/KAFKA-16874 Project: Kafka Issue

[jira] [Created] (KAFKA-16873) Remove StreamsConfig.INTERNAL_TASK_ASSIGNOR_CLASS

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16873: -- Summary: Remove StreamsConfig.INTERNAL_TASK_ASSIGNOR_CLASS Key: KAFKA-16873 URL: https://issues.apache.org/jira/browse/KAFKA-16873 Project: Kafka

[jira] [Created] (KAFKA-16872) Remove ClientState class

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16872: -- Summary: Remove ClientState class Key: KAFKA-16872 URL: https://issues.apache.org/jira/browse/KAFKA-16872 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-16871) Clean up internal AssignmentConfigs class in Streams

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16871: -- Summary: Clean up internal AssignmentConfigs class in Streams Key: KAFKA-16871 URL: https://issues.apache.org/jira/browse/KAFKA-16871 Project

[jira] [Created] (KAFKA-16869) Rewrite HighAvailabilityTaskAssignor to implement the new TaskAssignor interface

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16869: -- Summary: Rewrite HighAvailabilityTaskAssignor to implement the new TaskAssignor interface Key: KAFKA-16869 URL: https://issues.apache.org/jira/browse/KAFKA

[jira] [Created] (KAFKA-16868) Post KIP-924 StreamsPartitionAssignor code cleanup

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16868: -- Summary: Post KIP-924 StreamsPartitionAssignor code cleanup Key: KAFKA-16868 URL: https://issues.apache.org/jira/browse/KAFKA-16868 Project: Kafka

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

2024-05-31 Thread Sophie Blee-Goldman
trace will show them exactly where the bug in their assignment code occurred). And that's it! Thanks all Sophie On Tue, May 28, 2024 at 1:36 PM Sophie Blee-Goldman wrote: > Ah, one more very small thing: > > 3. We changed the name of a KafkaStreamsAssignment method from #assignme

[jira] [Created] (KAFKA-16867) Streams should run tag-based standby assignment based on rack ids

2024-05-31 Thread A. Sophie Blee-Goldman (Jira)
A. Sophie Blee-Goldman created KAFKA-16867: -- Summary: Streams should run tag-based standby assignment based on rack ids Key: KAFKA-16867 URL: https://issues.apache.org/jira/browse/KAFKA-16867

Re: [DISCUSS] KIP-1049: Add config log.summary.interval.ms to Kafka Streams

2024-05-29 Thread Sophie Blee-Goldman
houghts on this? On Thu, May 23, 2024 at 12:19 AM jiang dou wrote: > Thank you for your reply, > I do not recommend agreeing set log level is WARN, because INFO level logs > should be useful > > > Sophie Blee-Goldman 于2024年5月23日周四 04:30写道: > > > Thanks for the KIP! >

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

2024-05-28 Thread Sophie Blee-Goldman
; in the mix) On Tue, May 28, 2024 at 1:13 PM Sophie Blee-Goldman wrote: > Hey all, > > Two more quick updates to the KIP, please let me know if you have any > questions or feedback or naming suggestions: > > 1. We'd like to introduce an additional error code with the followi

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

2024-05-28 Thread Sophie Blee-Goldman
ition of the new TaskTopicPartition class Thanks! Sophie On Wed, May 15, 2024 at 3:41 PM Sophie Blee-Goldman wrote: > Thanks Bruno! > > First, need to make one quick fix to what I said in the previous email -- > the new rackId() getter will be added to Kafka

Re: [DISCUSS] KIP-1049: Add config log.summary.interval.ms to Kafka Streams

2024-05-22 Thread Sophie Blee-Goldman
Thanks for the KIP! I'm not against adding this as a config for this per se, but if this is causing you trouble right now you should be able to disable it via log4j configuration so you don't need to wait for a fix in Kafka Streams itself. Putting something like this in your log4j will shut off th

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

2024-05-15 Thread Sophie Blee-Goldman
t;>>>>>> significantly. > > >>>>>>> > > >>>>>>> > > >>>>>>> 102: It's unclear to me, how `.position` information is added. > > >>>>>>> The KIP only sa

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

2024-05-15 Thread Sophie Blee-Goldman
gt; > > > >> > > > > > >> 3, 4, 5, 6. Agreed, I'll make all those changes as suggested. > > > > > >> > > > > > >> 7. Combined with Matthias's point about RocksDB, I'm convinced > > that this > > > &

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

2024-05-15 Thread Sophie Blee-Goldman
; U3 > Yeah, I like the TaskInfo approach. However, task metadata interfaces > start to proliferate a bit too much in our code base. We have > TaskMetadata, TaskInfo, and finally Task that provide similar methods. I > think we should try to consolidate those interfaces. Does not need to > happen

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

2024-05-14 Thread Sophie Blee-Goldman
of these options, or a new suggestion of their own. As always, let us know if you have any questions or concerns or feedback of any kind. Thanks! On Mon, May 6, 2024 at 1:33 PM Sophie Blee-Goldman wrote: > Thanks guys. Updated the error codes in both the code and the explanation > under &

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

2024-05-14 Thread Sophie Blee-Goldman
+1 (binding) Thanks! On Tue, May 14, 2024 at 6:58 PM Matthias J. Sax wrote: > +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, > >> > >>

  1   2   3   4   5   6   7   8   9   >