Re: [DISCUSS] KIP-753: ACL authentication, Host field support IP network segment

2021-11-26 Thread Luke Chen
Hi Lobo, Thanks for the KIP! I like the idea to allow "IP subnet" to be passed into `--allow-host` option to set for a principle. It will be useful in production environment. Here's some comments: 1. I think "IP subnet" is more specific than "network segment", is that right? 2. Since you allow

[DISCUSS] KIP-804: OfflinePartitionsCount Tagged by Topic

2021-11-26 Thread Mason Legere
Hi All, I would like to start a discussion for KIP-804 , which proposes tagging the offline partition counter metric (managed by the controller) by the topic name of the corresponding offline

[jira] [Resolved] (KAFKA-12791) ConcurrentModificationException in KafkaProducer constructor

2021-11-26 Thread Lucas Bradstreet (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12791?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Bradstreet resolved KAFKA-12791. -- Resolution: Fixed > ConcurrentModificationException in KafkaProducer constructor >

Re: [VOTE] KIP-800: Add reason to LeaveGroupRequest

2021-11-26 Thread Bill Bejeck
Thanks for the KIP, David this seems like it will be very helpful. +1(binding) -Bill On Thu, Nov 25, 2021 at 10:02 AM David Jacot wrote: > Hi Tom, > > I do agree with you. For context, this is the current reason/message logged > by the consumer when enforceRebalance is called so I just kept

Re: KIP-769: Connect API to retrieve connector configuration definitions

2021-11-26 Thread Mickael Maison
Hi Chris, 1. If we want to expose worker plugins, I think we should do it via a separate endpoint. But to be honest, I'm not even sure I see strong use cases for exposing them as they are either enabled or not and can't be changed at runtime. So I'd prefer to stick to "connector level" plugins in

[jira] [Created] (KAFKA-13486) Kafka Connect: Failed to start task due to NPE

2021-11-26 Thread Geliba Uilte (Jira)
Geliba Uilte created KAFKA-13486: Summary: Kafka Connect: Failed to start task due to NPE Key: KAFKA-13486 URL: https://issues.apache.org/jira/browse/KAFKA-13486 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-13485) Restart connectors after RetriableException raised from Task::start()

2021-11-26 Thread Gunnar Morling (Jira)
Gunnar Morling created KAFKA-13485: -- Summary: Restart connectors after RetriableException raised from Task::start() Key: KAFKA-13485 URL: https://issues.apache.org/jira/browse/KAFKA-13485 Project:

Re: Handling retriable exceptions during Connect source task start

2021-11-26 Thread Gunnar Morling
Hi all, We encountered a similar situation in Debezium again, where an exception during Task::start() would be desirable to be retried. Would anything speak against implementing retriable support for Task::start() in Kafka Connect? Would it require a KIP? Thanks, --Gunnar Am Mo., 9. Aug.