[jira] [Resolved] (KAFKA-15471) Allow independently stop KRaft controllers or brokers

2023-12-15 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-15471. --- Resolution: Fixed > Allow independently stop KRaft controllers or brok

Re: [VOTE] KIP-979 Allow independently stop KRaft processes

2023-10-23 Thread Ron Dagostino
Hi Hailey. I'm +1 (binding), but could you add a "Rejected Alternatives" section to the KIP and mention the "--required-config " option that we decided against and the reason why we made the decision to reject it? There were some other small things (dash instead of dot in the parameter names, --n

Re: [DISCUSS] KIP-979: Allow independently stop KRaft controllers or brokers

2023-09-30 Thread Ron Dagostino
oding it will make the functionality much more flexible. I've updated >> the KIP and the code accordingly. Thanks for the great idea! >> >> Thanks, >> Hailey >> >> >>> On Fri, Sep 29, 2023 at 2:34 PM Ron Dagostino wrote: >>> >>

Re: [DISCUSS] KIP-979: Allow independently stop KRaft controllers or brokers

2023-09-29 Thread Ron Dagostino
gt; > > Hi Ron. Thanks for the response. I agree with your point. I'll make the > > corresponding changes in the KIP and KAFKA-15471 > > <https://issues.apache.org/jira/browse/KAFKA-15471>. > > > > On Thu, Sep 21, 2023 at 1:40 PM Ron Dagostino wrote: > >

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

2023-09-24 Thread Ron Dagostino
Hi Igor. I've opened https://issues.apache.org/jira/browse/KAFKA-15495 to identify the data loss scenario that I was referring to, including steps on how to reproduce it. I agree this is a separate issue from JBOD per se. The disk UUID that this KIP introduces does give us the ability to avoid l

[jira] [Created] (KAFKA-15495) KRaft partition truncated when the only ISR member restarts with and empty disk

2023-09-24 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-15495: - Summary: KRaft partition truncated when the only ISR member restarts with and empty disk Key: KAFKA-15495 URL: https://issues.apache.org/jira/browse/KAFKA-15495

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

2023-09-22 Thread Ron Dagostino
ll see the UUID mismatch between what the broker is presenting now and what it had presented the last time it registered. So we need to deal with this possibility even for the case where a broker has a single log directory. WDYT? Ron On Tue, Sep 19, 2023 at 10:04 AM Ron Dagostino wrote: >

Re: [DISCUSS] KIP-979: Allow independently stop KRaft controllers or brokers

2023-09-21 Thread Ron Dagostino
uch for the comment. I think it makes sense to me that we > provide an even more specific way to kill individual controllers/brokers. > I have one question: does the command line for ZooKeeper cluster provide > such a way to kill individual controllers/brokers? > > Thanks, > Hailey &g

Re: [DISCUSS] KIP-979: Allow independently stop KRaft controllers or brokers

2023-09-19 Thread Ron Dagostino
Thanks for the KIP, Hailey. It will be nice to provide some fine-grained control for when people running the broker and controller this way want to stop just one of them. One thing that occurs to me is that in a development environment someone might want to run multiple controllers and multiple b

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

2023-09-19 Thread Ron Dagostino
Ok, great, that makes sense, Igor. Thanks. +1 (binding) on the KIP from me. Ron > On Sep 13, 2023, at 11:58 AM, Igor Soarez wrote: > > Hi Ron, > > Thanks for drilling down on this. I think the KIP isn't really clear here, > and the metadata caching section you quoted needs clarification. >

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

2023-09-12 Thread Ron Dagostino
Thanks, Igor. Regarding the last point, I want to drill in on something. The KIP says this: "Replicas will also be considered offline if the replica references a log directory UUID (in the new field partitionRecord.Assignment.Directory) that is not present in the hosting Broker's latest registra

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

2023-09-10 Thread Ron Dagostino
Hi Igor. Thanks for all your work here. Before I can vote, I have the following questions/comments about the KIP: > When multiple log.dirs are configured, a new property will be included > in meta.properties — directory.id — which will identify each log directory > with a UUID. The UUID is rando

[jira] [Resolved] (KAFKA-15219) Support delegation tokens in KRaft

2023-08-19 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15219?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-15219. --- Fix Version/s: 3.6.0 Resolution: Fixed > Support delegation tokens in KR

Re: [VOTE] KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add Controller Registration

2023-07-27 Thread Ron Dagostino
singular one is > just an alias for compatibility or stick with singular for both. > > Ismael > > On Thu, Jul 27, 2023, 12:03 AM Colin McCabe wrote: > > > On Wed, Jul 26, 2023, at 07:09, Ron Dagostino wrote: > > > Thanks, Colin. +1 (binding) from me. > > >

Re: [VOTE] KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add Controller Registration

2023-07-26 Thread Ron Dagostino
Thanks, Colin. +1 (binding) from me. I will note that Ziming mentioned in the DISCUSS thread that "There is a mistake that we use `--bootstrap-server` instead of `--bootstrap-server(s)`, so should we also change the new argument `--bootstrap-controller` (no s).". I agree that this is an unfortun

[jira] [Created] (KAFKA-15098) KRaft migration does not proceed and broker dies if authorizer.class.name is set

2023-06-16 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-15098: - Summary: KRaft migration does not proceed and broker dies if authorizer.class.name is set Key: KAFKA-15098 URL: https://issues.apache.org/jira/browse/KAFKA-15098

Re: [VOTE] KIP-938: Add more metrics for measuring KRaft performance

2023-06-06 Thread Ron Dagostino
Thanks again for the KIP, Colin. +1 (binding). Ron > On Jun 6, 2023, at 7:02 AM, Igor Soarez wrote: > > Thanks for the KIP. > > Seems straightforward, LGTM. > Non binding +1. > > -- > Igor >

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

2023-06-05 Thread Ron Dagostino
> How to attract more feedback from committers for this proposal Hi Igor. I'll review the KIP later this week Ron On Mon, Jun 5, 2023 at 1:37 PM Igor Soarez wrote: > > Hi all, > > We just had a video call to discuss this KIP and I just wanted > update this thread with a note on the meeting. >

Re: [DISCUSS] KIP-938: Add more metrics for measuring KRaft performance

2023-06-02 Thread Ron Dagostino
Thanks for the clarification and explanation, Colin. Looks good to me. Ron On Fri, Jun 2, 2023 at 2:52 PM Colin McCabe wrote: > > Hi Ron, > > Thanks for the review. > > On Fri, Jun 2, 2023, at 11:26, Ron Dagostino wrote: > > Thanks for the KIP, Colin.

Re: [DISCUSS] KIP-938: Add more metrics for measuring KRaft performance

2023-06-02 Thread Ron Dagostino
Thanks for the KIP, Colin. The KIP cals one metric "NewActiveControllersCount" but we don't append "Count" to the other metric names (e.g. it is "TimedOutBrokerHeartbeats" instead of "TimedOutBrokerHeartbeatsCount"). Should we be consistent (either use the suffix everywhere or don't use it anywhe

[jira] [Created] (KAFKA-15039) Reduce logging level to trace in PartitionChangeBuilder.tryElection()

2023-05-30 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-15039: - Summary: Reduce logging level to trace in PartitionChangeBuilder.tryElection() Key: KAFKA-15039 URL: https://issues.apache.org/jira/browse/KAFKA-15039 Project

Re: [DISCUSS] Apache Kafka 3.5.0 release

2023-04-20 Thread Ron Dagostino
Hi Mickael. I would like to merge https://github.com/apache/kafka/pull/13532 (KAFKA-14887: No shutdown for ZK session expiration in feature processing) to the 3.5 branch. It is a very small and focused fix that can cause unexpected broker shutdowns when there is instability in the connectivity to

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

2023-04-19 Thread Ron Dagostino
Thanks for the KIP, Colin. There seems to be some inconsistency between sometimes referring to "TargetKRaftControllerQuorum" and other times referring to "DirectToKRaftControllerQuorum". Aside from that, it looks good to me. The symmetry of bootstrap servers and bootstrap controllers feels right

[jira] [Resolved] (KAFKA-14735) Improve KRaft metadata image change performance at high topic counts

2023-04-18 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14735?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14735. --- Resolution: Fixed > Improve KRaft metadata image change performance at high topic cou

[jira] [Resolved] (KAFKA-14890) Kafka initiates shutdown due to connectivity problem with Zookeeper and FatalExitError from ChangeNotificationProcessorThread

2023-04-12 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14890?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14890. --- Resolution: Duplicate Duplicate of https://issues.apache.org/jira/browse/KAFKA-14887

[jira] [Created] (KAFKA-14887) ZK session timeout can cause broker to shutdown

2023-04-10 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14887: - Summary: ZK session timeout can cause broker to shutdown Key: KAFKA-14887 URL: https://issues.apache.org/jira/browse/KAFKA-14887 Project: Kafka Issue Type

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

2023-03-07 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14351. --- Fix Version/s: 3.5.0 Resolution: Fixed > Implement controller mutation quotas

[jira] [Created] (KAFKA-14735) Improve KRaft metadata image change performance at high topic counts

2023-02-20 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14735: - Summary: Improve KRaft metadata image change performance at high topic counts Key: KAFKA-14735 URL: https://issues.apache.org/jira/browse/KAFKA-14735 Project

[jira] [Created] (KAFKA-14731) Upgrade ZooKeeper to 3.6.4

2023-02-17 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14731: - Summary: Upgrade ZooKeeper to 3.6.4 Key: KAFKA-14731 URL: https://issues.apache.org/jira/browse/KAFKA-14731 Project: Kafka Issue Type: Task Affects

[jira] [Resolved] (KAFKA-14711) kafaka-metadata-quorum.sh does not honor --command-config

2023-02-13 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14711?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14711. --- Resolution: Fixed > kafaka-metadata-quorum.sh does not honor --command-con

[jira] [Created] (KAFKA-14711) kafaka-metadata-quorum.sh does not honor --command-config

2023-02-13 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14711: - Summary: kafaka-metadata-quorum.sh does not honor --command-config Key: KAFKA-14711 URL: https://issues.apache.org/jira/browse/KAFKA-14711 Project: Kafka

Re: [ANNOUNCE] New committer: Stanislav Kozlovski

2023-01-17 Thread Ron Dagostino
Congratulations, Stan! Ron > On Jan 17, 2023, at 12:29 PM, Mickael Maison wrote: > > Congratulations Stanislav! > >> On Tue, Jan 17, 2023 at 6:06 PM Rajini Sivaram >> wrote: >> >> Congratulations, Stan! >> >> Regards, >> >> Rajini >> >>> On Tue, Jan 17, 2023 at 5:04 PM Tom Bentley wrot

Re: [ANNOUNCE] New committer: Justine Olshan

2022-12-29 Thread Ron Dagostino
Congratulations, Justine!Well-deserved., and I’m very happy for you. Ron > On Dec 29, 2022, at 6:13 PM, Israel Ekpo wrote: > > Congratulations Justine! > > >> On Thu, Dec 29, 2022 at 5:05 PM Greg Harris >> wrote: >> >> Congratulations Justine! >> >>> On Thu, Dec 29, 2022 at 1:37 PM Bi

Re: [ANNOUNCE] New committer: Ron Dagostino

2022-12-16 Thread Ron Dagostino
aram wrote: >>> Congratulations, Ron! Well deserved!! >>> >>> Regards, >>> >>> Rajini >>> >>> On Thu, Dec 15, 2022 at 11:42 AM Ron Dagostino >> wrote: >>> >>>> Thank you, everyone! >>>> >>&

Re: [ANNOUNCE] New committer: Ron Dagostino

2022-12-15 Thread Ron Dagostino
Thank you, everyone! Ron > On Dec 15, 2022, at 5:09 AM, Bruno Cadonna wrote: > > Congrats Ron! > > Best, > Bruno > >> On 15.12.22 10:23, Viktor Somogyi-Vass wrote: >> Congrats Ron! :) >>> On Thu, Dec 15, 2022 at 10:22 AM Mickael Maison >>> wrote: >>> Congratulations Ron! >>> >>> On Thu, De

Re: [ANNOUNCE] New committer: Viktor Somogyi-Vass

2022-12-15 Thread Ron Dagostino
Congrats to you too, Victor! Ron > On Dec 15, 2022, at 4:59 AM, Viktor Somogyi-Vass > wrote: > > Thank you everyone! :) > >> On Thu, Dec 15, 2022 at 10:22 AM Mickael Maison >> wrote: >> >> Congratulations Viktor! >> >>> On Thu, Dec 15, 2022 at 10:06 AM Tamas Barnabas Egyed >>> wrote: >>>

Re: [DISCUSS] Apache Kafka 3.4.0 release

2022-12-12 Thread Ron Dagostino
Hi Sophie. https://github.com/apache/kafka/pull/12856 "KAFKA-14392: KRaft broker heartbeat timeout should not exceed broker.session.timeout.ms" just merged to trunk and seems like a potential candidate for 3.4.0 as it is a very small and targeted fix with very low risk. Could it be included even t

Re: [DISCUSS] Apache Kafka 3.3.2

2022-12-12 Thread Ron Dagostino
Hi Chris. https://github.com/apache/kafka/pull/12856 "KAFKA-14392: KRaft broker heartbeat timeout should not exceed broker.session.timeout.ms" just merged to trunk and seems like a reasonable candidate for 3.3.2. Can it be included? Ron On Fri, Nov 18, 2022 at 10:57 AM Chris Egerton wrote: > >

[jira] [Resolved] (KAFKA-14392) KRaft broker heartbeat timeout should not exceed broker.session.timeout.ms

2022-12-12 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14392. --- Fix Version/s: 3.5.0 Resolution: Fixed > KRaft broker heartbeat timeout should

[jira] [Resolved] (KAFKA-14394) BrokerToControllerChannelManager has 2 separate timeouts

2022-11-22 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14394. --- Resolution: Not A Problem > BrokerToControllerChannelManager has 2 separate timeo

[jira] [Created] (KAFKA-14394) BrokerToControllerChannelManager has 2 separate timeouts

2022-11-16 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14394: - Summary: BrokerToControllerChannelManager has 2 separate timeouts Key: KAFKA-14394 URL: https://issues.apache.org/jira/browse/KAFKA-14394 Project: Kafka

[jira] [Created] (KAFKA-14392) KRaft should comment controller.socket.timeout.ms <= broker.session.timeout.ms

2022-11-15 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14392: - Summary: KRaft should comment controller.socket.timeout.ms <= broker.session.timeout.ms Key: KAFKA-14392 URL: https://issues.apache.org/jira/browse/KAFKA-14

[jira] [Created] (KAFKA-14371) quorum-state file contains empty/unused clusterId field

2022-11-09 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14371: - Summary: quorum-state file contains empty/unused clusterId field Key: KAFKA-14371 URL: https://issues.apache.org/jira/browse/KAFKA-14371 Project: Kafka

[jira] [Created] (KAFKA-14195) Fix KRaft AlterConfig policy usage for Legacy/Full case

2022-09-01 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14195: - Summary: Fix KRaft AlterConfig policy usage for Legacy/Full case Key: KAFKA-14195 URL: https://issues.apache.org/jira/browse/KAFKA-14195 Project: Kafka

[jira] [Resolved] (KAFKA-14051) KRaft remote controllers do not create metrics reporters

2022-08-15 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14051. --- Resolution: Fixed > KRaft remote controllers do not create metrics report

[jira] [Created] (KAFKA-14105) Remove quorum.all_non_upgrade for system tests

2022-07-25 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14105: - Summary: Remove quorum.all_non_upgrade for system tests Key: KAFKA-14105 URL: https://issues.apache.org/jira/browse/KAFKA-14105 Project: Kafka Issue Type

[jira] [Created] (KAFKA-14057) Support dynamic reconfiguration in KRaft remote controllers

2022-07-08 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14057: - Summary: Support dynamic reconfiguration in KRaft remote controllers Key: KAFKA-14057 URL: https://issues.apache.org/jira/browse/KAFKA-14057 Project: Kafka

[jira] [Created] (KAFKA-14056) Test reading of old messages formats in ZK-to-KRaft upgrade test

2022-07-08 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14056: - Summary: Test reading of old messages formats in ZK-to-KRaft upgrade test Key: KAFKA-14056 URL: https://issues.apache.org/jira/browse/KAFKA-14056 Project: Kafka

[jira] [Created] (KAFKA-14051) KRaft remote controllers do not create metrics reporters

2022-07-06 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-14051: - Summary: KRaft remote controllers do not create metrics reporters Key: KAFKA-14051 URL: https://issues.apache.org/jira/browse/KAFKA-14051 Project: Kafka

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

2022-05-12 Thread Ron Dagostino
Hi Niket. Thanks for the KIP. Are all the fields you specified always known? For example, might a new controller not have a last fetch time for other voters, and then what would it send in the response? If this is possible then we should be explicit about what is to be sent in this case. Ron

[jira] [Created] (KAFKA-13552) Unable to dynamically change broker log levels on KRaft

2021-12-16 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13552: - Summary: Unable to dynamically change broker log levels on KRaft Key: KAFKA-13552 URL: https://issues.apache.org/jira/browse/KAFKA-13552 Project: Kafka

Re: [DISCUSS] Apache Kafka 3.1.0 release

2021-11-23 Thread Ron Dagostino
Hi David. I would like to nominate https://issues.apache.org/jira/projects/KAFKA/issues/KAFKA-13456 "Tighten KRaft config checks/constraints" as a 3.1.0 blocker. The existing configuration constraints/checks related to KRaft currently do not eliminate certain illegal configuration combinations. T

[jira] [Created] (KAFKA-13456) controller.listener.names is required for all KRaft nodes, not just controllers

2021-11-15 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13456: - Summary: controller.listener.names is required for all KRaft nodes, not just controllers Key: KAFKA-13456 URL: https://issues.apache.org/jira/browse/KAFKA-13456

Re: [VOTE] KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

2021-09-27 Thread Ron Dagostino
Hi Kirk -- thanks again for the KIP! +1 (non-binding) from me. A minor issues I just spotted: - subClaimName: optional override name of the sub claim; defaults to scope s/defaults to scope/defaults to sub/ Ron On Mon, Sep 27, 2021 at 1:20 PM Kirk True wrote: > Hi all! > > I'd like

Re: [VOTE] KIP 771: KRaft brokers should not expose controller metrics

2021-09-07 Thread Ron Dagostino
+1, non-binding from me. Thanks, Ryan. Ron > On Sep 7, 2021, at 5:54 PM, Guozhang Wang wrote: > > Thanks Ryan, > > Read the KIP and it makes sense. +1 as well. > >> On Tue, Sep 7, 2021 at 1:42 PM Colin McCabe wrote: >> >> +1 (binding) >> >> thanks, Ryan >> >> best, >> Colin >> >>> On T

Re: [VOTE] 3.0.0 RC1

2021-09-02 Thread Ron Dagostino
Hi Konstantine. I have opened a probable blocker ticket https://issues.apache.org/jira/browse/KAFKA-13270. I will work on a PR shortly. The description on that ticket is as follows: The implementation of https://issues.apache.org/jira/browse/ZOOKEEPER-3593 in ZooKeeper version 3.6.0 decreased t

[jira] [Created] (KAFKA-13270) Kafka may fail to connect to ZooKeeper, retry forever, and never start

2021-09-02 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13270: - Summary: Kafka may fail to connect to ZooKeeper, retry forever, and never start Key: KAFKA-13270 URL: https://issues.apache.org/jira/browse/KAFKA-13270 Project

Re: [DISCUSS] KIP-771: KRaft broker should not expose controller metrics

2021-08-27 Thread Ron Dagostino
Thanks for the KIP, Ryan. I agree this makes sense. It also reflects the state of affairs right now: KRaft nodes that do not have the controller role currently do not expose these metrics. Assuming this KIP ends up being accepted, we would then close KAFKA-13140 and its associated PR https://git

[jira] [Created] (KAFKA-13224) broker.id does not appear in config's originals map when setting just node.id

2021-08-23 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13224: - Summary: broker.id does not appear in config's originals map when setting just node.id Key: KAFKA-13224 URL: https://issues.apache.org/jira/browse/KAFKA-

Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-08-19 Thread Ron Dagostino
Hi Konstantine. A potential 3.0 blocker was discovered today, https://issues.apache.org/jira/projects/KAFKA/issues/KAFKA-13219. The BrokerState metric is not working for KRaft clusters -- it always indicates that the broker is in the "not running" state. A PR is available at https://github.com/a

[jira] [Created] (KAFKA-13219) BrokerState metric not working for KRaft clusters

2021-08-19 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13219: - Summary: BrokerState metric not working for KRaft clusters Key: KAFKA-13219 URL: https://issues.apache.org/jira/browse/KAFKA-13219 Project: Kafka Issue

Re: [DISCUSS] KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC

2021-08-14 Thread Ron Dagostino
Hi Kirk -- thanks for the KIP! Having concrete implementations out-of-the-box will be very helpful. > As seen in this diagram, the login callback is executed on the client and the validate callback is executed on the broker. There was no diagram when I looked. Maybe there is a broken link or so

[jira] [Created] (KAFKA-13192) broker.id and node.id can be specified inconsistently

2021-08-11 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13192: - Summary: broker.id and node.id can be specified inconsistently Key: KAFKA-13192 URL: https://issues.apache.org/jira/browse/KAFKA-13192 Project: Kafka

Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-07-27 Thread Ron Dagostino
Hi Konstantine. I've opened KAFKA-13137 as a potential blocker. An approved PR is available at https://github.com/apache/kafka/pull/11131. The kafka.controller metrics that the KRaft controllers expose have the wrong MBean names. Ron > On Jul 27, 2021, at 8:13 PM, Ryan Dielhenn > wrote: > > 

[jira] [Created] (KAFKA-13140) KRaft brokers do not expose kafka.controller metrics, breaking backwards compatibility

2021-07-27 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13140: - Summary: KRaft brokers do not expose kafka.controller metrics, breaking backwards compatibility Key: KAFKA-13140 URL: https://issues.apache.org/jira/browse/KAFKA-13140

[jira] [Created] (KAFKA-13137) KRaft Controller Metric MBean names are incorrectly quoted

2021-07-26 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13137: - Summary: KRaft Controller Metric MBean names are incorrectly quoted Key: KAFKA-13137 URL: https://issues.apache.org/jira/browse/KAFKA-13137 Project: Kafka

[jira] [Resolved] (KAFKA-13069) Add magic number to DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde

2021-07-22 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13069?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-13069. --- Resolution: Invalid Flexible fields are sufficient as per KIP-590 VOTE email thread, so a

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2021-07-13 Thread Ron Dagostino
Hi everyone. I know it has been 9 months since the last message appeared on this vote thread, but a potential oversight exists in the implementation of DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde from https://github.com/apache/kafka/pull/9103. Specifically, there is no magic number at the to

[jira] [Created] (KAFKA-13069) Add magic number to DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde

2021-07-12 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-13069: - Summary: Add magic number to DefaultKafkaPrincipalBuilder.KafkaPrincipalSerde Key: KAFKA-13069 URL: https://issues.apache.org/jira/browse/KAFKA-13069 Project

[jira] [Created] (KAFKA-12897) KRaft Controller cannot create topic with more partitions than racks

2021-06-04 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12897: - Summary: KRaft Controller cannot create topic with more partitions than racks Key: KAFKA-12897 URL: https://issues.apache.org/jira/browse/KAFKA-12897 Project

[jira] [Created] (KAFKA-12799) Extend TestSecurityRollingUpgrade system test to KRaft

2021-05-17 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12799: - Summary: Extend TestSecurityRollingUpgrade system test to KRaft Key: KAFKA-12799 URL: https://issues.apache.org/jira/browse/KAFKA-12799 Project: Kafka

Re: [VOTE] KIP-730: Producer ID generation in KRaft mode

2021-05-06 Thread Ron Dagostino
Thanks again for the KIP, David. +1 (non-binding) from me. Ron On Tue, May 4, 2021 at 11:21 AM David Arthur wrote: > Hello everyone, I'd like to start the vote on KIP-730 which adds a new RPC > for producer ID generation in KRaft mode. > > > https://cwiki.apache.org/confluence/display/KAFKA/KI

Re: [DISCUSS] KIP-730: Producer ID generation in KRaft mode

2021-04-16 Thread Ron Dagostino
id, >>> >>> Just putting my paranoid hat here :) Could we name the req/resp name as >>> "AllocateProducerIds" instead of "AllocateProducerId"? Otherwise, LGTM! >>> >>> Guozhang >>> >>>> On Thu, Apr 8, 2021 at

Re: [DISCUSS] KIP-730: Producer ID generation in KRaft mode

2021-04-08 Thread Ron Dagostino
o make it possible. But I figured this is worth mentioning even if we explicitly decide to reject it. Ron On Thu, Apr 8, 2021 at 3:16 PM Ron Dagostino wrote: > > Oh, I see. Yes, my mistake -- I read it wrong. You are right that > all we need in the metadata log is the latest value allocate

Re: [DISCUSS] KIP-730: Producer ID generation in KRaft mode

2021-04-08 Thread Ron Dagostino
t producer Id that > was allocated. It's kind of like a high watermark for producer IDs. This > actually saves us from needing an extra field in the record (the KIP has > just ProducerIdEnd => int64 in the record). > > Does that make sense? > > On Wed, Apr 7, 2021 at 8:44 AM

Re: [DISCUSS] KIP-730: Producer ID generation in KRaft mode

2021-04-07 Thread Ron Dagostino
Thanks for the KIP, David. With the RPC returning a start and length, should the record in the metadata log do the same thing for consistency and to save the byte per record? Ron On Tue, Apr 6, 2021 at 11:06 PM Ismael Juma wrote: > > Great, thanks. Instead of calling it "bridge release", can w

[jira] [Created] (KAFKA-12505) Should kafka-storage.sh accept a non-UUID for its --cluster-id parameter?

2021-03-18 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12505: - Summary: Should kafka-storage.sh accept a non-UUID for its --cluster-id parameter? Key: KAFKA-12505 URL: https://issues.apache.org/jira/browse/KAFKA-12505 Project

[jira] [Created] (KAFKA-12488) Be more specific about enabled SASL mechnanisms in system tests

2021-03-16 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12488: - Summary: Be more specific about enabled SASL mechnanisms in system tests Key: KAFKA-12488 URL: https://issues.apache.org/jira/browse/KAFKA-12488 Project: Kafka

[jira] [Created] (KAFKA-12480) Reuse bootstrap servers in clients when last alive broker in cluster metadata is unavailable

2021-03-16 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12480: - Summary: Reuse bootstrap servers in clients when last alive broker in cluster metadata is unavailable Key: KAFKA-12480 URL: https://issues.apache.org/jira/browse/KAFKA-12480

[jira] [Created] (KAFKA-12455) OffsetValidationTest.test_broker_rolling_bounce failing for Raft quorums

2021-03-11 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12455: - Summary: OffsetValidationTest.test_broker_rolling_bounce failing for Raft quorums Key: KAFKA-12455 URL: https://issues.apache.org/jira/browse/KAFKA-12455 Project

[jira] [Created] (KAFKA-12402) client_sasl_mechanism should be an explicit list instead of a .csv string

2021-03-02 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12402: - Summary: client_sasl_mechanism should be an explicit list instead of a .csv string Key: KAFKA-12402 URL: https://issues.apache.org/jira/browse/KAFKA-12402 Project

[jira] [Created] (KAFKA-12374) Add missing config sasl.mechanism.controller.protocol

2021-02-24 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12374: - Summary: Add missing config sasl.mechanism.controller.protocol Key: KAFKA-12374 URL: https://issues.apache.org/jira/browse/KAFKA-12374 Project: Kafka

[jira] [Created] (KAFKA-12348) The metadata module currently uses Yammer metrics. Should it uses Kafka metrics instead?

2021-02-19 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12348: - Summary: The metadata module currently uses Yammer metrics. Should it uses Kafka metrics instead? Key: KAFKA-12348 URL: https://issues.apache.org/jira/browse/KAFKA-12348

[jira] [Created] (KAFKA-12318) system tests need to fetch Topic IDs via Admin Client instead of via ZooKeeper

2021-02-09 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-12318: - Summary: system tests need to fetch Topic IDs via Admin Client instead of via ZooKeeper Key: KAFKA-12318 URL: https://issues.apache.org/jira/browse/KAFKA-12318

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2020-12-12 Thread Ron Dagostino
Thanks for shepherding this KIP through the extended discussion, Colin. I think we’ve ended up in a good place. I’m sure there will be more tweaks along the way, but the fundamentals are in place. +1 (non-binding) from me. Ron > On Dec 11, 2020, at 4:39 PM, Colin McCabe wrote: > > Hi all,

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-02 Thread Ron Dagostino
> latest committed offset doesn't always advance on every log append. > > > > Given that the new broker epoch won't be visible until the commit has > happened, I have changed this to "the next available offset in the log" > > > 104. REGI

Re: [VOTE] KIP-684 - Support mutual TLS authentication on SASL_SSL listeners

2020-11-16 Thread Ron Dagostino
+1 (non-binding). Thanks, Rajini. Ron > On Nov 16, 2020, at 6:31 AM, Rajini Sivaram wrote: > > Hi all, > > I would like to start vote on KIP-684 to support TLS client authentication > (mTLS) on SASL_SSL listeners: > > - > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-684+-+Sup

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-11-10 Thread Ron Dagostino
names and then the broker will take that value (the first in the list?) and convert that to a security protocol via listener.security.protocol.map. Also, if the security protocol is SASL_{PLAINTEXT,SSL}, what config will defne the SASL mechanism that the broker should use? Ron On Wed, Oct 28, 2020

Re: [DISCUSS] KIP-684 - Support mutual TLS authentication on SASL_SSL listeners

2020-11-09 Thread Ron Dagostino
Hi Rajini. Thanks for the clearly-written KIP. The addition of mTLS for SASL connections makes a lot of sense. It was especially helpful that you documented the history of why we hadn’t supported this in the past and the issue related to backwards compatibility for misconfigured brokers — it

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-28 Thread Ron Dagostino
org.apache.kafka.common.resource.ResourceType enum. Does ConfigRecord.ResourceType need to be a String? Ron On Sun, Oct 25, 2020 at 6:04 AM Ron Dagostino wrote: > > Hi Colin and Jun. > > Regarding these issues: > > 83.1 It seems that the broker can transition from FENCED to RUNNING > without registering for a new b

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-25 Thread Ron Dagostino
assigns the broker a new broker epoch, based on > > the latest committed offset in the log." This seems inaccurate since the > > latest committed offset doesn't always advance on every log append. > > > > 104. REGISTERING(1) : It says "Otherwise, the broker mov

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-22 Thread Ron Dagostino
shutdown vs. having died -- one that dies appears in the metadata as being Offline, whereas one that gracefully shuts down does not. I don't see a way to imply this gracefully-shutdown state, though I may be missing it. Ron On Thu, Oct 22, 2020 at 1:32 PM Ron Dagostino wrote: > > H

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-22 Thread Ron Dagostino
HI Colin. A FencedBrokerRecord appears in the metadata log when a broker is fenced. What appears in the metadata log to indicate that a broker is no longer fenced? Does a BrokerRecord appear? That seems to communicate a bunch of unnecessary data in this context (endpoints, features, rack). If

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-19 Thread Ron Dagostino
Hi Colin. Thanks for the hard work on this KIP. I have some questions about what happens to a broker when it becomes fenced (e.g. because it can't send a heartbeat request to keep its lease). The KIP says "When a broker is fenced, it cannot process any client requests. This prevents brokers whi

[jira] [Created] (KAFKA-10592) system tests not running after python3 merge

2020-10-09 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-10592: - Summary: system tests not running after python3 merge Key: KAFKA-10592 URL: https://issues.apache.org/jira/browse/KAFKA-10592 Project: Kafka Issue Type

Re: [VOTE] KIP-630: Kafka Raft Snapshot

2020-10-05 Thread Ron Dagostino
Thanks for the KIP, Jose. +1 (non-binding) from me. I do have one comment/confusion. Upon re-reading the latest version, I am confused about the name of the proposed "metadata.snapshot.min.records" config. Is this a size, or is it a count? I think it is about a size but want to be sure. I als

[jira] [Created] (KAFKA-10556) NPE if sasl.mechanism is unrecognized

2020-09-30 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-10556: - Summary: NPE if sasl.mechanism is unrecognized Key: KAFKA-10556 URL: https://issues.apache.org/jira/browse/KAFKA-10556 Project: Kafka Issue Type: Task

Re: Why should SASL principal be unchanged upon reauth

2020-09-28 Thread Ron Dagostino
you think there are > portions of the code that work under the assumption that this check exists? > >> On Sun, Sep 27, 2020 at 12:07 AM Ron Dagostino wrote: >> >> Hi Gokul. I looked back at the discussion thread, and it appears it was an >> arbitr

Re: Why should SASL principal be unchanged upon reauth

2020-09-26 Thread Ron Dagostino
Hi Gokul. I looked back at the discussion thread, and it appears it was an arbitrary decision. https://lists.apache.org/thread.html/45c09f226386c0b1dc5f9b36e112882a20414d5900f8d778969e633e%40%3Cdev.kafka.apache.org%3E Ron On Thu, Sep 24, 2020 at 11:03 AM Gokul Ramanan Subramanian < gokul24...@g

[jira] [Resolved] (KAFKA-6664) KIP-269 Substitution Within Configuration Values

2020-09-15 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-6664?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-6664. -- Resolution: Won't Do KIP was not accepted > KIP-269 Substitution Within Configuratio

  1   2   3   >