[ https://issues.apache.org/jira/browse/KAFKA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16678419#comment-16678419 ]
Randall Hauch commented on KAFKA-6890: -------------------------------------- [~Natengall], apologies for not associating KIP-296 with this issue. Hopefully the link to the KIP here will help people from not making that same mistake. :-) I do understand the use cases for allowing all client configs to be overridden in connectors, but for me it's a matter of usability. In particular, how easy is it to allow users to get into trouble by doing something they don't understand? Do users think they *have* to provide all of the `producer.*` properties in every connector? Ideally the KIP would address these concerns as well as those mentioned above. The KIP is a good start, but there are a few things that it should address. Perhaps the KIP should explain the potential risks of this change (usability, complexity), and address why the benefits outweigh Connect breaking its current abstraction where connectors don't know about Kafka clients. One way to do that is to go into more detail about _multiple_ use cases and to explain what is required today to achieve them and how this proposal enables/simplifies each use case. Consider including the following use cases, though there may be more: * security: the connector uses a specific principal (different from what's in the worker config) * custom partitioners: allow a source connector to use a specific partitioner * tuning: the connector tunes some of the consumer or producer configs for buffers, max bytes, max poll times, etc. * multi-cluster: the connector reads/writes to a different broker cluster than where the connect internal information is stored (including source connector offsets) I think the KIP should also address: # How does someone creating a connector configuration know which `consumer.` or `producer.` properties to specify in a connector config versus which don't need to be defined and inherited instead? Perhaps I missed it, but the KIP should explicitly specify that the `consumer.` and `producer.` properties in the connector configurations _override_ the `consumer.` and `producer.` properties in the worker configuration. # What happens to the `producer.` and `consumer.` properties in a connector configuration? In particular I'm concerned that the connector see and use them inappropriately. It's one thing for Connect's abstraction to be broken at the configuration level, but it's something altogether different to break this abstraction such that the connector implementations start using their own producer and consumer clients. If we were to pass them through to the connector, then any connector that already defines `producer.` or `consumer.` properties might be negatively affected by this change (which would need to be addressed in the compatibility section). # You've mentioned the difference between a Connect operator team and the people that are deploying connectors. I think it is useful for the Connect worker configs to explicitly allow writing to different clusters, so that by default the Connect cluster enforces using only the same cluster. Or can we instead allow the worker to define which "passthrough modes" are allowed, such as a combination of "security", "broker", "tuning", etc.? Or should this be a whitelist regex for all properties that can be overridden? Which approach is more usable? # How does this change affect the actual worker? Does this change the number of producers and consumers that are created for each connector task? (I think the answer is no; this simply changes the properties used to create those clients.) This needs to be outlined in the KIP to allay any concerns about changing behaviors. # "Connector properties that are prefixed with `producer.` and `consumer.` are now 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() invocation, and the remainder of the configuration key will be used as the client configuration key." We should never use "substring" to do this kind of operation. The ConnectorConfig and WorkerConfigs all extend AbstractConfig, which provides an {{originalsWithPrefix(String prefix)}} method that does this, but importantly it internally uses a RecordingMap to record which properties end up being used for later reporting. Look at the [existing code|https://github.com/apache/kafka/blob/70d882861e1bf3eb503c84a31834e8b628de2df9/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java#L143] for how the `producer.` properties are extracted from the worker config. # Is it possible to define `consumer.` properties in a source connector, or `producer.` properties in a sink connector? IMO, Connect should prevent these, but it might be tricky to implement this check correctly such that the validation of the connector config fails in the normal way. Also, is Connect responsible for validating the properties prefixed with `producer.` and `consumer.`? If this KIP is going to allow a connector to override all configurations, then the rejected alternatives section should describe why only allowing a connector to override a subset of properties was rejected. And, a few of the items I mentioned above have multiple options, and anything approach not used should be described in the rejected alternatives section. Keep up the good work. > 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)