[ 
https://issues.apache.org/jira/browse/KAFKA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16677049#comment-16677049
 ] 

Allen Tang commented on KAFKA-6890:
-----------------------------------

I really appreciate your input, Randall! 

The KIP to accompany this JIRA is here: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-296%3A+Connector+level+configurability+for+client+configs
To respond to your concerns, I should also add that defining bootstrap.servers 
at the connector level and then subsequently changing the bootstrap.servers 
configuration at the worker level is actually something that we have performed 
in the past as it was the most graceful solution to what we were trying to 
achieve. We've had an occasion where we were tasked with provisioning an 
entirely new Kafka cluster, migrating all topics from the old cluster to the 
new cluster, and reconfiguring all producers and consumers that were 
interfacing with the old cluster to instead point to the new cluster -- Kafka 
Connect connectors included.

In the absence of a bootstrap.servers overriding capability on a 
connector-by-connector basis, we would only have the worker bootstrap.servers 
to go by, which meant we would have to face a highly coordinated 
inter-departmental effort with over forty connectors within the Kafka Connect 
cluster to account for, along with all of their downstream business-facing 
real-time implications after flipping the cluster-wide switch. 

By providing configurability of bootstrap.servers at the connector-level, 
connectors became decoupled from one another, and they were no longer required 
to read/write data where the internal topics live, allowing for customers to 
migrate their connectors over to the new Kafka cluster at their own pace and on 
their own schedule. Eventually, the three internal topics used for Kafka 
Connect were mirrored over to the new Kafka cluster and the cluster-wide 
boostrap.servers configuration change was applied, thereby fully decoupling 
Kafka Connect from the older Kafka cluster.

If blacklisting overrides for specific client configs, like bootstrap.servers, 
is something you feel strongly about, we may be able to achieve this via 
whitelisting of client configs, defined by administrators of Kafka Connect 
clusters, at the cluster-level within worker properties. Let me know your 
thoughts on this.

> Add connector level configurability for producer/consumer client configs
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-6890
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6890
>             Project: Kafka
>          Issue Type: New Feature
>          Components: KafkaConnect
>            Reporter: Allen Tang
>            Priority: Minor
>
> Right now, each source connector and sink connector inherit their client 
> configurations from the worker properties. Within the worker properties, all 
> configurations that have a prefix of "producer." or "consumer." are applied 
> to all source connectors and sink connectors respectively.
> We should also provide connector-level overrides whereby connector properties 
> that are prefixed with "producer." and "consumer." are used to feed into the 
> producer and consumer clients embedded within source and sink connectors 
> respectively. The prefixes will be removed via a String#substring() call, and 
> the remainder of the connector property key will be used as the client 
> configuration key. The value is fed directly to the client as the 
> configuration value.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to