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

Onur Karaman commented on KAFKA-3494:
-------------------------------------

Are you proposing this {{UniqueClientidWithSequence}} actually override the 
client id, or that it just be an additional tag only used by the metrics?

This logic already exists when client id is not defined by the consumer config:
{code}
clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
if (clientId.length() <= 0)
    clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement();
Map<String, String> metricsTags = new LinkedHashMap<String, String>();
metricsTags.put("client-id", clientId);
{code}

> mbeans overwritten with identical clients on a single jvm
> ---------------------------------------------------------
>
>                 Key: KAFKA-3494
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3494
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Onur Karaman
>
> Quotas today are implemented on a (client-id, broker) granularity. I think 
> one of the motivating factors for using a simple quota id was to allow 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 and reusing it 
> had relatively low impact.
> Continuing the above example, let's say a service spins up multiple 
> KafkaConsumers in one jvm sharing the same client-id because they want those 
> consumers to be quotad as a single entity. Sharing client-ids within a single 
> jvm would cause problems in client metrics since the mbeans tags only go as 
> granular as the client-id.
> An easy example is kafka-metrics count. Here's a sample code snippet:
> {code}
> package org.apache.kafka.clients.consumer;
> import java.util.Collections;
> import java.util.Properties;
> import org.apache.kafka.common.TopicPartition;
> public class KafkaConsumerMetrics {
>     public static void main(String[] args) throws InterruptedException {
>         Properties properties = new Properties();
>         properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
> "localhost:9092");
>         properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "test");
>         properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
> "org.apache.kafka.common.serialization.StringDeserializer");
>         
> properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
> "org.apache.kafka.common.serialization.StringDeserializer");
>         properties.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, 
> "testclientid");
>         KafkaConsumer<String, String> kc1 = new KafkaConsumer<>(properties);
>         KafkaConsumer<String, String> kc2 = new KafkaConsumer<>(properties);
>         kc1.assign(Collections.singletonList(new TopicPartition("t1", 0)));
>         while (true) {
>             kc1.poll(1000);
>             System.out.println("kc1 metrics: " + kc1.metrics().size());
>             System.out.println("kc2 metrics: " + kc2.metrics().size());
>             Thread.sleep(1000);
>         }
>     }
> }
> {code}
> jconsole shows one mbean 
> kafka.consumer:type=kafka-metrics-count,client-id=testclientid consistently 
> with value 40.
> but stdout shows:
> {code}
> kc1 metrics: 69
> kc2 metrics: 40
> {code}
> I think the two possible solutions are:
> 1. add finer granularity to the mbeans to distinguish between the clients
> 2. require client ids to be unique per jvm like KafkaStreams has done



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to