Re: [VOTE] KIP-853: KRaft Controller Membership Changes

2024-04-22 Thread Jason Gustafson
Thanks Jose. +1. Great KIP! On Fri, Mar 29, 2024 at 11:16 AM Jun Rao wrote: > Hi, Jose, > > Thanks for the KIP. +1 > > Jun > > On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio > wrote: > > > Hi all, > > > > I would like to call a vote to adopt KIP-853. > > > > KIP:

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

2024-02-21 Thread Jason Gustafson
+1 Thanks for the KIP! On Wed, Feb 21, 2024 at 9:15 AM Jun Rao wrote: > Hi, Apoorv, > > Thanks for the KIP. +1 > > Jun > > On Mon, Feb 19, 2024 at 2:32 PM Apoorv Mittal > wrote: > > > Hi, > > I’d like to start the voting for KIP-1019: Expose method to determine > > Metric Measurability. > > >

Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-05 Thread Jason Gustafson
Hey Jose, A few more questions: 1. When adding a voter, the KIP proposes to return a timeout error if the voter cannot catch up in time. It might be more useful to return a more specific error so that an operator can understand why the timeout occurred. Alternatively, perhaps we could keep the

[jira] [Created] (KAFKA-16179) NPE handle ApiVersions during controller failover

2024-01-19 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-16179: --- Summary: NPE handle ApiVersions during controller failover Key: KAFKA-16179 URL: https://issues.apache.org/jira/browse/KAFKA-16179 Project: Kafka

Re: [VOTE] KIP-994: Minor Enhancements to ListTransactions and DescribeTransactions APIs

2024-01-11 Thread Jason Gustafson
HI Raman, Thanks for the KIP! +1 from me. One small thing: we will probably have to overload the constructor for TransactionDescription in order to add the new update time field to avoid breaking the API. We might consider whether we need the overload to be public or not. Best, Jason On Tue,

Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-10 Thread Jason Gustafson
ely? Or do we give up and return a timeout error if the new voter cannot catch up? Probably the latter makes the most sense. Thanks, Jason On Tue, Jan 9, 2024 at 11:42 PM Colin McCabe wrote: > On Tue, Jan 9, 2024, at 17:07, Jason Gustafson wrote: > > Hi Jose, > > > > T

Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-09 Thread Jason Gustafson
Hi Jose, Thanks for the KIP! A few initial questions below: 1. In the user experience section, the user is expected to provide supply the UUID for each voter. I'm assuming this is the directory.id coming from KIP-858. I thought it was generated by the format comand automatically? It seems like

[jira] [Created] (KAFKA-16012) Incomplete range assignment in consumer

2023-12-13 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-16012: --- Summary: Incomplete range assignment in consumer Key: KAFKA-16012 URL: https://issues.apache.org/jira/browse/KAFKA-16012 Project: Kafka Issue Type

Re: [VOTE] KIP-996: Pre-Vote

2023-12-07 Thread Jason Gustafson
+1 Thanks for the KIP! Nice to see progress with the raft protocol. On Thu, Dec 7, 2023 at 5:10 PM Alyssa Huang wrote: > Hey folks, > > I would like to start a vote on Pre-vote  Thank you Jose, Jason, Luke, > and Jun for your comments on the discussion thread! > > Here's the link to the

Re: [VOTE] KIP-896: Remove old client protocol API versions in Kafka 4.0

2023-12-07 Thread Jason Gustafson
Minor correction: only `SaslHandshake` was introduced in KIP-43. `SaslAuthenticate` came later in KIP-152. On Thu, Dec 7, 2023 at 3:18 PM Jason Gustafson wrote: > Hey Ismael, > > I'm considering if we can do something in this KIP for the SASL baggage > we've accumulated. Prior to t

Re: [VOTE] KIP-896: Remove old client protocol API versions in Kafka 4.0

2023-12-07 Thread Jason Gustafson
Hey Ismael, I'm considering if we can do something in this KIP for the SASL baggage we've accumulated. Prior to the existence of the `SaslHandshake` API, we supported the raw SASL protocol. The main gap was that it did not support negotiation of the SASL method. This was fixed in

Re: [DISCUSS] KIP-994: Minor Enhancements to ListTransactions and DescribeTransactions APIs

2023-11-28 Thread Jason Gustafson
Hey Raman, Thanks for the KIP! I think it makes sense. I agree that this becomes especially useful in the context of KIP-939 because transactions can last an indefinite amount of time, but it is useful even today. A large cluster may have a very large number of ongoing transactions at any time,

Re: [DISCUSS] KIP-996: Pre-Vote

2023-11-28 Thread Jason Gustafson
ncludes a bumped epoch when in > actuality there is no epoch bump. I've changed the VoteRequest json a bit > in the KIP, let me know what you think. > > On Mon, Nov 27, 2023 at 1:40 PM Jason Gustafson > > wrote: > > > Hey Alyssa, > > > > Thanks for the KI

Re: [DISCUSS] KIP-996: Pre-Vote

2023-11-27 Thread Jason Gustafson
Hey Alyssa, Thanks for the KIP! I have a couple questions: 1. I might have missed it in the KIP, but could you clarify what happens when a pre-vote fails (i.e. a majority of voters reject the potential candidacy)? The transition descriptions only mention what happens if the prospective leader

Re: [VOTE] KIP-1001; CurrentControllerId Metric

2023-11-20 Thread Jason Gustafson
The KIP makes sense. +1 On Mon, Nov 20, 2023 at 12:37 PM David Arthur wrote: > Thanks Colin, > > +1 from me > > -David > > On Tue, Nov 14, 2023 at 3:53 PM Colin McCabe wrote: > > > Hi all, > > > > I'd like to call a vote for KIP-1001: Add CurrentControllerId metric. > > > > Take a look here: >

Re: [VOTE] KIP-1000: List Client Metrics Configuration Resources

2023-11-20 Thread Jason Gustafson
+1 Thanks for the KIP! On Mon, Nov 20, 2023 at 9:31 AM Jun Rao wrote: > Hi, Andrew, > > Thanks for the KIP. +1 > > Jun > > On Thu, Nov 16, 2023 at 9:12 AM Andrew Schofield < > andrew_schofield_j...@outlook.com> wrote: > > > Hi Apoorv, > > Thanks for your vote. > > > > Initially, I put support

Re: [DISCUSS] KIP-1000: List Client Metrics Configuration Resources

2023-11-16 Thread Jason Gustafson
Hey Andrew, Thanks for the KIP. Just clarifying a couple small details. 1. I assume any broker can handle this API, so admin clients will choose a node randomly? 2. Does the controller need to support this API? If not, we can drop "controller" from "listeners." Thanks, Jason On Thu, Nov 16,

[jira] [Created] (KAFKA-15828) Protect clients from broker hostname reuse

2023-11-14 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-15828: --- Summary: Protect clients from broker hostname reuse Key: KAFKA-15828 URL: https://issues.apache.org/jira/browse/KAFKA-15828 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-15221) Potential race condition between requests from rebooted followers

2023-10-11 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-15221. - Fix Version/s: (was: 3.5.2) Resolution: Fixed > Potential race condit

Re: [VOTE] KIP-951: Leader discovery optimisations for the client

2023-10-03 Thread Jason Gustafson
+1 Thanks for the KIP On Tue, Oct 3, 2023 at 12:30 PM David Jacot wrote: > Thanks for the KIP. +1 from me as well. > > Best, > David > > Le mar. 3 oct. 2023 à 20:54, Jun Rao a écrit : > > > Hi, Mayank, > > > > Thanks for the detailed explanation in the KIP. +1 from me. > > > > Jun > > > > On

Re: [VOTE] KIP-714: Client metrics and observability

2023-09-13 Thread Jason Gustafson
Hey Andrew, +1 on the KIP. For many users of Kafka, it may not be fully understood how much of a challenge client monitoring is. With tens of clients in a cluster, it is already difficult to coordinate metrics collection. When there are thousands of clients, and when the cluster operator has no

[jira] [Resolved] (KAFKA-14694) RPCProducerIdManager should not wait for a new block

2023-06-22 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14694?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14694. - Fix Version/s: 3.6.0 Resolution: Fixed > RPCProducerIdManager should not w

Re: [VOTE] KIP-909: DNS Resolution Fallures Should Not Fail the Client

2023-04-25 Thread Jason Gustafson
+1 Thanks Philip! On Thu, Apr 13, 2023 at 7:49 AM Kirk True wrote: > +1 (non-binding) > > > On Apr 10, 2023, at 1:53 PM, Philip Nee wrote: > > > > Hey everyone! > > > > I'm starting a vote for KIP-909: DNS Resolution Fallures Should Not Fail > > the Client

Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

2023-04-21 Thread Jason Gustafson
Hey Colin, The KIP makes sense overall. Nice to clarify the contract between clients and the controllers. The use of `DirectToKRaftControllerQuorum` will help prevent misconfiguration. In fact, I wonder if we can return a fatal error instead of NOT_CONTROLLER so that the client would immediately

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-04-21 Thread Jason Gustafson
nd we can throw a fatal, BootstrapResolutionException (so > not connection exception anymore) afterward. > > I think that aligns with the goal of this KIP. > > P > > On Thu, Apr 20, 2023 at 9:23 AM Jason Gustafson > > wrote: > > > Hey Philip, > > > >

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-04-20 Thread Jason Gustafson
s > *bootstrap.auto.retry.ms > <http://bootstrap.auto.retry.ms> *instead, to indicate a configurable > period of automatic retry. What do you think? > > Cheers, > P > > On Wed, Apr 19, 2023 at 7:17 PM Jason Gustafson > > wrote: > > > Hey Phillip, >

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-04-19 Thread Jason Gustafson
Hey Phillip, The KIP looks good. 5 minutes seems like a reasonable tradeoff. I do wonder if it is necessary to treat bootstrap timeout as a fatal error though. It seems possible that the exception might be caught by handlers in existing applications which may not expect that the client needs to

Re: [VOTE] KIP-915: Txn and Group Coordinator Downgrade Foundation

2023-03-30 Thread Jason Gustafson
+1. Thanks Jeff. On Thu, Mar 30, 2023 at 7:41 AM Alexandre Dupriez < alexandre.dupr...@gmail.com> wrote: > Thanks for the KIP and clarifications, Jeff. > > +1 (non binding) > > Le jeu. 30 mars 2023 à 14:41, David Jacot > a écrit : > > > > Thanks for the KIP, Jeff. > > > > +1 (binding) > > > >

[jira] [Created] (KAFKA-14831) Illegal state errors should be fatal in transactional producer

2023-03-21 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14831: --- Summary: Illegal state errors should be fatal in transactional producer Key: KAFKA-14831 URL: https://issues.apache.org/jira/browse/KAFKA-14831 Project: Kafka

[jira] [Created] (KAFKA-14830) Illegal state error in transactional producer

2023-03-21 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14830: --- Summary: Illegal state error in transactional producer Key: KAFKA-14830 URL: https://issues.apache.org/jira/browse/KAFKA-14830 Project: Kafka Issue

Re: [VOTE] KIP-903: Replicas with stale broker epoch should not be allowed to join the ISR

2023-02-28 Thread Jason Gustafson
Thanks Calvin, +1 from me. On Mon, Feb 27, 2023 at 9:41 AM Calvin Liu wrote: > Hi Jason, > Updated the fields accordingly. Also, rename the BrokerState to > ReplicaState. > Thanks. > > On Wed, Feb 22, 2023 at 4:38 PM Jason Gustafson > > wrote: > > > Hi Calvin,

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-02-28 Thread Jason Gustafson
ht check, > as it seems exposing an API might be harder to push through. > > Thanks! > P > > On Tue, Feb 28, 2023 at 10:53 AM Jason Gustafson > > wrote: > > > One more random thought I had just as I pushed send. We're currently > > treating this problem somewha

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-02-28 Thread Jason Gustafson
a timeout which bounds the total time that the client should wait to connect to the bootstrap servers? Something like ` bootstrap.servers.connection.timeout.ms`. -Jason On Tue, Feb 28, 2023 at 10:44 AM Jason Gustafson wrote: > Hi Philip, > > An alternative is not to fail at all. Every othe

Re: [DISCUSS] KIP-909: Allow clients to rebootstrap DNS lookup failure

2023-02-28 Thread Jason Gustafson
Hi Philip, An alternative is not to fail at all. Every other network error is caught and handled internally in the client. We see this case as different because a DNS resolution error may imply misconfiguration. Could it also imply that the DNS server is unavailable? I'm not sure why that case

Re: [VOTE] KIP-903: Replicas with stale broker epoch should not be allowed to join the ISR

2023-02-22 Thread Jason Gustafson
Hi Calvin, The `BrokerState` struct I suggested would replace the `BrokerId` field in older versions. { "name": "ReplicaId", "type": "int32", "versions": "0-13", "entityType": "brokerId", "about": "The broker ID of the follower, of -1 if this request is from a consumer." }, {

Re: [VOTE] KIP-903: Replicas with stale broker epoch should not be allowed to join the ISR

2023-02-21 Thread Jason Gustafson
Hi Calvin, thanks for the KIP! A couple questions below: - Would it make sense to make the broker epoch a tagged field inside the Fetch request? It is a little annoying to increase the size of consumer fetch requests for a field that they will not use. Perhaps we could factor out broker id and

[jira] [Resolved] (KAFKA-14664) Raft idle ratio is inaccurate

2023-02-15 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14664?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14664. - Resolution: Fixed > Raft idle ratio is inaccur

[jira] [Resolved] (KAFKA-6793) Unnecessary warning log message

2023-02-13 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6793?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6793. Fix Version/s: 3.5.0 Resolution: Fixed We resolved this by changing the logging

[jira] [Resolved] (KAFKA-13972) Reassignment cancellation causes stray replicas

2023-02-06 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13972?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-13972. - Resolution: Fixed > Reassignment cancellation causes stray repli

[jira] [Created] (KAFKA-14672) Producer queue time does not reflect batches expired in the accumulator

2023-02-01 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14672: --- Summary: Producer queue time does not reflect batches expired in the accumulator Key: KAFKA-14672 URL: https://issues.apache.org/jira/browse/KAFKA-14672

[jira] [Created] (KAFKA-14664) Raft idle ratio is inaccurate

2023-01-30 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14664: --- Summary: Raft idle ratio is inaccurate Key: KAFKA-14664 URL: https://issues.apache.org/jira/browse/KAFKA-14664 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-14644) Process should stop after failure in raft IO thread

2023-01-25 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14644. - Fix Version/s: 3.5.0 Resolution: Fixed > Process should stop after fail

[jira] [Created] (KAFKA-14648) Do not fail clients if bootstrap servers is not immediately resolvable

2023-01-24 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14648: --- Summary: Do not fail clients if bootstrap servers is not immediately resolvable Key: KAFKA-14648 URL: https://issues.apache.org/jira/browse/KAFKA-14648 Project

[jira] [Created] (KAFKA-14644) Process should stop after failure in raft IO thread

2023-01-20 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14644: --- Summary: Process should stop after failure in raft IO thread Key: KAFKA-14644 URL: https://issues.apache.org/jira/browse/KAFKA-14644 Project: Kafka

Re: [VOTE] KIP-890: Transactions Server Side Defense

2023-01-17 Thread Jason Gustafson
+1. Thanks Justine! -Jason On Tue, Jan 10, 2023 at 3:46 PM Colt McNealy wrote: > (non-binding) +1. Thank you for the KIP, Justine! I've read it; it makes > sense to me and I am excited for the implementation. > > Colt McNealy > *Founder, LittleHorse.io* > > > On Tue, Jan 10, 2023 at 10:46 AM

[jira] [Resolved] (KAFKA-14612) Topic config records written to log even when topic creation fails

2023-01-12 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14612?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14612. - Fix Version/s: 3.4.0 Resolution: Fixed > Topic config records written to

[jira] [Created] (KAFKA-14618) Off by one error in generated snapshot IDs causes misaligned fetching

2023-01-12 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14618: --- Summary: Off by one error in generated snapshot IDs causes misaligned fetching Key: KAFKA-14618 URL: https://issues.apache.org/jira/browse/KAFKA-14618 Project

[jira] [Created] (KAFKA-14612) Topic config records written to log even when topic creation fails

2023-01-10 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14612: --- Summary: Topic config records written to log even when topic creation fails Key: KAFKA-14612 URL: https://issues.apache.org/jira/browse/KAFKA-14612 Project

Re: [DISCUSS] KIP-890 Server Side Defense

2023-01-06 Thread Jason Gustafson
produce request. > > > > Finally with respect to the authorizations, I think it makes sense to > skip > > topic authorizations, but I'm a bit confused by the "leader ID" field. > > Wouldn't we just want to flag the request as from a broker (does it > matter > &

Re: [DISCUSS] KIP-890 Server Side Defense

2023-01-05 Thread Jason Gustafson
Hi Justine, Thanks for the proposal. I was thinking about the implementation a little bit. In the current proposal, the behavior depends on whether we have an old or new client. For old clients, we send `DescribeTransactions` and verify the result and for new clients, we send

[jira] [Resolved] (KAFKA-14417) Producer doesn't handle REQUEST_TIMED_OUT for InitProducerIdRequest, treats as fatal error

2022-12-08 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14417?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14417. - Fix Version/s: 4.0.0 3.3.2 Resolution: Fixed > Produ

Re: [VOTE] KIP-866 ZooKeeper to KRaft Migration

2022-11-22 Thread Jason Gustafson
Thanks, +1 from me. I suspect we might be in for at least one surprise with the re-implemented controller RPCs, but I agree the alternative has risks as well. Best, Jason On Mon, Nov 14, 2022 at 12:00 PM Colin McCabe wrote: > On Fri, Nov 11, 2022, at 08:59, David Arthur wrote: > > Thanks,

[jira] [Created] (KAFKA-14397) Idempotent producer may bump epoch and reset sequence numbers prematurely

2022-11-16 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14397: --- Summary: Idempotent producer may bump epoch and reset sequence numbers prematurely Key: KAFKA-14397 URL: https://issues.apache.org/jira/browse/KAFKA-14397

[jira] [Resolved] (KAFKA-13964) kafka-configs.sh end with UnsupportedVersionException when describing TLS user with quotas

2022-11-09 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13964?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-13964. - Resolution: Duplicate Thanks for reporting the issue. This will be resolved by https

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

2022-10-28 Thread Jason Gustafson
+1 Thanks for 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. > > Best, > David > > On Wed, Oct 19, 2022 at 6:10 PM Jun Rao

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

2022-10-20 Thread Jason Gustafson
hat would > allow us to abort the transaction when the coordinator is reloaded. I > wanted to cover this separately from the current proposal. Do you > think that we should do this in this KIP? > > Cheers, > David > > On Wed, Oct 19, 2022 at 6:53 PM Jason Gustafson > wrot

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

2022-10-19 Thread Jason Gustafson
Hi David, A few questions below: 1. In regard to this comment: > Every member is uniquely identified by a UUID. This is called the Member ID. This UUID is generated on the server side and given to the member when it joins the group. It is used in all the communication with the group coordinator

[jira] [Resolved] (KAFKA-14316) NoSuchElementException in feature control iterator

2022-10-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14316?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14316. - Fix Version/s: 3.4.0 3.3.2 Resolution: Fixed

[jira] [Created] (KAFKA-14319) Storage tool format command does not work with old metadata versions

2022-10-18 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14319: --- Summary: Storage tool format command does not work with old metadata versions Key: KAFKA-14319 URL: https://issues.apache.org/jira/browse/KAFKA-14319 Project

[jira] [Created] (KAFKA-14316) NoSuchElementException in feature control iterator

2022-10-18 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14316: --- Summary: NoSuchElementException in feature control iterator Key: KAFKA-14316 URL: https://issues.apache.org/jira/browse/KAFKA-14316 Project: Kafka

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

2022-10-13 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14296?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14296. - Fix Version/s: 3.4.0 3.3.2 Resolution: Fixed > Partit

[jira] [Resolved] (KAFKA-14292) KRaft broker controlled shutdown can be delayed indefinitely

2022-10-13 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14292?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14292. - Fix Version/s: 3.4.0 3.3.2 Resolution: Fixed > KRaft bro

[jira] [Created] (KAFKA-14292) KRaft broker controlled shutdown can be delayed indefinitely

2022-10-11 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14292: --- Summary: KRaft broker controlled shutdown can be delayed indefinitely Key: KAFKA-14292 URL: https://issues.apache.org/jira/browse/KAFKA-14292 Project: Kafka

[ANNOUNCE] New committer: Deng Ziming

2022-10-10 Thread Jason Gustafson
Hi All The PMC for Apache Kafka has invited Deng Ziming to become a committer, and we are excited to announce that he has accepted! Ziming has been contributing to Kafka for about three years. He has authored more than 100 patches and helped to review nearly as many. In particular, he made

[jira] [Resolved] (KAFKA-14247) Implement EventHandler interface and DefaultEventHandler

2022-10-03 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14247. - Resolution: Fixed > Implement EventHandler interface and DefaultEventHand

[jira] [Resolved] (KAFKA-14236) ListGroups request produces too much Denied logs in authorizer

2022-09-21 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14236. - Fix Version/s: 3.4.0 Resolution: Fixed > ListGroups request produces too m

[jira] [Resolved] (KAFKA-14240) Ensure kraft metadata log dir is initialized with valid snapshot state

2022-09-19 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14240?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14240. - Resolution: Fixed > Ensure kraft metadata log dir is initialized with valid snaps

[jira] [Resolved] (KAFKA-14238) KRaft replicas can delete segments not included in a snapshot

2022-09-17 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14238?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14238. - Resolution: Fixed > KRaft replicas can delete segments not included in a snaps

[jira] [Created] (KAFKA-14240) Ensure kraft metadata log dir is initialized with valid snapshot state

2022-09-16 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14240: --- Summary: Ensure kraft metadata log dir is initialized with valid snapshot state Key: KAFKA-14240 URL: https://issues.apache.org/jira/browse/KAFKA-14240 Project

[jira] [Resolved] (KAFKA-14215) KRaft forwarded requests have no quota enforcement

2022-09-12 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14215?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14215. - Resolution: Fixed > KRaft forwarded requests have no quota enforcem

[jira] [Created] (KAFKA-14224) Consumer should auto-commit prior to cooperative partition revocation

2022-09-12 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14224: --- Summary: Consumer should auto-commit prior to cooperative partition revocation Key: KAFKA-14224 URL: https://issues.apache.org/jira/browse/KAFKA-14224 Project

[jira] [Created] (KAFKA-14215) KRaft forwarded requests have no quota enforcement

2022-09-09 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14215: --- Summary: KRaft forwarded requests have no quota enforcement Key: KAFKA-14215 URL: https://issues.apache.org/jira/browse/KAFKA-14215 Project: Kafka

[jira] [Created] (KAFKA-14201) Consumer should not send group instance ID if committing with empty member ID

2022-09-02 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14201: --- Summary: Consumer should not send group instance ID if committing with empty member ID Key: KAFKA-14201 URL: https://issues.apache.org/jira/browse/KAFKA-14201

[jira] [Resolved] (KAFKA-14177) Correctly support older kraft versions without FeatureLevelRecord

2022-08-25 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14177?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14177. - Resolution: Fixed > Correctly support older kraft versions without FeatureLevelRec

[jira] [Created] (KAFKA-14183) Kraft bootstrap metadata file should use snapshot header/footer

2022-08-25 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14183: --- Summary: Kraft bootstrap metadata file should use snapshot header/footer Key: KAFKA-14183 URL: https://issues.apache.org/jira/browse/KAFKA-14183 Project: Kafka

[jira] [Resolved] (KAFKA-13166) EOFException when Controller handles unknown API

2022-08-22 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-13166. - Resolution: Fixed > EOFException when Controller handles unknown

[jira] [Resolved] (KAFKA-13914) Implement kafka-metadata-quorum.sh

2022-08-20 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13914?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-13914. - Fix Version/s: 3.3.0 Resolution: Fixed > Implement kafka-metadata-quorum

[jira] [Resolved] (KAFKA-14167) Unexpected UNKNOWN_SERVER_ERROR raised from kraft controller

2022-08-17 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14167?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14167. - Resolution: Fixed > Unexpected UNKNOWN_SERVER_ERROR raised from kraft control

[jira] [Resolved] (KAFKA-13940) DescribeQuorum returns INVALID_REQUEST if not handled by leader

2022-08-17 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13940?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-13940. - Resolution: Fixed > DescribeQuorum returns INVALID_REQUEST if not handled by lea

[jira] [Created] (KAFKA-14167) Unexpected UNKNOWN_SERVER_ERROR raised from kraft controller

2022-08-15 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14167: --- Summary: Unexpected UNKNOWN_SERVER_ERROR raised from kraft controller Key: KAFKA-14167 URL: https://issues.apache.org/jira/browse/KAFKA-14167 Project: Kafka

[jira] [Resolved] (KAFKA-14154) Persistent URP after controller soft failure

2022-08-15 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14154. - Resolution: Fixed > Persistent URP after controller soft fail

[jira] [Created] (KAFKA-14166) Consistent toString implementations for byte arrays in generated messages

2022-08-15 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14166: --- Summary: Consistent toString implementations for byte arrays in generated messages Key: KAFKA-14166 URL: https://issues.apache.org/jira/browse/KAFKA-14166

[jira] [Resolved] (KAFKA-13986) DescribeQuorum does not return the observers (brokers) for the Metadata log

2022-08-11 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-13986. - Fix Version/s: 3.3.0 Resolution: Fixed > DescribeQuorum does not ret

[jira] [Resolved] (KAFKA-14163) Build failing in streams-scala:compileScala due to zinc compiler cache

2022-08-11 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14163?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14163. - Resolution: Workaround > Build failing in streams-scala:compileScala due to z

[jira] [Created] (KAFKA-14163) Build failing in scala:compileScala due to zinc compiler cache

2022-08-11 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14163: --- Summary: Build failing in scala:compileScala due to zinc compiler cache Key: KAFKA-14163 URL: https://issues.apache.org/jira/browse/KAFKA-14163 Project: Kafka

[jira] [Created] (KAFKA-14154) Persistent URP after controller soft failure

2022-08-09 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14154: --- Summary: Persistent URP after controller soft failure Key: KAFKA-14154 URL: https://issues.apache.org/jira/browse/KAFKA-14154 Project: Kafka Issue

[jira] [Created] (KAFKA-14152) Add logic to fence kraft brokers which have fallen behind in replication

2022-08-09 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14152: --- Summary: Add logic to fence kraft brokers which have fallen behind in replication Key: KAFKA-14152 URL: https://issues.apache.org/jira/browse/KAFKA-14152

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

2022-08-09 Thread Jason Gustafson
Thanks Justine, +1 from me. On Tue, Aug 9, 2022 at 1:12 AM Sagar wrote: > Thanks for the KIP. > > +1(non-binding) > > Sagar. > > On Tue, Aug 9, 2022 at 1:13 PM David Jacot > wrote: > > > Thanks for the KIP, Justine. The proposal makes sense to me. I am +1 > > (binding). > > > > Cheers, > >

[jira] [Resolved] (KAFKA-14144) AlterPartition is not idempotent when requests time out

2022-08-09 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14144. - Resolution: Fixed > AlterPartition is not idempotent when requests time

[jira] [Resolved] (KAFKA-14104) Perform CRC validation on KRaft Batch Records and Snapshots

2022-08-08 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14104?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14104. - Resolution: Fixed > Perform CRC validation on KRaft Batch Records and Snapsh

Re: [DISCUSS] KIP-858: Handle JBOD broker disk failure in KRaft

2022-08-08 Thread Jason Gustafson
Hi Igor, Thanks for the KIP. It looks like it's on a good track. I have a few suggestions to throw into the mix: 1. (nit): Instead of "storage id," maybe we could call it "directory id"? It seems a little clear since each log dir gets a unique id. 2. Rather than introducing a new RPC to

[jira] [Created] (KAFKA-14139) Replaced disk can lead to loss of committed data even with non-empty ISR

2022-08-03 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14139: --- Summary: Replaced disk can lead to loss of committed data even with non-empty ISR Key: KAFKA-14139 URL: https://issues.apache.org/jira/browse/KAFKA-14139

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

2022-08-02 Thread Jason Gustafson
t; > value that transactional.id.expiration.ms has? > > > > Let me know your thoughts. > > Thanks, > > Justine > > > > On Fri, Jul 29, 2022 at 10:54 AM Ismael Juma wrote: > > > > > +1 for having 1 day as the default and for including this change i

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

2022-07-28 Thread Jason Gustafson
Yeah, good idea. I'm happy to submit the specs I wrote for normal kafka replication. It will make them more accessible and I have long been looking for help reviewing them. Hopefully it will also provide a better chance to keep them in sync with the codebase as we update protocols. -Jason On

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

2022-07-27 Thread Jason Gustafson
ory -- especially since there is the chance for > producer > > > IDs to expire silently. > > > I do think that 1 day is fairly reasonable. If I don't hear any > > conflicting > > > opinions I can go ahead and update the default. > > > > > > Just

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

2022-07-26 Thread Jason Gustafson
Hi Justine, Thanks for the KIP. Although I hate seeing new configurations, I think this is a good change. Combining these timeout behaviors into a single configuration was definitely a mistake, but we didn't anticipate the problem with the producer id cache. I do wonder if we can make the default

[jira] [Resolved] (KAFKA-14078) Replica fetches to follower should return NOT_LEADER error

2022-07-25 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-14078. - Resolution: Fixed > Replica fetches to follower should return NOT_LEADER er

[jira] [Created] (KAFKA-14078) Replica fetches to follower should return NOT_LEADER error

2022-07-14 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14078: --- Summary: Replica fetches to follower should return NOT_LEADER error Key: KAFKA-14078 URL: https://issues.apache.org/jira/browse/KAFKA-14078 Project: Kafka

Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-07-14 Thread Jason Gustafson
Hey Jose, Thanks for volunteering to manage the release! KIP-833 is currently slotted for 3.3. We've been getting some help from Jack Vanlighty to validate the raft implementation in TLA+ and with frameworks like Jepsen. The specification is written here if anyone is interested:

[jira] [Created] (KAFKA-14077) KRaft should support recovery from failed disk

2022-07-14 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-14077: --- Summary: KRaft should support recovery from failed disk Key: KAFKA-14077 URL: https://issues.apache.org/jira/browse/KAFKA-14077 Project: Kafka Issue

  1   2   3   4   5   6   7   8   9   10   >