[jira] [Created] (KAFKA-13962) KRaft StripedReplicaPlacer should handle replicas in controlled shutdown

2022-06-07 Thread David Jacot (Jira)
David Jacot created KAFKA-13962: --- Summary: KRaft StripedReplicaPlacer should handle replicas in controlled shutdown Key: KAFKA-13962 URL: https://issues.apache.org/jira/browse/KAFKA-13962 Project

Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-06-03 Thread David Jacot
Hi José, KIP-841 has been accepted. Could we add it to the release plan? Thanks, David On Wed, May 11, 2022 at 11:04 PM Chris Egerton wrote: > > Hi José, > > Could we add KIP-618 ( > https://cwiki.apache.org/confluence/display/KAFKA/KIP-618%3A+Exactly-Once+Support+for+Source+Connectors) > to

Re: [VOTE] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-06-03 Thread David Jacot
Thanks all! The vote passes with binding +1 votes from Colin, David, José and myself. David On Fri, Jun 3, 2022 at 2:14 AM José Armando García Sancio wrote: > > Thanks for proposing this improvement David Jacot. I think it is going > to make the graceful shutdown process much more

Re: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-06-03 Thread David Jacot
That's correct. David On Fri, Jun 3, 2022 at 2:11 AM José Armando García Sancio wrote: > > David Jacot wrote: > > At the moment, the KIP stipulates that the broker remains in > > InControlledShutdown state until it is re-registered with a new > > incarnation id. This i

Re: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-06-02 Thread David Jacot
nce we are adding a new > field to a metadata record. Otherwise we might have trouble downgrading the > software if this field has been serialized. > > Cheers, > David > > On Thu, Jun 2, 2022 at 11:04 AM David Jacot > wrote: > > > Hi José, > > > > At the

Re: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-06-02 Thread David Jacot
Hi José, At the moment, the KIP stipulates that the broker remains in InControlledShutdown state until it is re-registered with a new incarnation id. This implies that a broker can be both fenced and in controlled shutdown state. We could make them mutually exclusive but I think that there is

Re: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-06-01 Thread David Jacot
? > > Ismael > > On Tue, May 31, 2022, 6:36 AM David Jacot > wrote: > > > Hi all, > > > > Thanks for your feedback. > > > > I just updated the KIP as follows: > > * I propose to add a `TopicId` field at the same time while we are

Re: [VOTE] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-06-01 Thread David Jacot
rror codes so we > > can track which ones are returned in which RPC version. I also wonder > > Here I'm referring to AlterPartitionResponse.json > > cheers, > Colin > > > > > > > On Tue, May 31, 2022, at 08:36, David Jacot wrote: > >> Hi folks, > >> > >> I'd like to start a vote for KIP-841: > >> https://cwiki.apache.org/confluence/x/phmhD. > >> > >> Thanks, > >> David

[VOTE] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-05-31 Thread David Jacot
Hi folks, I'd like to start a vote for KIP-841: https://cwiki.apache.org/confluence/x/phmhD. Thanks, David

Re: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-05-31 Thread David Jacot
Hi all, Thanks for your feedback. I just updated the KIP as follows: * I propose to add a `TopicId` field at the same time while we are bumping the AlterPartition API version. * I propose to add the `InShuttingDown` field to the registration records to track if a broker is in controlled

Re: [VOTE] KIP-843: Adding metricOrElseCreate method to Metrics

2022-05-30 Thread David Jacot
Hi all, Looking at the current Metrics' API, we have `sensor` which gets or creates a sensor. How about using `metric` to follow the same naming convention? Best, David On Mon, May 30, 2022 at 9:18 AM Bruno Cadonna wrote: > > Hi Sagar, > Hi Ismael, > > what about getOrCreateMetric()? > > Best,

Re: [VOTE] KIP-833: Mark KRaft as Production Ready

2022-05-24 Thread David Jacot
+1. Thanks Colin! On Tue, May 24, 2022 at 4:50 AM Luke Chen wrote: > > +1 from me. > Kraft is coming!!! > > Luke > > On Tue, May 24, 2022 at 7:26 AM Israel Ekpo wrote: > > > +1 (non-binding) from me. > > > > I am very happy to finally see this. > > > > On Mon, May 23, 2022 at 6:08 PM Jason

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

2022-05-21 Thread David Jacot
+1 (binding) Thanks for the KIP! David Le ven. 20 mai 2022 à 21:08, David Arthur a écrit : > Hey Niket, +1 (binding) from me. > > -David > > On Thu, May 19, 2022 at 2:07 PM José Armando García Sancio > wrote: > > > Hey Niket, > > > > I took a look at the latest KIP. It looks like

Re: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-05-20 Thread David Jacot
This KIP is pretty straight forward. I will start a vote on Monday if no one objects. Best, David On Wed, May 18, 2022 at 5:55 PM David Jacot wrote: > > Hi, > > I created a small KIP to strengthen the AlterPartition API in KRaft mode: > https://cwiki.apache.org/confluence/x/p

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

2022-05-19 Thread David Jacot (Jira)
David Jacot created KAFKA-13916: --- Summary: Fenced replicas should not be allowed to join the ISR in KRaft (KIP-841) Key: KAFKA-13916 URL: https://issues.apache.org/jira/browse/KAFKA-13916 Project

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

2022-05-19 Thread David Jacot
Hi Niket, Thanks for the KIP. I have a few minor comments: 1. We should keep DescribeQuorumResult's constructor package-private for now in my opinion. We have been debating about this in KIP-692 and KIP-777 but as we haven't reached a consensus yet, we should be on the conservative side here.

[DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-05-18 Thread David Jacot
Hi, I created a small KIP to strengthen the AlterPartition API in KRaft mode: https://cwiki.apache.org/confluence/x/phmhD Let me know what you think. Best, David

Re: [ANNOUNCE] Apache Kafka 3.1.1

2022-05-16 Thread David Jacot
eal-time streaming applications that transform or react > to the streams of data. > > > Apache Kafka is in use at large and small companies worldwide, including > Capital One, Goldman Sachs, ING, LinkedIn, Netflix, Pinterest, Rabobank, > Target, The New York Times, Uber, Yelp, and

[jira] [Resolved] (KAFKA-12703) Allow unencrypted private keys when using PEM files

2022-05-16 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12703?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-12703. - Fix Version/s: 3.3.0 Resolution: Fixed > Allow unencrypted private keys when using

[jira] [Resolved] (KAFKA-13790) ReplicaManager should be robust to all partition updates from kraft metadata log

2022-05-09 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13790. - Fix Version/s: 3.3.0 Reviewer: Jason Gustafson Resolution: Fixed

Re: [VOTE] 3.2.0 RC1

2022-05-06 Thread David Jacot
Thanks for running the release, Bruno. 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 doc and the Javadocs. +1 (binding) Best, David On Thu, May 5,

Re: [VOTE] 3.1.1 RC1

2022-05-06 Thread David Jacot
Thanks for running the release, Tom. 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 noticed the same issues as others on the website. I

[jira] [Created] (KAFKA-13844) Pull Partition's volatile fields into a consolidate `PartitionState`

2022-04-21 Thread David Jacot (Jira)
David Jacot created KAFKA-13844: --- Summary: Pull Partition's volatile fields into a consolidate `PartitionState` Key: KAFKA-13844 URL: https://issues.apache.org/jira/browse/KAFKA-13844 Project: Kafka

[jira] [Resolved] (KAFKA-13542) Utilize the new Consumer#enforceRebalance(reason) API in Streams

2022-04-13 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13542?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13542. - Fix Version/s: 3.2.0 (was: 3.3.0) Resolution: Fixed > Util

[jira] [Resolved] (KAFKA-13828) Ensure reasons sent by the consumer are small

2022-04-13 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13828?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13828. - Fix Version/s: 3.2.0 Resolution: Fixed > Ensure reasons sent by the consumer are sm

[jira] [Created] (KAFKA-13828) Ensure reasons sent by the consumer are small

2022-04-13 Thread David Jacot (Jira)
David Jacot created KAFKA-13828: --- Summary: Ensure reasons sent by the consumer are small Key: KAFKA-13828 URL: https://issues.apache.org/jira/browse/KAFKA-13828 Project: Kafka Issue Type: Bug

Re: [VOTE] 3.1.1 RC0

2022-04-12 Thread David Jacot
Hi Tom, Thanks for running the release. I wonder if we should include: https://github.com/apache/kafka/commit/134c432d6452de1bfb99d0f6b455a58c16bc626a. This is a follow up of KAFKA-13794. What do you think? Best, David On Fri, Apr 8, 2022 at 6:18 PM Tom Bentley wrote: > > Hello Kafka users,

[jira] [Resolved] (KAFKA-13801) Kafka server does not respect MetricsReporter interface contract for dynamically configured reporters

2022-04-07 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13801?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13801. - Fix Version/s: 3.3.0 Reviewer: David Jacot Assignee: Xavier Léauté

[jira] [Resolved] (KAFKA-13778) Fetch from follower should never run the preferred read replica selection

2022-04-05 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13778?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13778. - Fix Version/s: 3.3.0 Reviewer: David Jacot Resolution: Fixed > Fetch f

Re: [VOTE] KIP-800: Add reason to LeaveGroupRequest

2022-04-04 Thread David Jacot
have removed the prefixing. It does not bring much anyway as we are still able to distinguish a custom reason from the default one. [1] https://github.com/apache/kafka/pull/11873 Best, David On Wed, Dec 1, 2021 at 2:17 PM David Jacot wrote: > > Hi all, > > With 4 binding +1 votes (G

[jira] [Resolved] (KAFKA-13783) Remove reason prefixing in JoinGroupRequest and LeaveGroupRequest

2022-03-31 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13783?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13783. - Resolution: Fixed > Remove reason prefixing in JoinGroupRequest and LeaveGroupRequ

[jira] [Created] (KAFKA-13783) Remove reason prefixing in JoinGroupRequest and LeaveGroupRequest

2022-03-30 Thread David Jacot (Jira)
David Jacot created KAFKA-13783: --- Summary: Remove reason prefixing in JoinGroupRequest and LeaveGroupRequest Key: KAFKA-13783 URL: https://issues.apache.org/jira/browse/KAFKA-13783 Project: Kafka

[jira] [Resolved] (KAFKA-13767) Fetch from consumers should return immediately when preferred read replica is defined by the leader

2022-03-29 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13767?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13767. - Reviewer: David Jacot Resolution: Fixed > Fetch from consumers should return immediat

Re: [VOTE] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-26 Thread David Jacot
+1 (binding). Thanks for the KIP. Best, David Le ven. 25 mars 2022 à 07:11, Luke Chen a écrit : > Hi Sergio, > > Thanks for the KIP! > +1(binding) from me. > > Thank you. > Luke > > On Fri, Mar 25, 2022 at 1:40 PM Sergio Daniel Troiano > wrote: > > > Hi lads, > > > > I would like to start the

Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-25 Thread David Jacot
st regards > Sergio Troiano > > On Fri, 25 Mar 2022 at 06:50, David Jacot wrote: > > > Hi Sergio, > > > > I made a suggestion a few weeks ago about the name about the parameter but > > haven’t got a response for it. Did you consider it? > >

Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-24 Thread David Jacot
Hi Sergio, I made a suggestion a few weeks ago about the name about the parameter but haven’t got a response for it. Did you consider it? Do we need to update the rejected alternatives section to mention the alternative options discussed in this thread? Thanks, David Le ven. 25 mars 2022 à

Re: [DISCUSS] KIP-651 - Support PEM format for SSL certificates and

2022-03-24 Thread David Jacot
Hi all, Thanks Dejan for bringing this up. Relaxing this constraint seems reasonable to me. I guess we would have to relax it for the keystores at some point in the future as well (with Java 18). Let's wait a few days to see what others think about this. Best, David On Wed, Mar 23, 2022 at

Re: [VOTE] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-23 Thread David Jacot
Hi Artem, Thanks for the KIP. This is a really nice improvement! +1 (binding) from me. David On Tue, Mar 22, 2022 at 9:35 PM Jun Rao wrote: > > Hi, Artem, > > Thanks for the KIP. +1 from me. > > Jun > > On Mon, Mar 21, 2022 at 4:52 PM Artem Livshits > wrote: > > > Hi all, > > > > I'd like to

[jira] [Resolved] (KAFKA-13752) Using `equals` instead of `==` when Uuid compare in Java

2022-03-22 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13752?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13752. - Fix Version/s: 3.2.0 3.1.1 3.3.0 Resolution: Fixed

[jira] [Resolved] (KAFKA-13497) Debug Log RegexRouter transform

2022-03-18 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13497. - Fix Version/s: 3.2.0 Reviewer: David Jacot Resolution: Fixed > Debug

[jira] [Resolved] (KAFKA-13584) Fix `kafka.network.SocketServerTest.testUnmuteChannelWithBufferedReceives` flaky test

2022-03-18 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13584. - Resolution: Fixed Fixed by https://github.com/apache/kafka/commit

[jira] [Resolved] (KAFKA-13750) Client Compatability KafkaTest uses invalid idempotency configs

2022-03-17 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13750?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13750. - Fix Version/s: 3.2.0 3.1.1 3.0.2 Resolution: Fixed

[jira] [Resolved] (KAFKA-13509) Support max timestamp in GetOffsetShell

2022-03-17 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13509?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13509. - Reviewer: David Jacot Resolution: Fixed > Support max timestamp in GetOffsetSh

Re: [DISCUSS] Apache Kafka 3.1.1

2022-03-09 Thread David Jacot
+1. Thanks Tom! Le mer. 9 mars 2022 à 19:10, Bill Bejeck a écrit : > Thanks Tom! It's a +1 for me. > > -Bill > > On Wed, Mar 9, 2022 at 12:00 PM Ismael Juma wrote: > > > Thanks Tom. +1 > > > > Ismael > > > > > > On Wed, Mar 9, 2022 at 8:10 AM Tom Bentley wrote: > > > > > Hi, > > > > > > I'd

Re: [VOTE] 3.0.1 RC0

2022-03-09 Thread David Jacot
Thanks for running the release, Mickael. 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. However, the document still references 3.0.0 in all

Re: [DISCUSS] KIP-824 Allowing dumping segmentlogs limiting the batches in the output

2022-03-05 Thread David Jacot
Hi Sergio, I wonder if « max-bytes » would be a better name than « max-batches-size ». The intend is more explicit. What do you think? Best, David Le sam. 5 mars 2022 à 10:36, Luke Chen a écrit : > Hi Sergio, > > Thanks for the explanation! Very clear! > I think we should put this example and

[jira] [Resolved] (KAFKA-13706) org.apache.kafka.test.MockSelector doesn't remove closed connections from its 'ready' field

2022-03-04 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13706?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13706. - Fix Version/s: 3.2.0 Reviewer: David Jacot Assignee: Vincent Jiang

Re: [DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2022-03-04 Thread David Jacot
Hi Artem, Thanks for the KIP! I have a few comments: 1. In the preamble of the proposed change section, there is still a mention of the -1 approach. My understanding is that we have moved away from it now. 2. I am a bit concerned by the trick suggested about the DefaultPartitioner and the

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

2022-03-03 Thread David Jacot
+1 (binding). Thanks for the KIP! On Fri, Jan 28, 2022 at 6:13 PM Tom Bentley wrote: > > Hi Luke, > > Thanks for the KIP, +1 (binding). > > Kind regards, > > Tom > > On Wed, 19 Jan 2022 at 13:16, Luke Chen wrote: > > > Hi all, > > > > Bump this thread to see if there are other comments to this

Re: [VOTE] KIP-815: Support max-timestamp in GetOffsetShell

2022-02-28 Thread David Jacot
- 4 +1(binding) votes (Luke, David, Mickael and John) > > Thanks to all that participated in the discussion and voting, > > -- > Ziming Deng > > > > On Feb 22, 2022, at 2:56 PM, David Jacot wrote: > > > > For reference, here is the KIP: > > https://cwiki

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

2022-02-21 Thread David Jacot
gt; return generationId; > } > > Thank you. > Luke > > On Mon, Feb 21, 2022 at 5:24 PM David Jacot > wrote: > > > Hi Luke, > > > > I apologize for my late reply. I was out for a while. > > > > Coming back to my previous point, could you also

Re: [VOTE] KIP-815: Support max-timestamp in GetOffsetShell

2022-02-21 Thread David Jacot
For reference, here is the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-815%3A++Support+max-timestamp+in+GetOffsetShell Thanks for the KIP! +1 (binding) Best, David Le mar. 22 févr. 2022 à 04:03, deng ziming a écrit : > Hey all, I'm starting the voting on KIP-815. > > This

Re: [VOTE] KIP-797: Accept duplicate listener on port for IPv4/IPv6

2022-02-21 Thread David Jacot
Hi Matthew, Thanks for the KIP. I have a few minor comments: 1. In the public interface section, could we spell out the configurations that we are changing with this KIP? The name does not change but the semantic is so it is good to be clear. 2. In the proposed changes section, I would rather

Re: [VOTE] KIP-815: Replace KafkaConsumer with AdminClient in GetOffsetShell

2022-02-21 Thread David Jacot
nfigurations, the main difference > are default.api.timeout.ms <http://default.api.timeout.ms/>, retries, > request.timeout.ms <http://request.timeout.ms/>,metadata.max.age.ms > <http://metadata.max.age.ms/>. Please take a look again when you are free. > > >

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

2022-02-21 Thread David Jacot
can add the `StickyAssignor` > > > in your paragraph talking about `CooperativeStickyAssignor` as well. > > > > > > * This sentence: "otherwise, we'll take the ownedPartitions as default > > > generation(-1)." does not read right to me, maybe need to

[jira] [Resolved] (KAFKA-13435) Static membership protocol should let the leader skip assignment (KIP-814)

2022-02-14 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13435?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13435. - Reviewer: Jason Gustafson Resolution: Fixed > Static membership protocol should

[jira] [Resolved] (KAFKA-13636) Committed offsets could be deleted during a rebalance if a group did not commit for a while

2022-02-10 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13636?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13636. - Fix Version/s: 3.0.1 2.8.2 3.2.0 3.1.1

Re: [VOTE] KIP-704: Send a hint to partition leader to recover the partition

2022-02-07 Thread David Jacot
+1, thanks! Le lun. 7 févr. 2022 à 17:41, José Armando García Sancio a écrit : > Thanks for the vote Jason. > > This is a friendly reminder to vote on this KIP. > > -- > -José >

[DISCUSS] Should we automatically close stale PRs?

2022-02-05 Thread David Jacot
Hi team, I find our ever growing back of PRs a little frustrating, don't you? I just made a pass over all the list and a huge chunk of the PRs are abandoned, outdated or irrelevant with the current code base. For instance, we still have PRs opened back in 2015. There is not a Github Action [1]

Re: [DISCUSS] KIP-817: Fix inconsistency in dynamic application log levels

2022-02-04 Thread David Jacot
Hi Dongjin, Thanks for the KIP. It is not so clear to me why we decided not to support OFF in the first place. I understand that entirely disabling a logger is rare. I find the KIP a bit week at the moment for two reasons: 1) The KIP says that the levels that we use are not fully consistent

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

2022-02-04 Thread David Jacot
nd also add a field > >> into `JoinGroupResponse` in `JoinGroupResponseMember` field. That way, the > >> broker can identify the old member from `JoinGroupRequest`. And the > >> assignor can also get the "generation" info via the `Subscription` > >> instance. >

Re: [DISCUSS] Apache Kafka 3.2.0 release

2022-02-04 Thread David Jacot
+1. Thanks for volunteering, Bruno! Le ven. 4 févr. 2022 à 16:03, Bruno Cadonna a écrit : > Hi, > > I'd like to volunteer to be the release manager for our next > feature release, 3.2.0. If there are no objections, I'll send > out the release plan soon. > > Best, > Bruno >

Re: [DISCUSS] KIP-704: Send a hint to broker if it is an unclean leader

2022-02-03 Thread David Jacot
Sancio wrote: > > Thanks for the feedback David Jacot > > David Jacot wrote: > > I have one question regarding how fetch from followers will > > work when the leader is recovering. My understanding is that > > the leader will reject any produce and fetch requests with a &g

Re: [VOTE] KIP-814: Static membership protocol should let the leader skip assignment

2022-02-03 Thread David Jacot
ship protocol should let the > > leader skip assignment > > > > Hi David, > > > > Thanks for the KIP. > > > > I'm +1(non-binding) > > > > Thanks. > > Luke > > > > Jason Gustafson 於 2022年2月1日 週二 上午7:11 > > 寫道: > > >

[jira] [Resolved] (KAFKA-13583) Fix FetchRequestBetweenDifferentIbpTest flaky tests

2022-02-03 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13583?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13583. - Fix Version/s: 3.2.0 3.1.1 Reviewer: David Jacot Resolution

[jira] [Resolved] (KAFKA-13637) User default.api.timeout.ms config as default timeout for KafkaConsumer.endOffsets

2022-02-03 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13637. - Fix Version/s: 3.0.1 3.2.0 3.1.1 Reviewer: David

Re: [DISCUSS] KIP-704: Send a hint to broker if it is an unclean leader

2022-02-02 Thread David Jacot
Hi José, Thanks for the KIP. I have one question regarding how fetch from followers will work when the leader is recovering. My understanding is that the leader will reject any produce and fetch requests with a NOT_LEADER_OR_FOLLOWER error while the followers will fence any fetch requests based

Re: [VOTE] KIP-815: Replace KafkaConsumer with AdminClient in GetOffsetShell

2022-02-01 Thread David Jacot
at > we use > different timeout in `KafkaConsumer.beginningOffsets` and > `KafkaConsumer.endOffsets`, > I will fix this. > > Please help to review the KIP and the bug, thank you. > > Best, > Ziming Deng > > > > On Feb 1, 2022, at 6:31 PM, David Jac

Re: [VOTE] KIP-815: Replace KafkaConsumer with AdminClient in GetOffsetShell

2022-02-01 Thread David Jacot
will retry until `default.api.timeout.ms` is reached and it seems that the admin client does the same by default. Do you confirm this? Best, David On Mon, Jan 31, 2022 at 11:12 AM David Jacot wrote: > > Hey, > > Thanks for the KIP. I have a few comments: > > 1. I think that it would

Re: [DISCUSS] KIP-795: Add public APIs for AbstractCoordinator

2022-02-01 Thread David Jacot
Hi Hector, Thanks for the KIP. I finally had a bit of time to read it. I understand that a few services have been leveraging Kafka's Group Membership Protocol to do leader election and/or service discovery. However, I am not entirely convinced that Kafka should be used in that way because

Re: [VOTE] KIP-815: Replace KafkaConsumer with AdminClient in GetOffsetShell

2022-01-31 Thread David Jacot
Hey, Thanks for the KIP. I have a few comments: 1. I think that it would be better to name the KIP: "GetOffsetShell should support max-timestamp" or something like that as this is the initial intent of the change. 2. There is a typo: `OffsetSpce` -> `OffsetSpec`. 3. It would be great if we

[VOTE] KIP-814: Static membership protocol should let the leader skip assignment

2022-01-31 Thread David Jacot
Hi all, I'd like to start a vote about KIP-814: Static membership protocol should let the leader skip assignment. The KIP is here: https://cwiki.apache.org/confluence/x/C5-kCw. Best, David

Re: [DISCUSS] KIP-814: Static membership protocol should let the leader skip assignment

2022-01-28 Thread David Jacot
hat right? > I think you should make it clear in the KIP. > > Thank you. > Luke > > On Fri, Jan 28, 2022 at 8:00 PM David Jacot > wrote: > > > Hi all, > > > > If there are no further comments. I plan to start a vote thread > > on Monday. > > &g

Re: [DISCUSS] KIP-814: Static membership protocol should let the leader skip assignment

2022-01-28 Thread David Jacot
Hi all, If there are no further comments. I plan to start a vote thread on Monday. Best, David On Wed, Jan 26, 2022 at 3:16 PM David Jacot wrote: > > Hi Hector, > > Thanks for your comments. > > The idea is to add a flag to the onJoinLeader leader method. We >

[jira] [Resolved] (KAFKA-13614) Leader replication quota is applied to consumer fetches

2022-01-26 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13614?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13614. - Fix Version/s: 3.2.0 Reviewer: David Jacot Resolution: Fixed > Lea

Re: [DISCUSS] KIP-814: Static membership protocol should let the leader skip assignment

2022-01-26 Thread David Jacot
right. We are thinking about a new consumer protocol but we don't have anything concrete to share with the community yet. We hope to have something in the near future. Best, David On Wed, Jan 26, 2022 at 3:10 PM David Jacot wrote: > > Hey Jason, > > I've updated the KIP based on y

Re: [DISCUSS] KIP-814: Static membership protocol should let the leader skip assignment

2022-01-26 Thread David Jacot
's own subscription). The leader will then send an empty > > > `SyncGroup` request to collect its own assignment. Do I have that right? > > > > > > I think there might still be an edge case in this proposal (assuming I've > > > understood it correctly). In between the

[jira] [Resolved] (KAFKA-13585) Fix `kafka.server.ReplicaManagerTest.testReplicaAlterLogDirsWithAndWithoutIds` flaky test

2022-01-25 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13585?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13585. - Fix Version/s: 3.2.0 3.1.1 Reviewer: Jason Gustafson

[ANNOUNCE] Apache Kafka 3.1.0

2022-01-24 Thread David Jacot
Urban, David Arthur, David Jacot, David Mao, Dmitriy Fishman, Edoardo Comar, Ewen Cheslack-Postava, Greg Harris, Guozhang Wang, Igor Soarez, Ismael Juma, Israel Ekpo, Ivan Ponomarev, Jakub Scholz, James Galasyn, Jason Gustafson, Jeff Kim, Jim Galasyn, JoeCqupt, Joel Hamill, John Gray, John Roesler

Re: [DISCUSS] KIP-814: Static membership protocol should let the leader skip assignment

2022-01-24 Thread David Jacot
subscribed topics. The returning leader would not know about it > because it has no way to collect the full assignment. Do you think this is > a problem? > > Thanks, > Jason > > On Wed, Jan 19, 2022 at 7:27 AM David Jacot wrote: > > > Hi folks, > > > > I'd like

[jira] [Resolved] (KAFKA-13388) Kafka Producer nodes stuck in CHECKING_API_VERSIONS

2022-01-21 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13388?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13388. - Fix Version/s: 3.0.1 3.2.0 3.1.1 Reviewer: Rajini

[jira] [Resolved] (KAFKA-13591) Fix flaky test ControllerIntegrationTest.testTopicIdCreatedOnUpgrade

2022-01-21 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13591?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13591. - Fix Version/s: 3.2.0 3.1.1 Reviewer: Jason Gustafson

[RESULTS] [VOTE] Release Kafka version 3.1.0

2022-01-21 Thread David Jacot
This vote passes with 7 +1 votes (4 bindings) and no 0 or -1 votes. +1 votes PMC Members: * Randall Hauch * Bill Bejeck * Mickael Maison * David Jacot Committers: * No votes Community: * Michal Tóth * Jakub Scholz * Israel Ekpo 0 votes * No votes -1 votes * No votes Vote thread: https

Re: [VOTE] 3.1.0 RC1

2022-01-21 Thread David Jacot
"Kafka Streams demo > > and > > the app development tutorial" links still reference the 2.5 release. > > This is a very minor issue and it doesn't require a new RC, we can simply > > update the links once released. > > > > +1(binding) > > > >

[DISCUSS] KIP-814: Static membership protocol should let the leader skip assignment

2022-01-19 Thread David Jacot
Hi folks, I'd like to start a discussion for KIP-814: Static membership protocol should let the leader skip assignment. This is a small extension to the static membership protocol to address KAFKA-13435. The KIP is here: https://cwiki.apache.org/confluence/x/C5-kCw. Please let me know what you

Re: [VOTE] 3.1.0 RC1

2022-01-19 Thread David Jacot
t;- Briefly Walked Through 3.1 Documentation, Javadocs and Protocol Pages > > +1 (non-binding) for the release candidate > > Thanks for running this release > > Israel Ekpo > Lead Instructor, IzzyAcademy.com > https://www.youtube.com/c/izzyacademy > https://izzyacade

Re: Permission to contribute

2022-01-14 Thread David Jacot
Hey, Done. Thanks for your interest! Best, David On Fri, Jan 14, 2022 at 10:58 AM Daan Gertis wrote: > > Hey everyone, > > I would like to contribute a KIP that I think makes sense when building > event-driven architectures at scale. I have been working with kafka for the > past 6 years or

[jira] [Resolved] (KAFKA-13451) Add reason to JoinGroupRequest and LeaveGroupRequest

2022-01-13 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13451. - Fix Version/s: 3.2.0 Reviewer: David Jacot Resolution: Fixed > Add rea

[jira] [Resolved] (KAFKA-13496) Add reason to LeaveGroupRequest

2022-01-13 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13496?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13496. - Fix Version/s: 3.2.0 Reviewer: David Jacot Resolution: Fixed > Add rea

[jira] [Resolved] (KAFKA-13495) Add reason to JoinGroupRequest

2022-01-13 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13495?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13495. - Fix Version/s: 3.2.0 Reviewer: David Jacot Resolution: Fixed > Add rea

[VOTE] 3.1.0 RC1

2022-01-12 Thread David Jacot
Hello Kafka users, developers and client-developers, This is the second candidate for release of Apache Kafka 3.1.0. * Apache Kafka supports Java 17 * The FetchRequest supports Topic IDs (KIP-516) * Extend SASL/OAUTHBEARER with support for OIDC (KIP-768) * Add broker count metrics (KIP-748) *

[jira] [Created] (KAFKA-13592) Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-12 Thread David Jacot (Jira)
David Jacot created KAFKA-13592: --- Summary: Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions Key: KAFKA-13592 URL: https://issues.apache.org/jira/browse/KAFKA-13592

[jira] [Created] (KAFKA-13591) Fix flaky test ControllerIntegrationTest.testTopicIdCreatedOnUpgrade

2022-01-11 Thread David Jacot (Jira)
David Jacot created KAFKA-13591: --- Summary: Fix flaky test ControllerIntegrationTest.testTopicIdCreatedOnUpgrade Key: KAFKA-13591 URL: https://issues.apache.org/jira/browse/KAFKA-13591 Project: Kafka

Re: [VOTE] 3.1.0 RC0

2022-01-11 Thread David Jacot
> > +1 (non-binding). I used the staged Scala 2.13 binaries and the staging > > > Maven repository to run my tests. All seems to work fine. > > > > > > Thanks for running the release David! > > > > > > Jakub > > > > > > On Thu, Dec 23, 20

[jira] [Created] (KAFKA-13585) Fix `kafka.server.ReplicaManagerTest.testReplicaAlterLogDirsWithAndWithoutIds` flaky test

2022-01-10 Thread David Jacot (Jira)
David Jacot created KAFKA-13585: --- Summary: Fix `kafka.server.ReplicaManagerTest.testReplicaAlterLogDirsWithAndWithoutIds` flaky test Key: KAFKA-13585 URL: https://issues.apache.org/jira/browse/KAFKA-13585

[jira] [Created] (KAFKA-13584) Fix `kafka.network.SocketServerTest.testUnmuteChannelWithBufferedReceives` flaky test

2022-01-10 Thread David Jacot (Jira)
David Jacot created KAFKA-13584: --- Summary: Fix `kafka.network.SocketServerTest.testUnmuteChannelWithBufferedReceives` flaky test Key: KAFKA-13584 URL: https://issues.apache.org/jira/browse/KAFKA-13584

[jira] [Created] (KAFKA-13583) Fix FetchRequestBetweenDifferentIbpTest flaky tests

2022-01-10 Thread David Jacot (Jira)
David Jacot created KAFKA-13583: --- Summary: Fix FetchRequestBetweenDifferentIbpTest flaky tests Key: KAFKA-13583 URL: https://issues.apache.org/jira/browse/KAFKA-13583 Project: Kafka Issue Type

[jira] [Created] (KAFKA-13582) `TestVerifiableProducer. test_multiple_kraft_security_protocols` consistently fails

2022-01-10 Thread David Jacot (Jira)
David Jacot created KAFKA-13582: --- Summary: `TestVerifiableProducer. test_multiple_kraft_security_protocols` consistently fails Key: KAFKA-13582 URL: https://issues.apache.org/jira/browse/KAFKA-13582

[jira] [Resolved] (KAFKA-13312) 'NetworkDegradeTest#test_rate' should wait until iperf server is listening

2022-01-10 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13312?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13312. - Resolution: Fixed > 'NetworkDegradeTest#test_rate' should wait until iperf server is listen

[jira] [Resolved] (KAFKA-10712) Update release scripts to Python3

2022-01-10 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10712?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-10712. - Fix Version/s: 3.1.0 Resolution: Fixed > Update release scripts to Pyth

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