The definition for client id has always been "a logical name for an
application which (potentially) spans more than one process".

>From my point of view the rationalization that is most needed is client id
with "user" for the authenticated cases. There not quite the same but
they're similar.

I think all three of those uses are using client id for what it means so I
don't think we should necessarily introduce three more ids. It sounds like
the problem you have is actually that you don't like the default behavior
for JMX when two clients have the same id. Rather than removing the prior
JMX metric maybe you want it to register under some kind of mangled name
("xyz_1", say). Would that work or is there a need for introducing wholly
new concepts?

-Jay

On Fri, Apr 29, 2016 at 1:06 AM, Onur Karaman <onurkaraman.apa...@gmail.com>
wrote:

> Hey everyone. I think we might need to have an actual discussion on an
> issue I brought up a while ago in
> https://issues.apache.org/jira/browse/KAFKA-3494. It seems like client-ids
> are being used for too many things today:
> 1. kafka-request.log. This helps if you ever want to associate a client
> with a specific request. Maybe you're looking for a badly behaved client.
> Maybe the client has reported unexpectedly long response times from the
> broker and you want to figure out what was happening.
> 2. quotas. Quotas today are implemented on a (client-id, broker)
> granularity.
> 3. metrics. KafkaConsumer and KafkaProducer metrics only go as granular as
> the client-id.
>
> The reason I'm bringing this up is because it looks like there's a conflict
> in intent for client-ids between the quota and metrics scenarios. One of
> the motivating factors for choosing the client-id for quotas was that it
> allows for flexibility in the granularity of the quota enforcement. For
> instance, entire services can share the same id to get some form of
> (service, broker) granularity quotas. From my understanding, client-id was
> chosen as the quota id because it's a property that already exists on the
> clients, so we'd be able to quota older clients with no additional work,
> and reusing it had relatively low impact.
>
> So while quotas encourage reuse of client-ids across client instances,
> there is a common scenario where the metrics fall apart and mbeans get
> overwritten. It looks like if there are two KafkaConsumers or two
> KafkaProducers with the same client-id in the same jvm, then JmxReporter
> will unregister the first client's mbeans while registering the second
> client's mbeans.
>
> It seems like for the three use cases noted above (kafka-request.log,
> metrics, quotas), there are different desirable characteristics:
> 1. kafka-request.log at the very least would want an id that could
> distinguish individual client instances, but it might be nice to go even
> more granular at say a per connection level.
> 2. quotas would want an id that's sharable among a group of clients that
> wish to be quotad together. This id can be defined by the user.
> 3. metrics would want an id that could distinguish invidual client
> instance. This id can be defined by the user. We expect it to stay the same
> across process restarts so we can potentially associate metrics across
> process restarts.
>
> To resolve this, I think we'd want metrics to have another tag to
> differentiate mbeans from instances with the same client-id. Another
> alternative is to make quotas depend on a quota id instead of client-id (as
> brought up in KIP-55), but this means we no longer can quota older clients
> out of the box.
>
> Other suggestions are welcome!
>

Reply via email to