Thanks all. I've submitted a new PR with a possible implementation: https://github.com/apache/kafka/pull/6097. Note I did not include the group ID as part of the default client ID, mainly to avoid the connector name appearing twice by default. As noted in the original Jira ( https://issues.apache.org/jira/browse/KAFKA-5061), leaving out the group ID could lead to naming conflicts if multiple clusters run the same Kafka cluster. This would probably not be a problem for many (including us) as metrics exporters can usually be configured to include a cluster ID and guarantee uniqueness. Will be interested to hear your thoughts on this.
Paul On Mon, Jan 7, 2019 at 10:27 AM Ryanne Dolan <ryannedo...@gmail.com> wrote: > I'd also prefer to avoid the new configuration property if possible. Seems > like a lighter touch without it. > > Ryanne > > On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson <pdavid...@salesforce.com> > wrote: > > > Hi Konstantine, > > > > Thanks for your feedback! I think my reply to Ewen covers most of your > > points, and I mostly agree. If there is general agreement that changing > > the default behavior is preferable to a config change I will update my PR > > to use that approach. > > > > Paul > > > > On Fri, Jan 4, 2019 at 5:55 PM Konstantine Karantasis < > > konstant...@confluent.io> wrote: > > > > > Hi Paul. > > > > > > I second Ewen and I intended to give similar feedback: > > > > > > 1) Can we avoid a config altogether? > > > 2) If we prefer to add a config anyways, can we use a set of allowed > > values > > > instead of a boolean, even if initially these values are only two? As > the > > > discussion on Jira highlights, there is a potential for more naming > > > conventions in the future, even if now the extra functionality doesn't > > seem > > > essential. It's not optimal to have to deprecate a config instead of > just > > > extending its set of values. > > > 3) I agree, the config name sounds too general. How about > > > "client.ids.naming.policy" or "client.ids.naming" if you want two more > > > options? > > > > > > Konstantine > > > > > > On Fri, Jan 4, 2019 at 7:38 AM Ewen Cheslack-Postava < > e...@confluent.io> > > > wrote: > > > > > > > Hi Paul, > > > > > > > > Thanks for the KIP. A few comments. > > > > > > > > To me, biggest question here is if we can fix this behavior without > > > adding > > > > a config. In particular, today, we don't even set the client.id for > > the > > > > producer and consumer at all, right? The *only* way it is set is if > you > > > > include an override in the worker config, but in that case you need > to > > be > > > > explicitly opting in with a `producer.` or `consumer.` prefix, i.e. > the > > > > settings are `producer.client.id` and `consumer.client.id`. > > Otherwise, I > > > > think we're getting the default behavior where we generate unique, > > > > per-process IDs, i.e. via this logic > > > > > > > > > > > > > > https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L662-L664 > > > > > > > > If that's the case, would it maybe be possible to compatibly change > the > > > > default to use task IDs in the client ID, but only if we don't see an > > > > existing override from the worker config? This would only change the > > > > behavior when someone is using the default, but since the default > would > > > > just use what is effectively a random ID that is useless for > monitoring > > > > metrics, presumably this wouldn't affect any existing users. I think > > that > > > > would avoid having to introduce the config, give better out of the > box > > > > behavior, and still be a safe, compatible change to make. > > > > > > > > > > > > Other than that, just two minor comments. On the config naming, not > > sure > > > > about a better name, but I think the config name could be a bit > clearer > > > if > > > > we need to have it. Maybe something including "task" like > > > > "task.based.client.ids" or something like that (or change the type to > > be > > > an > > > > enum and make it something like task.client.ids=[default|task] and > > leave > > > it > > > > open for extension in the future if needed). > > > > > > > > Finally, you have this: > > > > > > > > *"Allow overriding client.id <http://client.id/> on a per-connector > > > > basis"* > > > > > > > > > > This is a much more complex change, and would require individual > > > > > connectors to be updated to support the change. In contrast, the > > > proposed > > > > > approach would immediately allow detailed consumer/producer > > monitoring > > > > for > > > > > all existing connectors. > > > > > > > > > > > > > I don't think this is quite accurate. I think the reason to reject is > > > that > > > > for your particular requirement for metrics, it simply doesn't give > > > enough > > > > granularity (there's only one value per entire connector), but it > > doesn't > > > > require any changes to connectors. The framework allocates all of > these > > > and > > > > there are already framework-defined config values that all connectors > > > share > > > > (some for only sinks or sources), so the framework can handle all of > > this > > > > without changes to connectors. Further, with connector-specific > > > overrides, > > > > you could get task-specific values if interpolation were supported in > > the > > > > config value (as we now do with managed secrets). For example, it > could > > > > support something like client.id=connector-${taskId} and the task ID > > > would > > > > be substituted automatically into the string. > > > > > > > > I don't necessarily like that solution (seems complicated and not a > > great > > > > user experience), but it could work. > > > > > > > > -Ewen > > > > > > > > > > > > > > > > > > > > On Thu, Dec 20, 2018 at 5:05 PM Paul Davidson < > > pdavid...@salesforce.com> > > > > wrote: > > > > > > > > > Hi everyone, > > > > > > > > > > I would like to start a discussion around the following KIP: > > > > > * > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique > > > > > < > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-411%3A+Add+option+to+make+Kafka+Connect+task+client+ID+values+unique > > > > > >* > > > > > > > > > > This proposes a small change to allow Kafka Connect the option to > > > > > auto-generate unique client IDs for each task. This enables > granular > > > > > monitoring of the producer / consumer client in each task. > > > > > > > > > > Feedback is appreciated, thanks in advance! > > > > > > > > > > Paul > > > > > > > > > > > > > > >