[DISCUSS] KIP-893: The Kafka protocol should support nullable structs

2022-12-01 Thread David Jacot
Hi all, I have drafted a very small KIP which proposes to support nullable struct in the Kafka protocol. This is something that we plan to use for KIP-848. The KIP is here: https://cwiki.apache.org/confluence/x/YJIODg Please let me know what you think. Best, David

[jira] [Created] (KAFKA-14425) Automated protocol should support nullable structs

2022-11-30 Thread David Jacot (Jira)
David Jacot created KAFKA-14425: --- Summary: Automated protocol should support nullable structs Key: KAFKA-14425 URL: https://issues.apache.org/jira/browse/KAFKA-14425 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-14422) Consumer rebalance stuck after new static member joins a group with members not supporting static members

2022-11-28 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14422?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14422. - Fix Version/s: 3.4.0 3.3.2 3.2.4 Resolution: Fixed

[jira] [Created] (KAFKA-14422) Consumer rebalance stuck after new static member joins a group with members not supporting static members

2022-11-28 Thread David Jacot (Jira)
David Jacot created KAFKA-14422: --- Summary: Consumer rebalance stuck after new static member joins a group with members not supporting static members Key: KAFKA-14422 URL: https://issues.apache.org/jira/browse/KAFKA

[jira] [Resolved] (KAFKA-14009) Rebalance timeout should be updated when static member rejoins

2022-11-24 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14009?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14009. - Fix Version/s: 3.4.0 3.3.2 Resolution: Fixed > Rebalance time

[jira] [Resolved] (KAFKA-14372) RackAwareReplicaSelector should choose a replica from the isr

2022-11-24 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14372?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14372. - Fix Version/s: 3.4.0 3.3.2 Resolution: Fixed

Re: [DISCUSS] Apache Kafka 3.3.2

2022-11-16 Thread David Jacot
+1. Thanks for volunteering! On Wed, Nov 16, 2022 at 4:09 PM Chris Egerton wrote: > > Hi all, > > I'd like to volunteer to be the release manager for the next bugfix > release, 3.3.2. > > If there are no objections, I'll send out a release plan by EOD (Eastern > Time) Friday that includes a list

[jira] [Resolved] (KAFKA-14334) DelayedFetch purgatory not completed when appending as follower

2022-11-16 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14334. - Fix Version/s: 3.4.0 3.3.2 Reviewer: David Jacot Resolution

Re: [VOTE] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-16 Thread David Jacot
+1 (binding). Thanks for the KIP, Rajini! On Tue, Nov 15, 2022 at 9:26 PM Maulin Vasavada wrote: > > +1 (non-binding). > > Makes sense, Rajini. This would be a great addition. > > On Tue, Nov 15, 2022 at 10:55 AM Rajini Sivaram > wrote: > > > Hi all, > > > > I would like to start vote on

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-16 Thread David Jacot
> > > > 03) Updated the KIP to use version 3, thanks. > > > > If there are no concerns or further comments, I will start voting later > > today. > > > > Thank you, > > > > Rajini > > > > > > On Fri, Nov 4, 2022 at 9:58 AM Da

[jira] [Created] (KAFKA-14391) Add ConsumerGroupHeartbeat API

2022-11-15 Thread David Jacot (Jira)
David Jacot created KAFKA-14391: --- Summary: Add ConsumerGroupHeartbeat API Key: KAFKA-14391 URL: https://issues.apache.org/jira/browse/KAFKA-14391 Project: Kafka Issue Type: Sub-task

Re: [DISCUSS] KIP-885: Expose Broker's Name and Version to Clients Skip to end of metadata

2022-11-11 Thread David Jacot
Hi Travis, Thanks for the KIP. That seems to be a useful addition. I have a few concerns/comments: 01: Managed Kafka services do not necessarily run a specific version or may not want to expose it. I suppose that they could keep an empty string. 02: I am a bit concerned by clients that could

[jira] [Resolved] (KAFKA-14363) Add new `group-coordinator` module

2022-11-08 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14363. - Resolution: Fixed > Add new `group-coordinator` mod

[jira] [Created] (KAFKA-14367) Introduce `GroupCoordinator` interface

2022-11-08 Thread David Jacot (Jira)
David Jacot created KAFKA-14367: --- Summary: Introduce `GroupCoordinator` interface Key: KAFKA-14367 URL: https://issues.apache.org/jira/browse/KAFKA-14367 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-14363) Add new `coordinator` module

2022-11-07 Thread David Jacot (Jira)
David Jacot created KAFKA-14363: --- Summary: Add new `coordinator` module Key: KAFKA-14363 URL: https://issues.apache.org/jira/browse/KAFKA-14363 Project: Kafka Issue Type: Sub-task

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-04 Thread David Jacot
to be implemented. Best, David On Fri, Nov 4, 2022 at 10:55 AM David Jacot wrote: > > Hi Rajini, > > Thanks for the KIP. I have a few questions/comments: > > 01. If I understood correctly, the plan is to add new assignors which > are rack aware. Is this right? I wonder if it

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-04 Thread David Jacot
Hi Rajini, Thanks for the KIP. I have a few questions/comments: 01. If I understood correctly, the plan is to add new assignors which are rack aware. Is this right? I wonder if it is a judicious choice here. The main drawback is that clients must be configured correctly in order to get the

Re: [ANNOUNCE] New Kafka PMC Member: Bruno Cadonna

2022-11-02 Thread David Jacot
Congrats, Bruno! Well deserved. Le mer. 2 nov. 2022 à 06:12, Randall Hauch a écrit : > Congratulations, Bruno! > > On Tue, Nov 1, 2022 at 11:20 PM Sagar wrote: > > > Congrats Bruno! > > > > Sagar. > > > > On Wed, Nov 2, 2022 at 7:51 AM deng ziming > > wrote: > > > > > Congrats! > > > > > > --

Re: [VOTE] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-28 Thread David Jacot
or all the hard work. > > -Jason > > On Tue, Oct 25, 2022 at 7:17 AM David Jacot > wrote: > > > Hi all, > > > > The vote has been open for a while. I plan to close it on Friday if > > there are no further comments in the discussion thread. > > >

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-25 Thread David Jacot
t; Thanks for the reply. > > The KIP mentioned downgrade support in a future KIP. So, with this KIP, > once the new records have been generated on the coordinator, there is no > path to downgrade the broker, is that correct? > > Thanks, > > Jun > > On Tue, Oct 25, 2022 at 7:1

Re: [VOTE] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-25 Thread David Jacot
r comments pending in the discussion thread, and > > > one > > > > > is about whether we should merge PreparePartitionAssignment with HB. > > > But > > > > I > > > > > think the KIP itself is in pretty good shape now. Thanks! > >

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-25 Thread David Jacot
> > Thanks, > > Jun > > > On Mon, Oct 24, 2022 at 7:49 AM Magnus Edenhill wrote: > > > Hi, one minor comment on the latest update: > > > > > > Den mån 24 okt. 2022 kl 16:26 skrev David Jacot > > > >: > > > > > * Jason point

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-25 Thread David Jacot
omment on the latest update: > > > Den mån 24 okt. 2022 kl 16:26 skrev David Jacot >: > > > * Jason pointed out that the member id handling is a tad weird. The > > group coordinator generates the member id and then trusts the member > > when it rejoins the group.

Re: [DISCUSS] KIP-874: TopicRoundRobinAssignor

2022-10-25 Thread David Jacot
Hi Mathieu, Thanks for the effort that you have put in creating this KIP. I just read it again and I am still confused by the use cases and the motivation. I suppose that this works for your data model but it does not seem to be a general pattern. Overall, I stick to the comment that I made in

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-24 Thread David Jacot
do this in this KIP? > > > > That is a reasonable idea. I am wondering if there are lighter weight > > options > > though. Suppose that we used separate records for assignment metadata > > and individual member assignments. In the metadata, we might identify > &g

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-20 Thread David Jacot
e metadata, we might identify > all the member IDs that are covered by the assignment. Then when we load > the assignment, we can validate that all the member assignment records > are present. If not, then we consider it invalid and begin a > new assignment. > Just a thought. > > Th

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-19 Thread David Jacot
be possible. > > 3. The doc mentions that member metadata is stored in separate records in > order to avoid the batch limit. The group assignment, on the other hand, is > still stored as a single record. Will that be a scalability problem? > > Thanks, > Jason > > On Wed, O

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-19 Thread David Jacot
UNRELEASED_INSTANCE_ID during that time period. In case of failure of the existing member, the new member has to wait until its session expires. Best, David On Wed, Oct 19, 2022 at 10:05 AM David Jacot wrote: > > Hi Jun, > > Thanks for your thorough review. There is already a vote thread if you &g

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-19 Thread David Jacot
Ids, using topicId in > ConsumerGroupInstallAssignmentRequest makes sense. > > 81. Sounds good. > > Jun > > On Tue, Oct 18, 2022 at 11:46 AM David Jacot > wrote: > > > Hi Jun, > > > > 81. I forgot to say that I put UniformAssignor as the first one in the &g

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-18 Thread David Jacot
Hi Jun, 81. I forgot to say that I put UniformAssignor as the first one in the list. I think that it should be the default one. Best, David On Tue, Oct 18, 2022 at 8:33 PM David Jacot wrote: > > Hi Jun, > > 80. Hmm. It seems prefer

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-18 Thread David Jacot
nator should be responsible > for mapping the topic names to topic ids. > > 81. group.consumer.assignors: Should we change the default values to > include the full class name? > > Thanks, > > Jun > > On Tue, Oct 18, 2022 at 2:36 AM David Jacot > wrote: > > > Hi

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-18 Thread David Jacot
> Thanks, > > Jun > > > On Mon, Oct 17, 2022 at 2:35 AM David Jacot > wrote: > > > Hi Jun, > > > > Thanks for your comments. Please find my answers below. > > > > 60. Sure. Let me use a concrete example to illustrate it. Let's assume > >

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-17 Thread David Jacot
och? If so, isn't there a case where another member might > be assigned the partition-to-be-committed for some time before the > partition is assigned back to this consumer, which would cause the > old-but-retried (with a newly acquired epoch) offset commit to commit an > old outdated offse

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-17 Thread David Jacot
recreated. For the assigned partitions, > perhaps we could include both topicId and name just like FetchOffsetRequest. > > Thanks, > > Jun > > On Fri, Oct 14, 2022 at 2:49 AM Luke Chen wrote: > > > Thanks for the update. > > Yes, I think using similar way as KIP-868 to fix t

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-14 Thread David Jacot
Hi Luke, Thanks for your questions. > 1. We will store the "targetAssignment" into log now. But as we know, there's max batch size limit (default 1MB), which means, we cannot support 1M partitions in one group (actually, it should be less than 60k partitions since we'll store {topicID+partition

Re: [DISCUSS] (continued) KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-10-14 Thread David Jacot
Hi Jun, Thanks for your comments. Please find my answers below. 60. Yes, we need both. PartitionAssignor.onAssignment is here to inform the customer assignor about the assignment decision taken with the full set of assigned partitions regardless of whether they are already revoked or not and the

Re: [VOTE] KIP-876: Time based cluster metadata snapshots

2022-10-14 Thread David Jacot
+1 (binding) Thanks for the KIP! Le ven. 14 oct. 2022 à 05:47, deng ziming a écrit : > Thanks for this KIP, > > +1 for this(binding). > > -- > Best, > Ziming > > > On Oct 14, 2022, at 8:11 AM, José Armando García Sancio > wrote: > > > > Hello all, > > > > I would like to start voting for

[jira] [Created] (KAFKA-14297) Automated protocol should support namespace

2022-10-13 Thread David Jacot (Jira)
David Jacot created KAFKA-14297: --- Summary: Automated protocol should support namespace Key: KAFKA-14297 URL: https://issues.apache.org/jira/browse/KAFKA-14297 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-14296) Partition leaders are not demoted during kraft controlled shutdown

2022-10-13 Thread David Jacot (Jira)
David Jacot created KAFKA-14296: --- Summary: Partition leaders are not demoted during kraft controlled shutdown Key: KAFKA-14296 URL: https://issues.apache.org/jira/browse/KAFKA-14296 Project: Kafka

Re: [DISCUSS] KIP-876: Time based cluster metadata snapshots

2022-10-12 Thread David Jacot
Hi José, Thanks for the KIP. That makes total sense. On nit, I would name the new property `metadata.log.snapshot.interval.ms` as `between` is implied by the `interval`. Best, David On Tue, Oct 11, 2022 at 9:16 PM José Armando García Sancio wrote: > > Hey all, > > I am interested in allowing

Re: [ANNOUNCE] New committer: Deng Ziming

2022-10-10 Thread David Jacot
Congrats! Well deserved. Best, David Le lun. 10 oct. 2022 à 20:40, Satish Duggana a écrit : > Congratulations Ziming!! > > On Mon, 10 Oct 2022 at 11:12, Chris Egerton > wrote: > > > Congrats, Ziming! > > > > On Mon, Oct 10, 2022, 13:29 Tom Bentley wrote: > > > > > Congratulations! > > > > >

Re: [DISCUSS] Apache Kafka 3.4.0 release

2022-10-05 Thread David Jacot
+1. Thanks, Sophie! Le mer. 5 oct. 2022 à 19:57, Luke Chen a écrit : > Hi Sophie, > > Thanks for volunteering! > > Luke > > On Thu, Oct 6, 2022 at 6:17 AM José Armando García Sancio > wrote: > > > Thanks for volunteering Sophie. > > > > On Wed, Oct 5, 2022 at 3:01 PM Sophie Blee-Goldman > >

Re: [VOTE] 3.3.1 RC0

2022-09-30 Thread David Jacot
Hey, I performed the following validations: * Verified all checksums and signatures. * Built from source and ran unit tests. * Ran the first quickstart steps for both ZK and KRaft. * Spotchecked the Javadocs. I am +1 (binding), assuming that the system tests look good. Thanks for running the

Re: [VOTE] KIP-868 Metadata Transactions

2022-09-28 Thread David Jacot
+1 (binding). Thanks for the KIP. I really like the approach! Best, David On Wed, Sep 28, 2022 at 4:20 AM Luke Chen wrote: > > Hi David, > > +1 (binding) from me. > > Thank you > Luke > > On Tue, Sep 27, 2022 at 11:02 PM David Arthur > wrote: > > > Hey folks, I'd like to start a vote on

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-26 Thread David Jacot
> partitions but only the metadata for `onAssignment` may be less confusing > and push users to separate the usage of these two more clearly, but since > we already introduced partitions in `onAssignment` for compatibility I'm > less keen on removing them. > > > Guozhang > &g

Re: [VOTE] 3.3.0 RC2

2022-09-26 Thread David Jacot
Thanks for running the release, José and David. I performed the following validations: * Verified all checksums and signatures. * Built from source and ran unit tests. * Ran the first quickstart steps for both ZK and KRaft. * Spotchecked the Javadocs. I have also noticed that the doc is

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-26 Thread David Jacot
do you think that we should not provide the partitions but only the metadata? Best, David On Fri, Sep 23, 2022 at 9:40 PM Guozhang Wang wrote: > > Hello David, > > On Fri, Sep 23, 2022 at 2:00 AM David Jacot > wrote: > > > Hey, > > > > > Just to clarify I

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-23 Thread David Jacot
e when the partitions are actually > assigned to it; while for assignors, the `onAssignment` is used to indicate > what decision is made regarding for this member, i.e. when the partitions > are decided to be given to it, but not necessarily meant that it has been > given, since that t

Re: [VOTE] KIP-792: Add "generation" field into consumer protocol

2022-09-22 Thread David Jacot
Thanks, Luke. Feel free to ping me for reviews. I am happy to help on this one. Cheers, David On Thu, Sep 22, 2022 at 11:00 AM Luke Chen wrote: > > Hi David, > > Sorry for the delay. > I'll complete it in v3.4.0. > > Thank you. > Luke > > On Thu, Sep 22, 2022 a

[jira] [Created] (KAFKA-14255) Fetching from follower should be disallowed if fetch from follower is disabled

2022-09-22 Thread David Jacot (Jira)
David Jacot created KAFKA-14255: --- Summary: Fetching from follower should be disallowed if fetch from follower is disabled Key: KAFKA-14255 URL: https://issues.apache.org/jira/browse/KAFKA-14255 Project

Re: [VOTE] KIP-792: Add "generation" field into consumer protocol

2022-09-22 Thread David Jacot
Hi Luke, Are you still interested in implementing this KIP? We need it for KIP-848. If you are not, we could find someone to take it over. Thanks, David On Thu, Mar 3, 2022 at 10:04 AM Luke Chen wrote: > > Thanks, Ziming! > > So, now, this KIP vote passed with 3 binding +1 votes (David, Tom, >

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-13 Thread David Jacot
he case of consumer, Connector/Task in the case of > > Connect, Leadership in the case of Schema Registry, and so on). > > > > > > > > > From: dev@kafka.apache.org At: 08/12/22 09:31:36 UTC-4:00To: > > dev@kafka.apache.org > > > Subject: Re: [DISCUSS] KIP-848:

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-12 Thread David Jacot
at request handling section, stating when coordinator will > trigger rebalance based on the HB's member metadata / reason? > 2) the "Rebalance Triggers" section to include what we described in "Group > Epoch - Trigger a rebalance" section as well? > > > Guozhang > &

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-12 Thread David Jacot
reement on > using separate APIs for Connect. I would revisit the doc and see what > changes are to be made. > > Thanks! > Sagar. > > On Tue, Aug 9, 2022 at 7:11 PM David Jacot > wrote: > > > Hi Sagar, > > > > Thanks for the feedback and the document. That'

[VOTE] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-09 Thread David Jacot
Hi all, Thank you all for the very positive discussion about KIP-848. It looks like folks are very positive about it overall. I would like to start a vote on KIP-848, which introduces a brand new consumer rebalance protocol. The KIP is here: https://cwiki.apache.org/confluence/x/HhD1D. Best,

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-09 Thread David Jacot
r to indicate > the next HB telling broker about so. WDYT about adding such an API on the > PartitionAssignor? > > > Guozhang > > > On Tue, Sep 6, 2022 at 6:09 AM David Jacot > wrote: > > > Hi Jun, > > > > I have updated the KIP to include your f

Re: [VOTE] KIP-860: Add client-provided option to guard against unintentional replication factor change during partition reassignments

2022-09-07 Thread David Jacot
+1 from me. Thanks, Stan! On Tue, Aug 23, 2022 at 12:10 PM Luke Chen wrote: > > Hi Stanislav, > > Thanks for the KIP. > The solution looks reasonable to me. > +1 from me. > > Thank you. > Luke > > On Tue, Aug 23, 2022 at 6:07 AM Stanislav Kozlovski > wrote: > > > Hello, > > > > I'd like to

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-06 Thread David Jacot
Hi Jun, I have updated the KIP to include your feedback. I have also tried to clarify the parts which were not cleared. Best, David On Fri, Sep 2, 2022 at 4:18 PM David Jacot wrote: > > Hi Jun, > > Thanks for your feedback. Let me start by answering your questions > inline an

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-09-02 Thread David Jacot
to epoch 2, B immediately gets into > "epoch=1, partitions=[foo-2]", which seems incorrect. > 38.2 When the group transitions to epoch 3, C seems to get into epoch=3, > partitions=[foo-1] too early. > 38.3 After A transitions to epoch 3, C still has A - epoch=2, > partitions=[f

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-08-29 Thread David Jacot
. It is an implementation detail after all so it does not have to be decided at this stage. We will likely start by trying to refactor the current implementation as a first step. Cheers, David On Mon, Aug 29, 2022 at 3:52 PM David Jacot wrote: > > Hi Luke, > > > 1.1. I think the state machine are: &q

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-08-29 Thread David Jacot
] > > https://cwiki.apache.org/confluence/display/KAFKA/%5BDRAFT%5DIntegrating+Kafka+Connect+With+New+Consumer+Rebalance+Protocol > > > > Thank you. > > Luke > > > > > > > > On Fri, Aug 12, 2022 at 9:31 PM Sagar wrote: > > > >> Thank you Gu

[jira] [Resolved] (KAFKA-14097) Separate configuration for producer ID expiry

2022-08-22 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14097?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14097. - Fix Version/s: 3.4.0 Reviewer: David Jacot Assignee: Justine Olshan

Re: [DISCUSS] KIP-854 Separate configuration for producer ID expiry

2022-08-19 Thread David Jacot
Sounds good. Thanks, Justine. Le ven. 19 août 2022 à 19:38, Justine Olshan a écrit : > Hi all, > > Followed up with David and Ismael offline. > Ismael explained that we probably don't want to increase complexity and > didn't think the value needed to be modified beyond tests. I agree with >

Re: [DISCUSS] KIP-854 Separate configuration for producer ID expiry

2022-08-18 Thread David Jacot
Given that we already have `transaction.abort.timed.out.transaction.cleanup.interval.ms` and `transaction.remove.expired.transaction.cleanup.interval.ms`, it seems OK to add another one for our case here. Regarding the name, I would follow the pattern that we use for those two existing configs. We

[jira] [Resolved] (KAFKA-14148) Outdated doc for reset-offsets option

2022-08-16 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14148?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14148. - Fix Version/s: 3.3.0 Resolution: Fixed > Outdated doc for reset-offsets opt

Re: [DISCISS] KIP-860: Add client-provided option to guard against unintentional replication factor change during partition reassignments

2022-08-16 Thread David Jacot
factor+change+during+partition+reassignments > - changes: > https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=217392873=4=3 > > If there aren't extra comments, I plan on starting a vote thread by the end > of this week. > > Best, > Stanislav > > On Tue,

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-08-12 Thread David Jacot
; into > > > >> > this new scheme is that even today the connect uses a > > > WorkerCoordinator > > > >> > extending from AbstractCoordinator to empower rebalances of > > > >> > tasks/connectors. The WorkerCoordinator sets the pro

Re: Permission to assign Apache Kafka Jiras to myself

2022-08-11 Thread David Jacot
Hi Yash, You are all set. Best, David On Thu, Aug 11, 2022 at 3:47 PM Yash Mayya wrote: > > Hey folks, > > I can't currently assign Apache Kafka Jiras to myself and I just discovered > that someone needs to add me to the contributors list in order for me to be > able to do that. Could someone

[jira] [Resolved] (KAFKA-14140) Ensure an offline or in-controlled-shutdown replica is not eligible to join ISR in ZK mode

2022-08-10 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-14140. - Resolution: Fixed > Ensure an offline or in-controlled-shutdown replica is not eligible to j

Re: [DISCUSS]: Including TLA+ in the repo

2022-08-10 Thread David Jacot
That's a great idea. I am +1 as well. Thanks, David On Tue, Aug 9, 2022 at 11:54 PM Guozhang Wang wrote: > > +1 as well. I think adding such proofs in the repo could encourage more > people reviewing and challenging it, helping to improve whenever we see > fit. Also it helps readers better

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-08-09 Thread David Jacot
> > >> > > > >> > I think the changes to support connect would be similar at a high > > level > > >> to > > >> > the changes in streams mainly because of the Client side assignors > > being > > >> > used in both. At

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-08-09 Thread David Jacot
t; section: > When the group coordinator handle a ConsumerGroupPrepareAssignmentRequest > request: > -> It should be "handle a ConsumerGroupDescribe request" Thanks for reporting the typos. I will fix them. Best, David On Tue, Aug 9, 2022 at 3:40 PM David Jacot wrote: > > Hi S

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-08-09 Thread David Jacot
re for now in order to ensure that this KIP is compatible with what we will do for Connect in the future. Best, David On Mon, Aug 8, 2022 at 2:41 PM David Jacot wrote: > > Hi all, > > I am back from vacation. I will go through and address your comments > in the coming days.

Re: [DISCISS] KIP-860: Add client-provided option to guard against unintentional replication factor change during partition reassignments

2022-08-09 Thread David Jacot
Throwing an UnsupportedVersionException with an appropriate message seems to be the best option when the new API is not supported and AllowReplicationFactorChange is not set to the default value. Cheers, David On Mon, Aug 8, 2022 at 6:25 PM Vikas Singh wrote: > > I personally like the UVE

Re: [VOTE] KIP-854 Separate configuration for producer ID expiry

2022-08-09 Thread David Jacot
Thanks for the KIP, Justine. The proposal makes sense to me. I am +1 (binding). Cheers, David On Mon, Aug 8, 2022 at 6:18 PM Justine Olshan wrote: > > Hi all, > I'd like to start a vote for KIP-854: Separate configuration for producer > ID expiry. > > KIP: >

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-08-08 Thread David Jacot
t a high > > level > > >> to > > >> > the changes in streams mainly because of the Client side assignors > > being > > >> > used in both. At an implementation level, we might need to make a lot > > of > > >> > changes to get onto this new assignment prot

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-15 Thread David Jacot
I'll be away from July 18th to August 8th with limited access to my emails so I will address new comments and questions when I come back. Cheers, David On Fri, Jul 15, 2022 at 2:16 PM David Jacot wrote: > > Hi Sagar, > > Thanks for your comments. > > 1) Yes. That refers to

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-15 Thread David Jacot
a lot of other places, so have I understood it incorrectly ? > > > Regarding connect , it might be out of scope of this discussion, but from > what I understood it would probably be running in client side assignor mode > even on the new rebalance protocol as it has its own Custom A

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-15 Thread David Jacot
matically deleted. Things would be different if groups were a first class resource in the cluster. Best, David On Fri, Jul 15, 2022 at 1:30 PM David Jacot wrote: > > Thanks Hector! Our goal is to move forward with specialized API > instead of relying on one generic API. For Connect,

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-15 Thread David Jacot
us > reasons (auditability, visibility, consistency, etc.). Similarly, I am not > sure about automatically deleting configs in a way that they cannot be > recovered. A good property for modern systems is to minimize the number of > unrecoverable data loss scenarios. > > Ismael &

Re: [DISCUSS] Apache Kafka 3.2.1 release

2022-07-13 Thread David Jacot
+1. Thanks David. Le mer. 13 juil. 2022 à 23:43, José Armando García Sancio a écrit : > +1. Thanks for volunteering David. > > -- > -José >

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-13 Thread David Jacot
ead we still use Pattern > while just documenting that our subscription may be rejected by the server. > Since the incompatible case is a very rare scenario I felt using an > overloaded `String` based subscription may be more vulnerable to various > invalid regexes. > > > Guozhang

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-12 Thread David Jacot
e with your point. As a second though, using topic ids may be better here for the delete and recreation case. Also, I suppose that we may allow users to subscribe with topic ids in the future because that is the only way to be really robust to topic re-creation. Best, David On Tue, Jul 12, 2022

Re: [DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-12 Thread David Jacot
Request? Is the idea that > > since we have to resolve the regex on the server, we can do the same for > > the topic name? The difference is that sending the regex is more efficient > > whereas sending the topic names is less efficient. Furthermore, delete and > > recreat

Re: [VOTE] KIP-847

2022-07-08 Thread David Jacot
n, the KIP title > > should also be updated. > > > > Luke > > > > On Fri, Jun 24, 2022 at 8:33 PM David Jacot > > wrote: > > > > > Thanks for the KIP, Artem. > > > > > > I am +1 (binding). > > > > > > A smal

Re: [VOTE] KIP-840: Config file option for MessageReader/MessageFormatter in ConsoleProducer/ConsoleConsumer

2022-07-06 Thread David Jacot
+1 (binding). Thanks for the KIP! It is a useful addition. Best, David On Wed, Jul 6, 2022 at 9:10 PM Colin McCabe wrote: > > +1 (binding). > > thanks, Alexandre. > > On Mon, Jun 27, 2022, at 05:15, Alexandre Garnier wrote: > > Hello! > > > > A little ping on this vote. > > > > Thanks. > > > >

Re: [Vote] KIP-787 - MM2 Interface to manage Kafka resources

2022-07-06 Thread David Jacot
Thanks for the KIP, Omnia! +1 (binding) On Wed, Jul 6, 2022 at 5:02 PM Omnia Ibrahim wrote: > > Hi, > Can we have one last binding vote for this KIP, please? > > Omnia > > On Tue, Jun 28, 2022 at 3:36 PM Omnia Ibrahim > wrote: > > > Thanks, Tom, I have updated the KIP to reflect these minor

[DISCUSS] KIP-848: The Next Generation of the Consumer Rebalance Protocol

2022-07-06 Thread David Jacot
Hi all, I would like to start a discussion thread on KIP-848: The Next Generation of the Consumer Rebalance Protocol. With this KIP, we aim to make the rebalance protocol (for consumers) more reliable, more scalable, easier to implement for clients, and easier to debug for operators. The KIP is

[jira] [Created] (KAFKA-14048) The Next Generation of the Consumer Rebalance Protocol

2022-07-06 Thread David Jacot (Jira)
David Jacot created KAFKA-14048: --- Summary: The Next Generation of the Consumer Rebalance Protocol Key: KAFKA-14048 URL: https://issues.apache.org/jira/browse/KAFKA-14048 Project: Kafka Issue

Re: [VOTE] KIP-851: : Add requireStable flag into ListConsumerGroupOffsetsOptions

2022-07-01 Thread David Jacot
Hi Guozhang, Thanks for the KIP! I agree with Luke. `requireStable` seems more consistent. Regarding the kafka-consumer-group command line tool, I wonder if there is real value in doing it. We don't necessarily have to add all the options to it but we could if it is proven to be useful. Anyway,

Re: [DISCUSS] KIP-842: Add richer group offset reset mechanisms

2022-06-29 Thread David Jacot
Thanks for the KIP. I read it and I am also worried by the complexity of the new configurations. They are not easy to grasp. I need to digest it a bit more, I think. Best, David Le mer. 29 juin 2022 à 02:25, Matthias J. Sax a écrit : > Thanks for the KIP. > > I don't think I fully digested

Re: [VOTE] KIP-847

2022-06-24 Thread David Jacot
Thanks for the KIP, Artem. I am +1 (binding). A small nit: ProducerIdCount should be used in the motivation. Best, David On Thu, Jun 23, 2022 at 10:26 PM Artem Livshits wrote: > > Hello, > > I'd like to start a vote on KIP-847 >

Re: [DISCUSS] KIP-847: Add ProducerCount metrics

2022-06-22 Thread David Jacot
Hi Artem, The KIP LGTM. Thanks, David On Tue, Jun 21, 2022 at 9:32 PM Artem Livshits wrote: > > If there is no other feedback I'm going to start voting in a couple days. > > -Artem > > On Fri, Jun 17, 2022 at 3:50 PM Artem Livshits > wrote: > > > Thank you for your feedback. Updated the KIP

[jira] [Created] (KAFKA-14013) Limit the length of the `reason` field sent on the wire

2022-06-20 Thread David Jacot (Jira)
David Jacot created KAFKA-14013: --- Summary: Limit the length of the `reason` field sent on the wire Key: KAFKA-14013 URL: https://issues.apache.org/jira/browse/KAFKA-14013 Project: Kafka Issue

[jira] [Resolved] (KAFKA-13998) JoinGroupRequestData 'reason' can be too large

2022-06-20 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13998?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13998. - Resolution: Fixed > JoinGroupRequestData 'reason' can be too la

[jira] [Resolved] (KAFKA-13916) Fenced replicas should not be allowed to join the ISR in KRaft (KIP-841)

2022-06-16 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13916?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13916. - Fix Version/s: 3.3.0 Resolution: Fixed > Fenced replicas should not be allowed to j

Re: [VOTE] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-06-14 Thread David Jacot
3ZhA-Qvc0rZBYddc4u9667> > >> > >> Thanks > >> - Niket > >> > >> > >>> On May 24, 2022, at 2:30 PM, José Armando García Sancio > >> wrote: > >>> > >>> +1 (binding). > >>> > >>> On Mon

[jira] [Created] (KAFKA-13975) Mechanism to gate advertised APIs/versions based on MetadataVersion

2022-06-09 Thread David Jacot (Jira)
David Jacot created KAFKA-13975: --- Summary: Mechanism to gate advertised APIs/versions based on MetadataVersion Key: KAFKA-13975 URL: https://issues.apache.org/jira/browse/KAFKA-13975 Project: Kafka

[jira] [Created] (KAFKA-13974) Rename `INVALID_UPDATE_VERSION` to `INVALID_PARTITION_EPOCH`

2022-06-09 Thread David Jacot (Jira)
David Jacot created KAFKA-13974: --- Summary: Rename `INVALID_UPDATE_VERSION` to `INVALID_PARTITION_EPOCH` Key: KAFKA-13974 URL: https://issues.apache.org/jira/browse/KAFKA-13974 Project: Kafka

[jira] [Resolved] (KAFKA-13944) Shutting down broker can be elected as partition leader in KRaft

2022-06-08 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13944?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13944. - Fix Version/s: 3.3.0 Assignee: David Jacot (was: Jose Armando Garcia Sancio

<    1   2   3   4   5   6   7   8   9   10   >