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

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

2022-06-01 Thread Colin McCabe
gt; BrokerRegistrationChangeRecord it would be possible to avoid the version >> bump, since we're using tagged fields, but it's better to have it for >> consistency, I think.) > > Noted. > > Let me update the KIP to incorporate all your feedback.

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

2022-05-31 Thread Colin McCabe
> We should add a comment in AlterPartitionResponse about the new error > code INELIGIBLE_REPLICA. This is very important for error 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,

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

2022-05-31 Thread Colin McCabe
Hi David, Thanks again for working on this. It's an important one, for sure. It looks like the proposal has changed a bit so I'll re-review. With regard to the topic ID field in AlterPartitionRequest, I think this is a good idea. It's also good that it replaces the name instead of supplementing

[jira] [Resolved] (KAFKA-13937) StandardAuthorizer throws "ID 5t1jQ3zWSfeVLMYkN3uong not found in aclsById" exceptions into broker logs

2022-05-25 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13937?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13937. -- Fix Version/s: 3.3.0 Resolution: Duplicate I believe this is KAFKA-13649, which was

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

2022-05-25 Thread Colin McCabe
gt; aiming for that, in this case. > > Ismael > > On Tue, May 17, 2022 at 10:53 AM Colin McCabe wrote: > >> Hi, >> >> I'd like to start the vote on KIP-833: Mark KRaft as Production Ready. >> https://cwiki.apache.org/confluence/x/8xKhD >> >> thanks, >> Colin >>

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

2022-05-25 Thread Colin McCabe
In the list below, I accidentally included Luke Chen in the non-binding section. Actually, his vote belongs in the binding section. Sorry about that. best, Colin On Wed, May 25, 2022, at 11:42, Colin McCabe wrote: > Thanks, everyone! > > With binding +1 votes from: > Jason Gusta

[jira] [Resolved] (KAFKA-13909) Restart Kafka in KRaft mode with ACLs ends in a RuntimeException

2022-05-25 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13909?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13909. -- Fix Version/s: 3.3.0 Resolution: Duplicate > Restart Kafka in KRaft mode with A

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

2022-05-25 Thread Colin McCabe
Thanks, everyone! With binding +1 votes from: Jason Gustafson David Jacot John Roesler José Armando García Sancio Bill Bejeck and non-binding +1 votes from: Israel Ekpo Luke Chen the vote passes. thanks, all. Colin On Tue, May 24, 2022, at 08:43, Bill Bejeck wrote: > Hi Colin, > > Thanks for

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

2022-05-23 Thread Colin McCabe
Hi José, Yes, that matches my understanding. I added a section on "bridge releases" to the KIP. This section basically reiterates what we discussed in KIP-500, but it's good to have anyway. best, Colin On Wed, May 18, 2022, at 14:49, José Armando García Sancio wrote: > Hi Colin, > > Thanks for

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

2022-05-22 Thread Colin McCabe
Hi David, Thanks for the KIP. LGTM! best, Colin On Wed, May 18, 2022, at 08:55, David Jacot wrote: > 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: [DISCUSS] KIP-841: Fenced replicas should not be allowed to join the ISR in KRaft

2022-05-22 Thread Colin McCabe
Hi Artem, In ZK mode, the concept of fenced replicas does not exist -- brokers are either in the cluster or out, and if they're out, we scrub all information about them from the metadata. (With the exception of replica sets they are in.) best, Colin On Fri, May 20, 2022, at 10:17, Artem Livshi

[jira] [Resolved] (KAFKA-13807) Ensure that we can set log.flush.interval.ms with IncrementalAlterConfigs

2022-05-19 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13807?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13807. -- Resolution: Fixed > Ensure that we can set log.flush.interval.ms with IncrementalAlterConf

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

2022-05-17 Thread Colin McCabe
Hi, I'd like to start the vote on KIP-833: Mark KRaft as Production Ready. https://cwiki.apache.org/confluence/x/8xKhD thanks, Colin

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

2022-05-16 Thread Colin McCabe
ion I can close this one and create a new one. > I think it's reasonable to keep this one open and make the changes you talked about above. regards, Colin > > Thanks, > Omnia > > On Fri, May 13, 2022 at 6:10 PM Colin McCabe wrote: > >> On Wed, May 11, 2022,

Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-13 Thread Colin McCabe
I agree. I would really like to see a builder interface here. If you changed the KIP to add this, you'd have to do the vote again, but I think it would be well worth it. best, Colin On Thu, May 12, 2022, at 00:13, Bruno Cadonna wrote: > Hi Francois, > > Modifying this KIP or starting a new one

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

2022-05-13 Thread Colin McCabe
ions. So while there may be other reasons to use this kind of agent-based architecture, added security isn't one. In any case, I think we should close this KIP since we already have an Admin API. There isn't a need to create a public API for admin operations. best, Colin > > On We

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

2022-05-11 Thread Colin McCabe
IP only conveys the first item and not the next > two > > It is just a thought. I will like to get your perspective > Names are always a hard part :) I didn't want the name to get too long. Maybe if others want to see a longer name I can change it -- I'm open. best, C

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

2022-05-11 Thread Colin McCabe
Mode >> >> Should this be broken up in three separate KIPs since it will be done in >> multiple releases? >> >> The current name of the KIP only conveys the first item and not the next >> two >> >> It is just a thought. I will like to get your perspectiv

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

2022-05-11 Thread Colin McCabe
Hi Omnia Ibrahim, I'm sorry, but I am -1 on adding competing Admin interfaces. This would create confusion and a heavier maintenance burden for the project. Since the org.apache.kafka.clients.admin.Admin interface is a Java interface, any third-party software that wants to insert its own implem

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

2022-05-11 Thread Colin McCabe
Thanks, Niket. I also agree with Jason that this is a public API despite the lack of command-line tool, so we do indeed need a KIP. :) One minor point: I suspect that whatever we end up naming the additional fields here, should also be the name of the metrics in KIP-835. So if we go with a metr

Re: [DISCUSS] KIP-835: Monitor KRaft Controller Quorum Health

2022-05-11 Thread Colin McCabe
Hi José, Thanks for the KIP! I think this will be a nice improvement. I had the same question as Luke and Jason: what's the default here for the NoOpRecord time? :) We should add a value here even if we think we'll adjust it later, just to give a feeling for how much traffic this would create.

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

2022-05-05 Thread Colin McCabe
> removal and 8 months from zk deprecation to zk removal. >> > >> > If necessary (due to important bugs or security issues), we can do a >> couple >> > of additional bug fix releases in the 3.5 series after 4.0 is released. >> > >> > Thoughts? >

[jira] [Created] (KAFKA-13874) Avoid synchronization in SocketServer metrics

2022-05-04 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13874: Summary: Avoid synchronization in SocketServer metrics Key: KAFKA-13874 URL: https://issues.apache.org/jira/browse/KAFKA-13874 Project: Kafka Issue Type

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

2022-05-03 Thread Colin McCabe
On Tue, May 3, 2022, at 23:16, Colin McCabe wrote: > > To be clear, the proposal here is to have the bridge release be 3.4 and > then move on to a ZK-free 4.0. With a 3.5 release as an option (but not > requirement) if we can't finish everything in time after 3.4. So that &g

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

2022-05-03 Thread Colin McCabe
sed below, in chronological terms a Kafka release is about 4 or 5 months. So if we have 3.3, followed by a 4.0 where we drop ZK, that would be about 8 to 10 months. If we need more time we can add a 3.4 which will add another 4-5 months in between. best, Colin > > Thank you. > Luke >

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

2022-05-03 Thread Colin McCabe
; (i.e. v3.4.0) to ZK removal (i.e. v4.0.0) is not too short. >> Do we have any expectation for the deprecation period? >> After all, this is not a small feature change, and users need more time to >> do the migration. >> But to be honest, I don't know how long is long

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

2022-05-03 Thread Colin McCabe
Hi all, I've written a KIP for marking KRaft as production ready. Please take a look if you have a chance: https://cwiki.apache.org/confluence/x/8xKhD thanks, Colin

Re: [DISCUSS] KIP-827: Expose logdirs total and usable space via Kafka API

2022-05-03 Thread Colin McCabe
Hi Mickael, Thanks for the KIP. In the API, I would suggest using an OptionalLong rather than a "magic value" of -1. best, Colin On Thu, Apr 7, 2022, at 02:16, Mickael Maison wrote: > Hi, > > I wrote a small KIP to expose the total and usable space of logdirs > via the DescribeLogDirs API: >

Re: [DISCUSS] KIP-660: Pluggable ReplicaPlacer

2022-05-03 Thread Colin McCabe
Hi Mickael, We did discuss this earlier, and I remember not being too enthusiastic about a pluggable policy here :) There have been several changes to the placement code in the last few weeks. (These are examples of the kind of changes that are impossible to do once an API is established, by t

[jira] [Created] (KAFKA-13867) Improve JavaDoc for MetadataVersion.java

2022-05-02 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13867: Summary: Improve JavaDoc for MetadataVersion.java Key: KAFKA-13867 URL: https://issues.apache.org/jira/browse/KAFKA-13867 Project: Kafka Issue Type

[jira] [Created] (KAFKA-13841) Fix a case where we were unable to place on fenced brokers In KRaft mode

2022-04-20 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13841: Summary: Fix a case where we were unable to place on fenced brokers In KRaft mode Key: KAFKA-13841 URL: https://issues.apache.org/jira/browse/KAFKA-13841 Project

[jira] [Created] (KAFKA-13835) Fix two bugs related to applying dynamic broker configs in KRaft

2022-04-18 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13835: Summary: Fix two bugs related to applying dynamic broker configs in KRaft Key: KAFKA-13835 URL: https://issues.apache.org/jira/browse/KAFKA-13835 Project: Kafka

[jira] [Resolved] (KAFKA-13651) Add audit logging to StandardAuthorizer

2022-04-13 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13651?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13651. -- Resolution: Fixed > Add audit logging to StandardAuthori

[jira] [Created] (KAFKA-13807) Ensure that we can set log.flush.interval.ms with IncrementalAlterConfigs

2022-04-07 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13807: Summary: Ensure that we can set log.flush.interval.ms with IncrementalAlterConfigs Key: KAFKA-13807 URL: https://issues.apache.org/jira/browse/KAFKA-13807 Project

[jira] [Created] (KAFKA-13774) AclAuthorizer should handle it a bit more gracefully if zookeeper.connect is null

2022-03-28 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13774: Summary: AclAuthorizer should handle it a bit more gracefully if zookeeper.connect is null Key: KAFKA-13774 URL: https://issues.apache.org/jira/browse/KAFKA-13774

[jira] [Created] (KAFKA-13743) kraft controller should prevent topics with conflicting metrics names from being created

2022-03-15 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13743: Summary: kraft controller should prevent topics with conflicting metrics names from being created Key: KAFKA-13743 URL: https://issues.apache.org/jira/browse/KAFKA-13743

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

2022-02-08 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13657: Summary: StandardAuthorizer should implement the early start listener logic described in KIP-801 Key: KAFKA-13657 URL: https://issues.apache.org/jira/browse/KAFKA-13657

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

2022-02-08 Thread Colin McCabe
Hi José, Thanks for the KIP. +1 (binding). best, Colin On Mon, Feb 7, 2022, at 19:03, Luke Chen wrote: > I'm +1 (non-binding) > > Thanks for the KIP! > > Luke > > On Tue, Feb 8, 2022 at 7:42 AM Raman Verma > wrote: > >> +1, thanks Jose. >> >> On Mon, Feb 7, 2022 at 8:56 AM David Jacot wrote:

[jira] [Created] (KAFKA-13651) StandardAuthorizer should implement all the required log4j stuff

2022-02-07 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13651: Summary: StandardAuthorizer should implement all the required log4j stuff Key: KAFKA-13651 URL: https://issues.apache.org/jira/browse/KAFKA-13651 Project: Kafka

[jira] [Created] (KAFKA-13650) Add StandardAuthorizer JMH benchmark

2022-02-07 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13650: Summary: Add StandardAuthorizer JMH benchmark Key: KAFKA-13650 URL: https://issues.apache.org/jira/browse/KAFKA-13650 Project: Kafka Issue Type: Improvement

[jira] [Created] (KAFKA-13649) StandardAuthorizer should not finish loading until it reads up to the high water mark

2022-02-07 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13649: Summary: StandardAuthorizer should not finish loading until it reads up to the high water mark Key: KAFKA-13649 URL: https://issues.apache.org/jira/browse/KAFKA-13649

[jira] [Resolved] (KAFKA-13193) Replica manager doesn't update partition state when transitioning from leader to follower with unknown leader

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13193. -- Resolution: Fixed > Replica manager doesn't update partition state when transition

[jira] [Created] (KAFKA-13646) Implement KIP-801: KRaft authorizer

2022-02-04 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13646: Summary: Implement KIP-801: KRaft authorizer Key: KAFKA-13646 URL: https://issues.apache.org/jira/browse/KAFKA-13646 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-12421) Improve controller's atomic grouping

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12421. -- Assignee: Jose Armando Garcia Sancio (was: HaiyuanZhao) Resolution: Fixed > Impr

[jira] [Resolved] (KAFKA-12214) Generated code does not include UUID or struct fields in its toString output

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12214. -- Resolution: Fixed > Generated code does not include UUID or struct fields in its toStr

[jira] [Resolved] (KAFKA-12271) Expose consistent Raft metadata to components

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12271?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12271. -- Fix Version/s: 3.0.0 Resolution: Fixed > Expose consistent Raft metadata to compone

[jira] [Resolved] (KAFKA-12209) Add the timeline data structures for the KIP-631 controller

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12209. -- Fix Version/s: 2.8.0 Assignee: Colin McCabe Resolution: Fixed > Add

[jira] [Resolved] (KAFKA-10724) Command to run single quorum in raft is missing "--config" parameters.

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10724?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-10724. -- Resolution: Fixed > Command to run single quorum in raft is missing "--config&quo

[jira] [Resolved] (KAFKA-9154) ProducerId generation should be managed by the Controller

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9154. - Fix Version/s: 3.1.0 Resolution: Fixed We implemented KIP-730 in Kafka 3.1. Closing this

[jira] [Resolved] (KAFKA-13503) Validate broker configs for KRaft

2022-02-04 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13503?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-13503. -- Fix Version/s: 3.2.0 Reviewer: Jose Armando Garcia Sancio Assignee: Colin

Re: [VOTE] KIP-801 new authorizer for kip-500 kraft mode

2022-02-04 Thread Colin McCabe
Hi all, The vote is now closed. KIP-801 has been approved with 3 binding +1s from Jason Gustafson, Manikumar Reddy, and David Arthur. Thanks, all. Colin On Wed, Feb 2, 2022, at 16:14, Colin McCabe wrote: > Thanks, Manikumar. Together with Jason and David's votes (David's vote

[jira] [Created] (KAFKA-13642) We should have a command-line tool + API to display fenced / unfenced / etc. brokers

2022-02-03 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13642: Summary: We should have a command-line tool + API to display fenced / unfenced / etc. brokers Key: KAFKA-13642 URL: https://issues.apache.org/jira/browse/KAFKA-13642

Re: [VOTE] KIP-801 new authorizer for kip-500 kraft mode

2022-02-02 Thread Colin McCabe
ding). Thanks for the KIP. > > On Tue, Feb 1, 2022 at 10:43 PM Jason Gustafson > wrote: > >> +1 Thanks! >> >> On Mon, Jan 31, 2022 at 6:20 PM Colin McCabe wrote: >> >> > Hi all, >> > >> > It looks like people using gmail are seeing the p

[jira] [Created] (KAFKA-13640) Implement final broker heartbeat in kraft

2022-02-02 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13640: Summary: Implement final broker heartbeat in kraft Key: KAFKA-13640 URL: https://issues.apache.org/jira/browse/KAFKA-13640 Project: Kafka Issue Type

Re: [DISCUSS] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-02-02 Thread Colin McCabe
> wrote: > >> Sounds good on the ordering, and yea I agree we can look at atomic ACL >> modifications in the future. Thanks! >> >> On Wed, Jan 12, 2022 at 3:53 PM Colin McCabe wrote: >> >> > Hi David, >> > >> > On Wed, Dec 15,

[VOTE] KIP-801 new authorizer for kip-500 kraft mode

2022-01-31 Thread Colin McCabe
Hi all, It looks like people using gmail are seeing the previous vote thread as merged with the discuss thread, so let me create a new thread in order to avoid confusion. Usually using a very different thread title works well enough to avoid the merging. Original vote thread: https://lists.ap

Re: [VOTE] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-01-31 Thread Colin McCabe
ards, Colin > Thanks, > Jason > > > > > On Mon, Jan 31, 2022 at 3:02 PM Colin McCabe wrote: > >> Hi all, >> >> Thanks, everyone. I've updated the KIP. >> >> I reorganized the design section to explain bootstrapping a bit better, >>

Re: [VOTE] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-01-31 Thread Colin McCabe
es.apache.org/jira/browse/KAFKA-13621. That said, I agree it > would be desirable if the same approach could be used by both the broker > and controller. > > -Jason > > > On Thu, Jan 27, 2022 at 4:14 PM Colin McCabe wrote: > >> On Thu, Jan 27, 2022, a

Re: [VOTE] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-01-27 Thread Colin McCabe
aleness for KRaft, we may want to consider doing the same for ZK, for consistency. Another approach, which I suspect is more realistic in practice, is to have a metric tracking staleness and fire off an alert when it grows too high. Making the stale node inaccessible is sort of the nuclear option

[jira] [Created] (KAFKA-13620) The request handler metric name for ControllerApis should be different than KafkaApis

2022-01-26 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13620: Summary: The request handler metric name for ControllerApis should be different than KafkaApis Key: KAFKA-13620 URL: https://issues.apache.org/jira/browse/KAFKA-13620

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

2022-01-26 Thread Colin McCabe
On Wed, Jan 26, 2022, at 09:14, José Armando García Sancio wrote: > Thanks for the feedback Colin. > > Colin wrote: >> We already have many classes that are called "partition state." For example, >> PartitionStates.java on the client side, PartitionStateMachine.scala and >> TopicPartitionStateZNo

Re: [VOTE] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-01-26 Thread Colin McCabe
in a future KIP since it's pretty cross-cutting (if we had such a limited bootstrapping user, all the authorizers could implement it, not just the KIP-801 one...) best, Colin > > Regards, > > Rajini > > > On Wed, Jan 26, 2022 at 1:50 AM Colin McCabe wrote: > >>

Re: [VOTE] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-01-25 Thread Colin McCabe
now when the controller is ready to accept these forwarded > requests (e.g. through an error code in the `Envelope` response). > > What do you think? > > Thanks, > Jason > > > > > > > On Wed, Jan 12, 2022 at 12:57 PM David Arthur > wrote: > >> +1 bind

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

2022-01-25 Thread Colin McCabe
On Fri, Jan 21, 2022, at 11:07, José Armando García Sancio wrote: > Hi all, > > The following suggestions are not strictly required to implement this > KIP but what do we think about: > > 1. Changing the name of the AlterIsr RPC to AlterPartition RPC. > > 2. Change the name of the field "CurrentIsr

Re: __consumer_offsets auto creation with `auto.create.topics.enable = false`

2022-01-24 Thread Colin McCabe
Kafka internal topics are… well… internal, and we don’t guarantee anything about their format or whether or not they are present, etc. There is no guarantee that Kafka will always use __consumer-offsets to manage offsets — it could change in the future. I think the big mistake we made is making

[jira] [Created] (KAFKA-13610) Make log.cleaner.enable dynamically configurable

2022-01-21 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13610: Summary: Make log.cleaner.enable dynamically configurable Key: KAFKA-13610 URL: https://issues.apache.org/jira/browse/KAFKA-13610 Project: Kafka Issue Type

[jira] [Created] (KAFKA-13609) Fix the exception type thrown from dynamic broker config validation

2022-01-21 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13609: Summary: Fix the exception type thrown from dynamic broker config validation Key: KAFKA-13609 URL: https://issues.apache.org/jira/browse/KAFKA-13609 Project: Kafka

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

2022-01-20 Thread Colin McCabe
Hi José, Thanks for the changes. "isLeaderRecovering" sounds pretty awkward. If we want to call this "leader recovery" then maybe the flag could be something like "inLeaderRecovery." Actually, how about "inElectionRecovery" to emphasize the fact that we are recovering from an unclean leader el

Re: [DISCUSS] KIP-719: Add Log4J2 Appender

2022-01-18 Thread Colin McCabe
On Wed, Jan 12, 2022, at 02:37, Viktor Somogyi-Vass wrote: > Hi Dongjin, > > We're also looking into this matter as our stack was also affected by all > the log4j hell and users increasingly pushing us to upgrade to log4j2 or > logback because of the existing vulnerabilities in log4j1. > Regarding

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

2022-01-18 Thread Colin McCabe
Hi José, Thanks for the KIP. The KIP talks a bit about "recovery," which is a new term (as far as I know). If I understand correctly, this is a state that the partition enters into after an unclean leader election. I would suggest using a different term for this, since we already use the term

Re: [DISCUSS] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-01-12 Thread Colin McCabe
tiple ACLs in ZK requires changing multiple znodes, which is not atomic. Given that users haven't asked for this and it would add substantial complexity, can be discuss it later once we have feature parity with the ZK version? best, Colin > > > On Tue, Dec 14, 2021 at 4:20 PM Coli

Re: [VOTE] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2022-01-12 Thread Colin McCabe
bumping this thread On Mon, Dec 13, 2021, at 16:46, Colin McCabe wrote: > Hi all, > > I'd like to start the vote on KIP-801: Implement an Authorizer that > stores metadata in __cluster_metadata > > The KIP is here: https://cwiki.apache.org/confluence/x/h5KqCw > > T

Re: [VOTE] KIP-778 KRaft upgrades

2021-12-16 Thread Colin McCabe
Thanks for the KIP, David! Great work. +1 (binding). Should the "./kafka-features.sh downgrade" command also have a --release flag, to match upgrade? Also, it seems like upgrade should have a --latest flag that upgrades everything to the latest installed version? best, Colin On Fri, Dec 10,

Re: [Proposal] It is hoped that Apache APISIX and Apache Kafka will carry out diversified community cooperation

2021-12-16 Thread Colin McCabe
Hi Yeliang Wang, Have you considered submitting a talk to Kafka Summit discussing how Apache APISIX uses Kafka? That might be interesting. I'm not sure what you mean by "fully integrating" the projects. Can you elaborate on what integration you see happening in the future? best, Colin On Sun,

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

2021-12-16 Thread Colin McCabe
;t read >> > or >> > > > write epoch, new IBP = do) >> > > > >> > > > I think this KIP doesn't care about the group epoch number at all. >> The >> > > > subscription metadata is passed from each member to group >&g

Re: [DISCUSS] KIP-801: Implement an Authorizer that stores metadata in __cluster_metadata

2021-12-14 Thread Colin McCabe
Thanks! > David > > > > > > > > On Tue, Dec 14, 2021 at 11:27 AM José Armando García Sancio > wrote: > >> Thanks for the additional information Colin. >> >> On Mon, Dec 13, 2021 at 4:43 PM Colin McCabe wrote: >> > >> > Hi José, >>

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