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

2023-04-19 Thread Colin McCabe
Hi all, I wrote a short KIP about allowing AdminClient to talk directly with the KRaft controller quorum. Check it out here: https://cwiki.apache.org/confluence/x/Owo0Dw best, Colin

[jira] [Resolved] (KAFKA-14775) Support SCRAM for broker to controller authentication

2023-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14775?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14775. -- Fix Version/s: 3.5.0 Assignee: Colin McCabe (was: Proven Provenzano

Re: [VOTE] KIP-902: Upgrade Zookeeper to 3.8.1

2023-04-14 Thread Colin McCabe
On Sun, Apr 9, 2023, at 19:17, Ismael Juma wrote: > > On Sun, Apr 9, 2023 at 4:53 PM Colin McCabe wrote: > >> We are going to deprecate ZK mode soon. So if this is indeed a requirement >> (no deprecated software in prod), perhaps those users will have to move to >> KR

[jira] [Resolved] (KAFKA-14894) MetadataLoader must call finishSnapshot after loading a snapshot

2023-04-12 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14894?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14894. -- Fix Version/s: 3.5.0 Reviewer: David Arthur Resolution: Fixed

[jira] [Created] (KAFKA-14894) MetadataLoader must call finishSnapshot after loading a snapshot

2023-04-11 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14894: Summary: MetadataLoader must call finishSnapshot after loading a snapshot Key: KAFKA-14894 URL: https://issues.apache.org/jira/browse/KAFKA-14894 Project: Kafka

Re: [VOTE] KIP-902: Upgrade Zookeeper to 3.8.1

2023-04-09 Thread Colin McCabe
to use Kafka. We are going to deprecate ZK mode soon. So if this is indeed a requirement (no deprecated software in prod), perhaps those users will have to move to KRaft mode. (Independently of what we decide here) best, Colin > Please, let me know your thoughts on the matter! > >

[jira] [Created] (KAFKA-14857) Fix some MetadataLoader bugs

2023-03-27 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14857: Summary: Fix some MetadataLoader bugs Key: KAFKA-14857 URL: https://issues.apache.org/jira/browse/KAFKA-14857 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-14436) Initialize KRaft with arbitrary epoch

2023-03-24 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14436. -- Fix Version/s: 3.4.0 Resolution: Won't Fix > Initialize KRaft with arbitra

[jira] [Created] (KAFKA-14846) Fix overly large record batches in ZkMigrationClient

2023-03-24 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14846: Summary: Fix overly large record batches in ZkMigrationClient Key: KAFKA-14846 URL: https://issues.apache.org/jira/browse/KAFKA-14846 Project: Kafka Issue

[jira] [Resolved] (KAFKA-14493) Zk to KRaft migration state machine in KRaft controller

2023-03-24 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14493?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14493. -- Fix Version/s: 3.4.0 Resolution: Fixed > Zk to KRaft migration state machine in KR

[jira] [Resolved] (KAFKA-14458) RPC Handler to ZkBrokers from KRaft Controller

2023-03-24 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14458. -- Fix Version/s: 3.4.0 Resolution: Fixed > RPC Handler to ZkBrokers from KR

[jira] [Resolved] (KAFKA-14446) API forwarding support in ZkBrokers

2023-03-24 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14446?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14446. -- Fix Version/s: 3.4.0 Resolution: Fixed > API forwarding support in ZkBrok

[jira] [Resolved] (KAFKA-14447) Controlled shutdown for ZK brokers during migration

2023-03-24 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14447?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14447. -- Fix Version/s: 3.4.0 (was: 3.4.1) Resolution: Fixed

[jira] [Created] (KAFKA-14835) Create ControllerServerMetricsPublisher

2023-03-22 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14835: Summary: Create ControllerServerMetricsPublisher Key: KAFKA-14835 URL: https://issues.apache.org/jira/browse/KAFKA-14835 Project: Kafka Issue Type

Re: [VOTE] KIP-902: Upgrade Zookeeper to 3.8.1

2023-03-09 Thread Colin McCabe
Hi, I'm struggling a bit with this KIP, because dropping support for rolling upgrades from old Kafka versions doesn't seem like something we should do in a minor release. But on the other hand, the next Kafka release won't have ZK at all. Maybe we should punt on this until and unless there is a

Re: [ANNOUNCE] New committer: Lucas Bradstreet

2023-02-21 Thread Colin McCabe
Congratulations, Lucas! best, Colin On Thu, Feb 16, 2023, at 13:54, Jun Rao wrote: > Hi, Everyone, > > The PMC of Apache Kafka is pleased to announce a new Kafka committer Lucas > Bradstreet. > > Lucas has been a long time Kafka contributor since Oct. 2018. He has been > extremely valuable for Ka

Re: [VOTE] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM for Kafka Brokers

2023-02-21 Thread Colin McCabe
Thanks, Proven. +1 (binding) best, Colin On Thu, Feb 16, 2023, at 07:23, Proven Provenzano wrote: > Hi, > > I'd like to start the vote on KIP-900: KRaft kafka-storage.sh API additions > to support SCRAM for Kafka Brokers. > > The KIP is here: KIP-900 >

Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-01-27 Thread Colin McCabe
at route, then maybe it makes sense to do SCRAM with a similar paradigm. So something like: --entity-type users --entity-name alice --add-config 'SCRAM-SHA-512=[password=alice-secret]' I do think this is doable with argparse4j. What do you think? best, Colin > > > On Tue, Ja

[jira] [Created] (KAFKA-14658) Do not open broker ports until we are ready to accept traffic

2023-01-27 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14658: Summary: Do not open broker ports until we are ready to accept traffic Key: KAFKA-14658 URL: https://issues.apache.org/jira/browse/KAFKA-14658 Project: Kafka

Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-01-23 Thread Colin McCabe
at this is how kafka-configs.sh works, and we don't want two different incompatible ways of specifying SCRAM users on the command line. > I think adding the Argparse4j support for reading the arguments from a file > is a must. Yeah, agreed. best, Colin > > --Proven > > >

Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-01-19 Thread Colin McCabe
Hi Proven, Thanks for putting this together. We always intended to have a way to bootstrap into using an all-SCRAM cluster, from scratch. I have two big comments here. First, I think we need a better interface than raw records. And second, I'm not sure that kafka-storage.sh is the right place

Re: [ANNOUNCE] New committer: Justine Olshan

2023-01-19 Thread Colin McCabe
Congratuations, Justine! Happy to hear it. I guess my message is a bit late too :) best, Colin On Tue, Jan 17, 2023, at 17:23, Kowshik Prakasam wrote: > Congrats, Justine! > > > Cheers, > Kowshik > > On Tue, Jan 17, 2023, 4:53 PM Guozhang Wang > wrote: > >> Congratulations, Justine (I'm also la

Re: [ANNOUNCE] New committer: Stanislav Kozlovski

2023-01-19 Thread Colin McCabe
Congratulations, Stan! Well deserved. best, Colin On Tue, Jan 17, 2023, at 07:50, Jun Rao wrote: > Hi, Everyone, > > The PMC of Apache Kafka is pleased to announce a new Kafka committer > Stanislav Kozlovski. > > Stan has been contributing to Apache Kafka since June 2018. He made various > contri

[jira] [Created] (KAFKA-14622) Create a junit test which would have caught KAFKA-14618

2023-01-13 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14622: Summary: Create a junit test which would have caught KAFKA-14618 Key: KAFKA-14622 URL: https://issues.apache.org/jira/browse/KAFKA-14622 Project: Kafka

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

2023-01-13 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14618?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14618. -- Resolution: Fixed > Off by one error in generated snapshot IDs causes misaligned fetch

Re: [DISCUSS] KIP-894: Use incrementalAlterConfigs API for syncing topic configurations

2023-01-13 Thread Colin McCabe
On Sat, Jan 7, 2023, at 11:39, Ismael Juma wrote: > Hi, > > Thanks for the KIP. Have we considered using incremental alter configs by > default and fallback to the legacy one if the former is unavailable? > > The config could have 3 possible values: requested, required, never. The > default would b

Re: [VOTE] 3.4.0 RC0

2023-01-13 Thread Colin McCabe
Thanks for putting this together, David! Looks like we found a 3.4 blocker, unfortunately: KAFKA-14618: Off by one error in generated snapshot IDs causes misaligned fetching. So we'll need a new RC. best, Colin On Wed, Jan 11, 2023, at 14:57, David Arthur wrote: > Hello Kafka users, developer

[jira] [Created] (KAFKA-14601) Improve exception handling in KafkaEventQueue

2023-01-06 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14601: Summary: Improve exception handling in KafkaEventQueue Key: KAFKA-14601 URL: https://issues.apache.org/jira/browse/KAFKA-14601 Project: Kafka Issue Type

[jira] [Created] (KAFKA-14538) Implement metadata transactions at arbitrary locations in the log

2022-12-20 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14538: Summary: Implement metadata transactions at arbitrary locations in the log Key: KAFKA-14538 URL: https://issues.apache.org/jira/browse/KAFKA-14538 Project: Kafka

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

2022-12-05 Thread Colin McCabe
+1 (binding) best, Colin On Mon, Dec 5, 2022, at 10:03, David Jacot wrote: > Hi all, > > As this KIP-893 is trivial and non-controversial, I would like to > start the vote on it. The KIP is here: > https://cwiki.apache.org/confluence/x/YJIODg > > Thanks, > David

Re: Supported Kafka/Zookeeper Version with ELK 8.4.3

2022-12-05 Thread Colin McCabe
Hi, Sorry, we do not develop ELK. In fact, I'm not sure what that acronym refers to. I would suggest checking in with support for that product / project, since it is not part of Apache Kafka. best, Colin On Fri, Oct 28, 2022, at 06:23, Kumar, Sudip wrote: > Hi Team, > > We are still waiting

Re: Ci stability

2022-12-05 Thread Colin McCabe
FYI, there was a memory leak that affected some of the tests which was fixed recently, so hopefully stability will improve a bit. See KAFKA-14433 for details. best, Colin On Thu, Nov 24, 2022, at 12:48, John Roesler wrote: > Hi Dan, > > I’m not sure if there’s a consistently used tag, but I’ve

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

2022-12-05 Thread Colin McCabe
Hi David, Thanks for posting this. I think it will be pretty useful. +1 for the idea best, Colin On Thu, Dec 1, 2022, at 08:57, David Jacot wrote: > 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 us

[jira] [Created] (KAFKA-14433) Clear all yammer metrics when test harnesses clean up

2022-12-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14433: Summary: Clear all yammer metrics when test harnesses clean up Key: KAFKA-14433 URL: https://issues.apache.org/jira/browse/KAFKA-14433 Project: Kafka Issue

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

2022-11-14 Thread Colin McCabe
es it LGTM to me. +1 (binding) Colin > Thanks! > David > > On Wed, Nov 9, 2022 at 5:46 PM Colin McCabe wrote: >> >> Hi David, >> >> Thanks for the response. Replies inline. >> >> On Wed, Nov 9, 2022, at 08:17, David Arthur wrote: >> >

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

2022-11-09 Thread Colin McCabe
C containing their IBP and readiness state. I guess the main negative thing is that it would be more network traffic (but probably not significantly more.) The main positive thing is that we'd never start an upgrade without first verifying the quorum configuration on the ZK-based brok

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

2022-11-08 Thread Colin McCabe
Hi David, Looks great. Some questions: I agree with Jun that it would be good to rename metadata.migration.enable to something more zk-specific. Maybe zk.metadata.migration.enable ? SImilarly, for MigrationRecord: can we rename this to ZkMigrationStateRecord? Then change MigrationState -> ZkMi

[jira] [Created] (KAFKA-14370) Properly close ImageWriter objects

2022-11-08 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14370: Summary: Properly close ImageWriter objects Key: KAFKA-14370 URL: https://issues.apache.org/jira/browse/KAFKA-14370 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-14351) Implement controller mutation quotas in KRaft

2022-11-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14351: Summary: Implement controller mutation quotas in KRaft Key: KAFKA-14351 URL: https://issues.apache.org/jira/browse/KAFKA-14351 Project: Kafka Issue Type

[jira] [Created] (KAFKA-14350) Support dynamically reconfiguring KRaft controller listeners

2022-11-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14350: Summary: Support dynamically reconfiguring KRaft controller listeners Key: KAFKA-14350 URL: https://issues.apache.org/jira/browse/KAFKA-14350 Project: Kafka

[jira] [Created] (KAFKA-14349) Support dynamically resizing the KRaft controller's thread pools

2022-11-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14349: Summary: Support dynamically resizing the KRaft controller's thread pools Key: KAFKA-14349 URL: https://issues.apache.org/jira/browse/KAFKA-14349 Project:

[jira] [Created] (KAFKA-14348) Consider renaming MetadataBatchProcessingTimeUs to MetadataDeltaProcessingTimeUs

2022-11-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14348: Summary: Consider renaming MetadataBatchProcessingTimeUs to MetadataDeltaProcessingTimeUs Key: KAFKA-14348 URL: https://issues.apache.org/jira/browse/KAFKA-14348

[jira] [Created] (KAFKA-14327) Unify KRaft snapshot generation between broker and controller

2022-10-20 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14327: Summary: Unify KRaft snapshot generation between broker and controller Key: KAFKA-14327 URL: https://issues.apache.org/jira/browse/KAFKA-14327 Project: Kafka

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

2022-10-17 Thread Colin McCabe
e immediately here and skip the complexity of implementing "deleting state." Topic IDs will exist in IBP 3.4, in both ZK and KRaft mode, so it should be OK, right? best, Colin > > Thanks, > > Jun > > On Fri, Oct 14, 2022 at 9:29 AM Colin McCabe wrote: > >> On

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

2022-10-14 Thread Colin McCabe
rallel. I do think we will reuse a bunch of the serialization / deserialization code for znodes and possibly the code for communicating with ZK. best, Colin > > 14. Good point and make sense. > > Thanks, > > Jun > > > > > On Wed, Oct 12, 2022 at 3:27 PM Colin McCabe

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

2022-10-14 Thread Colin McCabe
Thanks, José! +1 (binding) Colin On Fri, Oct 14, 2022, at 07:54, David Arthur wrote: > Thanks for the KIP, José! > > +1 (binding) > > -David > > On Fri, Oct 14, 2022 at 2:48 AM David Jacot wrote: >> >> +1 (binding) >> >> Thanks for the KIP! >> >> Le ven. 14 oct. 2022 à 05:47, deng ziming a >>

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

2022-10-12 Thread Colin McCabe
On Wed, Oct 5, 2022, at 10:06, Mickael Maison wrote: > Hi David, > > Thanks for starting this important KIP. > > I've just taken a quick look so far but I've got a couple of initial > questions: > > 1) What happens if a non KRaft compatible broker (or with > kafka.metadata.migration.enable set to

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

2022-10-12 Thread Colin McCabe
Thanks for putting this together, David. A few questions (maybe some of them overlap with Jun's?) 1. Rather than having MigrationCheckRequest, can we just add a new JSON field to the broker registration like "kraftReady": true? After all, we are already going to have to read the broker registra

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

2022-10-12 Thread Colin McCabe
Hi Jun, Thanks for taking a look. I can answer some questions here because I collaborated on this a bit, and David is on vacation for a few days. On Wed, Oct 12, 2022, at 14:41, Jun Rao wrote: > Hi, David, > > Thanks for the KIP. A few comments below. > > 10. It's still not very clear to me how

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

2022-10-12 Thread Colin McCabe
Thanks for the KIP, José. Given that we already have metadata.log.max.record.bytes.between.snapshots, we need to define how the two properties interact. I would expect that the time-based property would take effect only if the bytes-based property did not trigger. In other words, if you are reg

[jira] [Created] (KAFKA-14290) Fix bugs that could block KRaft controlled shutdown indefinitely

2022-10-11 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14290: Summary: Fix bugs that could block KRaft controlled shutdown indefinitely Key: KAFKA-14290 URL: https://issues.apache.org/jira/browse/KAFKA-14290 Project: Kafka

[jira] [Created] (KAFKA-14265) Prefix ACLs may shadow other prefix ACLs

2022-09-28 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14265: Summary: Prefix ACLs may shadow other prefix ACLs Key: KAFKA-14265 URL: https://issues.apache.org/jira/browse/KAFKA-14265 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-14259) BrokerRegistration#toString throws an exception, terminating metadata replay

2022-09-23 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14259: Summary: BrokerRegistration#toString throws an exception, terminating metadata replay Key: KAFKA-14259 URL: https://issues.apache.org/jira/browse/KAFKA-14259 Project

[jira] [Created] (KAFKA-14258) Add ducktape or junit test verifying that brokers can reload snapshots after startup

2022-09-23 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14258: Summary: Add ducktape or junit test verifying that brokers can reload snapshots after startup Key: KAFKA-14258 URL: https://issues.apache.org/jira/browse/KAFKA-14258

[jira] [Created] (KAFKA-14243) Disable unsafe downgrade in 3.3

2022-09-19 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14243: Summary: Disable unsafe downgrade in 3.3 Key: KAFKA-14243 URL: https://issues.apache.org/jira/browse/KAFKA-14243 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-14216) Remove ZK reference from org.apache.kafka.server.quota.ClientQuotaCallback javadoc

2022-09-12 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14216?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14216. -- Fix Version/s: 3.3 Reviewer: Luke Chen Resolution: Fixed > Remove

[jira] [Resolved] (KAFKA-14217) app-reset-tool.html should remove reference to --zookeeper flag that no longer exists

2022-09-12 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14217?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14217. -- Fix Version/s: 3.3 Reviewer: Luke Chen Resolution: Fixed > app-re

Re: Problem with Kafka KRaft 3.1.X

2022-09-11 Thread Colin McCabe
to report for my talk now, thanks > for the info. > > Regards, Paul > > On Sat, 10 Sept 2022 at 02:43, Colin McCabe wrote: > >> Hi Paul, >> >> As Keith wrote, it does sound like you are hitting a separate Linux limit >> like the max mmap count. >> >

[jira] [Created] (KAFKA-14217) app-reset-tool.html should remove reference to --zookeeper flag that no longer exists

2022-09-09 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14217: Summary: app-reset-tool.html should remove reference to --zookeeper flag that no longer exists Key: KAFKA-14217 URL: https://issues.apache.org/jira/browse/KAFKA-14217

[jira] [Created] (KAFKA-14216) Remove ZK reference from org.apache.kafka.server.quota.ClientQuotaCallback javadoc

2022-09-09 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14216: Summary: Remove ZK reference from org.apache.kafka.server.quota.ClientQuotaCallback javadoc Key: KAFKA-14216 URL: https://issues.apache.org/jira/browse/KAFKA-14216

Re: [DISCUSS] KIP-865 Metadata Transactions

2022-09-09 Thread Colin McCabe
Also, it looks like someone already claimed KIP-865, so I'd suggest grabbing a new number. :) Colin On Fri, Sep 9, 2022, at 09:38, Colin McCabe wrote: > Thanks for this KIP, David! > > In the "motivation" section, it might help to give a concrete example > of an ope

Re: Problem with Kafka KRaft 3.1.X

2022-09-09 Thread Colin McCabe
Hi Paul, As Keith wrote, it does sound like you are hitting a separate Linux limit like the max mmap count. I'm curious how many partitions you can create if you change that config! best, Colin On Tue, Sep 6, 2022, at 14:02, Keith Paulson wrote: > I've had similar errors cause by mmap counts;

Re: [DISCUSS] KIP-865 Metadata Transactions

2022-09-09 Thread Colin McCabe
Thanks for this KIP, David! In the "motivation" section, it might help to give a concrete example of an operation we want to be atomic. My favorite one is probably CreateTopics since it's easy to see that we want to create all of a topic or none of it, and a topic could be a potentially unbound

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

2022-09-07 Thread Colin McCabe
Hi Ismael, I think this issue comes up when people write software that automatically creates partition reassignments to balance the cluster. Cruise Control is one example; Confluent also has some software that does this. If there is already a reassignment that is going on for some partition and

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

2022-09-07 Thread Colin McCabe
+1 (binding). One nitpick: when the user sets AllowReplicationFactorChange = false, the exception the user gets back from AdminClient should mention that this was the problem. If the exception just says "The broker does not support ALTER_PARTITION_REASSIGNMENTS with version in range [1, 1]. Th

[jira] [Resolved] (KAFKA-14200) kafka-features.sh must exit with non-zero error code on error

2022-09-07 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14200?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14200. -- Reviewer: David Arthur Resolution: Fixed > kafka-features.sh must exit with non-z

[jira] [Resolved] (KAFKA-14197) Kraft broker fails to startup after topic creation failure

2022-09-06 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14197?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14197. -- Resolution: Duplicate > Kraft broker fails to startup after topic creation fail

[jira] [Created] (KAFKA-14204) QuorumController must correctly handle overly large batches

2022-09-06 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14204: Summary: QuorumController must correctly handle overly large batches Key: KAFKA-14204 URL: https://issues.apache.org/jira/browse/KAFKA-14204 Project: Kafka

[jira] [Created] (KAFKA-14200) kafka-features.sh must exit with non-zero error code on error

2022-09-02 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14200: Summary: kafka-features.sh must exit with non-zero error code on error Key: KAFKA-14200 URL: https://issues.apache.org/jira/browse/KAFKA-14200 Project: Kafka

[jira] [Resolved] (KAFKA-14187) kafka-features.sh: add support for --metadata

2022-08-30 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14187. -- Resolution: Fixed > kafka-features.sh: add support for --metad

Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-08-29 Thread Colin McCabe
Hi José, Thanks for creating the first RC. I found an issue where kafka-feature.sh needs some work for KRaft. So, it looks like we will have to sink this RC. I opened a blocker JIRA, KAFKA-14187, and attached a PR. This should not block testing of other parts of the release, so hopefully we w

Re: ARM/PowerPC builds

2022-08-29 Thread Colin McCabe
i, > > I am looking into the failures on PowerPC. > > I will share more details once I have something concrete & hopefully we > would be able to enable it again soon. > > -Amit Baheti > > -Original Message- > From: Colin McCabe > Sent: 04 August 20

Re: ARM/PowerPC builds

2022-08-29 Thread Colin McCabe
: Oskari Saarenmaa & Hannu Valtonen > m: +491603708037 > w: aiven.io e: matthew.dedetr...@aiven.io > On 4. Aug 2022, 19:10 +0200, Colin McCabe , wrote: >> Hi Matthew, >> >> Can you open a JIRA for the test failures you have seen on M1? >> >> By the way,

[jira] [Created] (KAFKA-14187) Fix kafka-features.sh command

2022-08-29 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14187: Summary: Fix kafka-features.sh command Key: KAFKA-14187 URL: https://issues.apache.org/jira/browse/KAFKA-14187 Project: Kafka Issue Type: Bug Affects

[jira] [Resolved] (KAFKA-14178) NoOpRecord incorrectly causes high controller queue time metric

2022-08-24 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14178?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-14178. -- Resolution: Fixed > NoOpRecord incorrectly causes high controller queue time met

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

2022-08-23 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14177: Summary: Correctly support older kraft versions without FeatureLevelRecord Key: KAFKA-14177 URL: https://issues.apache.org/jira/browse/KAFKA-14177 Project: Kafka

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

2022-08-05 Thread Colin McCabe
Hi Stanislav, Thanks for the KIP. I think this is a nice solution to the problem of not wanting to change the replication factor during reassignments. Just from a writing point of view, it would be nice for the first paragraph to be a bit more explicit about this goal. Maybe lead with "Many tim

[jira] [Resolved] (KAFKA-13313) In KRaft mode, CreateTopic should return the topic configs in the response

2022-08-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13313?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13313. -- Resolution: Duplicate > In KRaft mode, CreateTopic should return the topic configs in

Re: ARM/PowerPC builds

2022-08-04 Thread Colin McCabe
t? Should we >> start a JIRA with Apache Infra to root cause? >> >> — >> Divij Vaidya >> >> >> >> On Thu, Aug 4, 2022 at 12:42 AM Colin McCabe wrote: >> >> > Just a quick note. Today we committed >> > https://github.com/apache/kafka/pull

Re: [VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-03 Thread Colin McCabe
t >> listener-batch-load-error-count => metadata-apply-error-count > > Yeah, this makes sense. I have made the changes in naming you suggested > and updated the KIP. > > - Niket > > >> On Aug 3, 2022, at 2:00 PM, Colin McCabe wrote: >> >> I think the

ARM/PowerPC builds

2022-08-03 Thread Colin McCabe
Just a quick note. Today we committed https://github.com/apache/kafka/pull/12380 , "MINOR: Remove ARM/PowerPC builds from Jenkinsfile #12380". This PR removes the ARM and PowerPC builds from the Jenkinsfile. The rationale is that these builds seem to be failing all the time, and this is very d

Re: [VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-03 Thread Colin McCabe
stand the intent of your comment. > > - Niket > >> On Aug 2, 2022, at 3:34 PM, Colin McCabe wrote: >> >> Hi Niket, >> >> Thanks for the KIP -- much appreciated! The new metrics look very useful. >> >> I agree with the proposed error handling for

Re: [VOTE] KIP-859: Add Metadata Log Processing Error Related Metrics

2022-08-02 Thread Colin McCabe
Hi Niket, Thanks for the KIP -- much appreciated! The new metrics look very useful. I agree with the proposed error handling for errors on standby controllers and brokers. For active controllers, I think we should establish two points: 1. the active controller replays metadata before submitting

Re: [ANNOUNCE] New Kafka PMC Member: A. Sophie Blee-Goldman

2022-08-02 Thread Colin McCabe
Congratulations! best, Colin On Tue, Aug 2, 2022, at 12:09, Matthew Benedict de Detrich wrote: > Congratulations Sophie! > > -- > Matthew de Detrich > Aiven Deutschland GmbH > Immanuelkirchstraße 26, 10405 Berlin > Amtsgericht Charlottenburg, HRB 209739 B > > Geschäftsführer: Oskari Saarenmaa &

[jira] [Created] (KAFKA-14129) KRaft must check manual assignments for createTopics are contiguous

2022-08-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14129: Summary: KRaft must check manual assignments for createTopics are contiguous Key: KAFKA-14129 URL: https://issues.apache.org/jira/browse/KAFKA-14129 Project: Kafka

[jira] [Created] (KAFKA-14124) Improve QuorumController fault handling

2022-07-28 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14124: Summary: Improve QuorumController fault handling Key: KAFKA-14124 URL: https://issues.apache.org/jira/browse/KAFKA-14124 Project: Kafka Issue Type

Re: [DISCUSS] KIP-853: KRaft Voters Change

2022-07-21 Thread Colin McCabe
Hi José, Thanks for the KIP! I have not had time to fully digest it, but I had some initial questions: 1. It seems like the proposal is to have a UUID per partition directory on the voter. If I understand correctly, this is sometimes referred to as "VoterUUID" and sometimes as "ReplicaUUID." T

[jira] [Created] (KAFKA-14084) Support SCRAM when using KRaft mode

2022-07-18 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-14084: Summary: Support SCRAM when using KRaft mode Key: KAFKA-14084 URL: https://issues.apache.org/jira/browse/KAFKA-14084 Project: Kafka Issue Type: Improvement

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

2022-07-06 Thread Colin McCabe
+1 (binding). thanks, Alexandre. On Mon, Jun 27, 2022, at 05:15, Alexandre Garnier wrote: > Hello! > > A little ping on this vote. > > Thanks. > > Le jeu. 16 juin 2022 à 16:36, Alexandre Garnier a écrit : > >> Hi everyone. >> >> Anyone wants to give a last binding vote for this KIP? >> >> Thanks

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

2022-06-19 Thread Colin McCabe
ribute. I have touched > configurations before so I would want to pick up Modifying certain > dynamic configurations on the standalone KRaft controller if possible. > Best,Christo > On Wednesday, 4 May 2022, 02:03:47 BST, Colin McCabe > wrote: > > Hi all, > > I&#x

[jira] [Resolved] (KAFKA-13902) Support online metadata.version upgrades for KIP-704 in KRaft

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13902?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13902. -- Fix Version/s: 3.3 Assignee: David Arthur Resolution: Fixed > Support onl

[jira] [Resolved] (KAFKA-13755) Broker heartbeat event should have deadline

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13755. -- Fix Version/s: 3.3 Assignee: Colin McCabe Resolution: Fixed We now use

[jira] [Resolved] (KAFKA-13657) StandardAuthorizer should implement the early start listener logic described in KIP-801

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13657?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13657. -- Fix Version/s: 3.3 Resolution: Duplicate > StandardAuthorizer should implement

[jira] [Resolved] (KAFKA-13649) StandardAuthorizer should not finish loading until it is ready

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13649?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13649. -- Fix Version/s: 3.3 Resolution: Fixed We fixed this in the early.start.listeners work

[jira] [Resolved] (KAFKA-13206) shutting down broker needs to stop fetching as a follower in KRaft mode

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13206. -- Fix Version/s: 3.3 Resolution: Duplicate Thanks for the bug report. We can finally

[jira] [Resolved] (KAFKA-13181) ReplicaManager should start fetchers on UnfencedBrokerRecords

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13181?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13181. -- Fix Version/s: 3.0.0 Assignee: Colin McCabe (was: Jose Armando Garcia Sancio

[jira] [Resolved] (KAFKA-12714) Kafka 2.8 server not starting on Windows OS

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12714?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12714. -- Resolution: Cannot Reproduce > Kafka 2.8 server not starting on Windows

[jira] [Resolved] (KAFKA-13749) TopicConfigs and ErrorCode are not set in createTopics response in KRaft

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13749?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13749. -- Resolution: Fixed This was fixed by this commit: {code} commit

[jira] [Resolved] (KAFKA-12502) Quorum controller should return topic configs in CreateTopic response

2022-06-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12502. -- Resolution: Duplicate > Quorum controller should return topic configs in CreateTopic respo

Re: [VOTE] KIP-746: Revise KRaft Metadata Records

2022-06-01 Thread Colin McCabe
Hi all, I updated this with the changes to FeatureLevelRecord that we did in KIP-778. Since the original version was never implemented, it would have been confusing to leave it there, I think. best, Colin On Wed, Jul 21, 2021, at 17:41, Colin McCabe wrote: > Hi all, > > I made an ad

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