[ https://issues.apache.org/jira/browse/KAFKA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16673918#comment-16673918 ]
Peter M Elias commented on KAFKA-6890: -------------------------------------- I found this issue after reading through the source to determine if this was possible and imagined a very similar patch as is proposed here. We have a variety of use cases for this capability. They include throughput tuning, variations in max message sizes, and overriding partitioner implementations. I took a look at the current implementation and the first thing that jumped out was that in order to support this, the broker connection model changes from a single I/O thread per worker to needing a dedicated producer/consumer instance for each overridden connector. While I don't think this is necessarily a bad tradeoff for the config flexibility, it does mean more I/O threads and more network connections and while a small deployment with few connectors may not notice a difference, a larger deployment choosing to run many overridden connectors may need to be warned of the possible increase in overhead. In regards to the possible duplication [~cricket007] has mentioned for KAFKA-4159, I found [this line|https://github.com/apache/kafka/pull/2548/files#diff-316d2c222b623ee65e8065863bf4b9ceR368] on the proposed implementation to be less elegant than [~Natengall]'s approach as it causes state to accumulate on the `producerProps` map which could cause nondeterministic application of prior configuration to connectors not intended to receive said configuration. Isolating the specific overrides and dedicating the particular producer or consumer instance to the connector strikes me as the safest way to do it as proposed here. I am also in favor of blacklisting critical configuration values such as `bootstrap.servers` from being overridden. If that strikes people as too heavy-handed than perhaps a loud warning; after all, the worst case outcome is that the particular connector will fail. Any other connectors will simply continue to use the worker-level configured values assuming we continue with the approach of isolating the overrides to the source/sink task. > 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)