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

2021-11-04 Thread Luke Chen
Hi Artem, Thanks for the KIP! And thanks for reminding me to complete KIP-782, soon. :) Back to the KIP, I have some comments: 1. You proposed to have a new config: "partitioner.sticky.batch.size", but I can't see how we're going to use it to make the partitioner better. Please explain more in

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.1 #4

2021-11-04 Thread Apache Jenkins Server
See Changes: -- [...truncated 496588 lines...] [2021-11-05T00:03:05.815Z] > Task :raft:testClasses UP-TO-DATE [2021-11-05T00:03:05.815Z] > Task :connect:json:testJar

Re: [DISCUSS] KIP-778 KRaft Upgrades

2021-11-04 Thread Jun Rao
Hi, David, Jose and Colin, Thanks for the reply. A few more comments. 12. It seems that we haven't updated the AdminClient accordingly? 14. "Metadata snapshot is generated and sent to the other inactive controllers and to brokers". I thought we wanted each broker to generate its own snapshot

Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.0 #152

2021-11-04 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-786: Emit Metric Client Quota Values

2021-11-04 Thread Mason Legere
Hi All, Thanks for all the comments and suggestions! Addressing them in order: - Have you considered enabling the new metrics by default? > - If you prefer keeping a configuration to enable them, what about > renaming it to "client.quota.value.metric.enable" or even >

Re: [DISCUSS] KIP-786: Emit Metric Client Quota Values

2021-11-04 Thread Bob Barrett
+1 to Tom's point. Having a metric is a lot more convenient than needing to periodically call an API, and anyone who isn't interested in the metric should be able to just not collect it. Thanks for the KIP, Mason! I think this will be very useful. Under the rejected alternatives, you say the

[DISCUSS] KIP-794: Strictly Uniform Sticky Partitioner

2021-11-04 Thread Artem Livshits
Hello, This is the discussion thread for https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner . The proposal is a bug fix for https://issues.apache.org/jira/browse/KAFKA-10888, but it does include a client config change, therefore we have a KIP to

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

2021-11-04 Thread David Mao
Hey Magnus, I noticed that the KIP outlines the initial selectors supported as: - client_instance_id - CLIENT_INSTANCE_ID UUID string representation. - client_software_name - client software implementation name. - client_software_version - client software implementation version. In

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #553

2021-11-04 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-788: Allow configuring num.network.threads per listener

2021-11-04 Thread David Jacot
+1 (binding), thanks for the KIP! Best, David On Thu, Nov 4, 2021 at 12:44 PM Tom Bentley wrote: > > Hi Mickael, > > Thanks for the KIP, +1 (binding). > > Cheers, > > Tom > > On Thu, Nov 4, 2021 at 11:40 AM Rajini Sivaram > wrote: > > > Hi Mickael, > > > > +1 (binding) > > Thanks for the KIP!

Re: [VOTE] KIP-788: Allow configuring num.network.threads per listener

2021-11-04 Thread Tom Bentley
Hi Mickael, Thanks for the KIP, +1 (binding). Cheers, Tom On Thu, Nov 4, 2021 at 11:40 AM Rajini Sivaram wrote: > Hi Mickael, > > +1 (binding) > Thanks for the KIP! > > Regards, > > Rajini > > > On Thu, Nov 4, 2021 at 11:00 AM Mickael Maison > wrote: > > > Hi Luke, > > > > I've updated the

Re: Permissions to contribute to Apache Kafka

2021-11-04 Thread Bruno Cadonna
Hi Vicky, You are all setup now! Best, Bruno On 04.11.21 11:40, Vasiliki Papavasileiou wrote: Wiki ID: vicky_papavas Jira ID: vicky_papavas

Re: [VOTE] KIP-788: Allow configuring num.network.threads per listener

2021-11-04 Thread Rajini Sivaram
Hi Mickael, +1 (binding) Thanks for the KIP! Regards, Rajini On Thu, Nov 4, 2021 at 11:00 AM Mickael Maison wrote: > Hi Luke, > > I've updated the KIP accordingly. > > Thanks > > On Thu, Nov 4, 2021 at 8:41 AM Luke Chen wrote: > > > > Hi Mickael, > > Thanks for the KIP. > > It's great to

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #552

2021-11-04 Thread Apache Jenkins Server
See Changes: -- [...truncated 508625 lines...] [2021-11-04T11:19:55.550Z] [2021-11-04T11:19:55.550Z] > Task :streams:integrationTest [2021-11-04T11:19:55.550Z]

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.0 #151

2021-11-04 Thread Apache Jenkins Server
See Changes: -- [...truncated 345072 lines...] [2021-11-04T11:15:59.786Z] DeleteOffsetsConsumerGroupCommandIntegrationTest >

Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.1 #3

2021-11-04 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-786: Emit Metric Client Quota Values

2021-11-04 Thread Tom Bentley
This is a good point Luke. Unrelatedly, I've considered the value of exposing gauges for the expiry time of SSL certificates, which similarly change rarely. The metrics collected are used to build dashboards, create alerts etc. Thus to have, for example, an alert on approaching certificate expiry,

Re: [VOTE] KIP-788: Allow configuring num.network.threads per listener

2021-11-04 Thread Mickael Maison
Hi Luke, I've updated the KIP accordingly. Thanks On Thu, Nov 4, 2021 at 8:41 AM Luke Chen wrote: > > Hi Mickael, > Thanks for the KIP. > It's great to have the capability to fine tune the number of threads per > listener! > > Just 2 minor comments for the KIP: > 1. The discussion thread is

Permissions to contribute to Apache Kafka

2021-11-04 Thread Vasiliki Papavasileiou
Wiki ID: vicky_papavas Jira ID: vicky_papavas

Re: [DISCUSS] KIP-786: Emit Metric Client Quota Values

2021-11-04 Thread Luke Chen
Hi Mason, Thanks for the KIP. But I think since the quota value won't change from time to time unless admin alter it, it might be waste of resources to record it on each produce/fetch API. It can alternatively be achieved by using the kafka-configs.sh to describe ALL users/clients/default to have

[jira] [Resolved] (KAFKA-13430) Remove broker-wide quota properties from the documentation

2021-11-04 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13430?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-13430. - Fix Version/s: 3.0.0 3.1.0 Reviewer: David Jacot Resolution:

[GitHub] [kafka-site] dajac merged pull request #380: KAFKA-13430: Remove broker-wide quota properties from the documentation

2021-11-04 Thread GitBox
dajac merged pull request #380: URL: https://github.com/apache/kafka-site/pull/380 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail:

[jira] [Created] (KAFKA-13433) JsonConverter's method convertToJson when field is optional with default value and value is null, return default value.

2021-11-04 Thread Aiden Gong (Jira)
Aiden Gong created KAFKA-13433: -- Summary: JsonConverter's method convertToJson when field is optional with default value and value is null, return default value. Key: KAFKA-13433 URL:

Re: [VOTE] KIP-788: Allow configuring num.network.threads per listener

2021-11-04 Thread Luke Chen
Hi Mickael, Thanks for the KIP. It's great to have the capability to fine tune the number of threads per listener! Just 2 minor comments for the KIP: 1. The discussion thread is not attached in KIP 2. Israel raised the case-sensitive comment and your response didn't put into the KIP Otherwise,