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

Jun Rao commented on KAFKA-1481:
--------------------------------

Vladimir,

Thanks for the patch. Really appreciate your help. I realized that this is one 
of the biggest technical debt that we have accumulated over time. So, it may 
take some time to sort this out. So, bear with me. Some more comments.

30. About Taggable, I still have mixed feelings. I can see why you created it. 
However, my reasoning is that for a lot of the case classes (ClientIdTopic, 
CliendIdAndBroker) that we create, it's weird that some of them are taggable 
and some of them are not, depending whether they are used for tagging metric 
names or not. Those classes have no direct relationships with the metrics. 
Similarly, we only need to be aware of tags when creating metrics. Also, 
because of this, we change the constructor of SimpleConsumer. Since this is an 
API change, we should really try to avoid it. 

My feeling is that it's probably simpler if we just create regular case classes 
as before and generate metric tags explicitly when we create the metric. For 
example, in AbstractFetcherThread, we can do

class FetcherStats(clientIdAndBroker: ClientIdAndBroker) extends 
KafkaMetricsGroup {
  val requestRate = newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS,
                                                Map("cliendId" -> 
clientIdAndBroker.clientId,
                                                        "brokerHost" -> 
clientIdAndBroker.host,
                                                        "brokerPort" -> 
clientIdAndBroker.port))

and just have ClientIdAndBroker be the following case class.

case class ClientIdAndBroker(clientId: String, host: String, port: Int)

This way, the code is a bit cleaner since all the metric tag related stuff are 
isolated to those places when the metrics are created. So, I'd suggest that we 
remove Taggable.

31. AbstractFetcherThread:
31.1 You changed the meaning of clientId. clientId is used in the fetch request 
and we want to leave it as just the clientId string. Since the clientId should 
be uniquely representing a particular consumer client, we just need to include 
the clientId in the metric name. We don't need to include the consumer id in 
either the fetch request or the metric name since it's too long and has 
redundant info. 
31.2 FetcherLagStats: This is an existing problem. FetcherLagMetrics shouldn't 
be keyed off ClientIdBrokerTopicPartition. It should be keyed off 
ClientIdTopicPartition. This way, the metric name remains the same independent 
of the current leader of those partitions.

32. ZookeeperConsumerConnector:
32.1 FetchQueueSize: I agree that the metric name just needs to be tagged with 
clientId, topic and threadId. We don't need to include the consumerId since 
it's too long (note that topicThread._2 includes both the consumerId and the 
threadId).

33. KafkaMetricsGroup: Duplicate entries.
    // kafka.consumer.ConsumerTopicStats <-- kafka.consumer.{ConsumerIterator, 
PartitionTopicInfo}
    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", 
"MessagesPerSec"),
    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", 
"MessagesPerSec"),

    // kafka.consumer.ConsumerTopicStats
    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"),
    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"),

    // kafka.consumer.FetchRequestAndResponseStats <-- 
kafka.consumer.SimpleConsumer
    explicitMetricName("kafka.consumer", "FetchRequestAndResponseMetrics", 
"FetchResponseSize"),
    explicitMetricName("kafka.consumer", "FetchRequestAndResponseMetrics", 
"FetchRequestRateAndTimeMs"),
    explicitMetricName("kafka.consumer", "FetchRequestAndResponseMetrics", 
"FetchResponseSize"),
    explicitMetricName("kafka.consumer", "FetchRequestAndResponseMetrics", 
"FetchRequestRateAndTimeMs"),

    /**
     * ProducerRequestStats <-- SyncProducer
     * metric for SyncProducer in fetchTopicMetaData() needs to be removed when 
consumer is closed.
     */
    explicitMetricName("kafka.producer", "ProducerRequestMetrics", 
"ProducerRequestRateAndTimeMs"),
    explicitMetricName("kafka.producer", "ProducerRequestMetrics", 
"ProducerRequestSize"),
    explicitMetricName("kafka.producer", "ProducerRequestMetrics", 
"ProducerRequestRateAndTimeMs"),
    explicitMetricName("kafka.producer", "ProducerRequestMetrics", 
"ProducerRequestSize")

34. AbstractFetcherManager: Could you put the followings in 2 separate lines? 
Similar things happen in a few other files. Perhaps you need to change the 
formatting in your IDE?

   }, metricPrefix.toTags

  private def getFetcherId(topic: String, partitionId: Int) : Int = {    
Utils.abs(31 * topic.hashCode() + partitionId) % numFetchers



> Stop using dashes AND underscores as separators in MBean names
> --------------------------------------------------------------
>
>                 Key: KAFKA-1481
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1481
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 0.8.1.1
>            Reporter: Otis Gospodnetic
>            Priority: Critical
>              Labels: patch
>             Fix For: 0.8.3
>
>         Attachments: KAFKA-1481_2014-06-06_13-06-35.patch, 
> KAFKA-1481_2014-10-13_18-23-35.patch, KAFKA-1481_2014-10-14_21-53-35.patch, 
> KAFKA-1481_2014-10-15_10-23-35.patch, KAFKA-1481_2014-10-20_23-14-35.patch, 
> KAFKA-1481_2014-10-21_09-14-35.patch, 
> KAFKA-1481_2014-10-24_14-14-35.patch.patch, 
> KAFKA-1481_IDEA_IDE_2014-10-14_21-53-35.patch, 
> KAFKA-1481_IDEA_IDE_2014-10-15_10-23-35.patch, 
> KAFKA-1481_IDEA_IDE_2014-10-20_20-14-35.patch, 
> KAFKA-1481_IDEA_IDE_2014-10-20_23-14-35.patch
>
>
> MBeans should not use dashes or underscores as separators because these 
> characters are allowed in hostnames, topics, group and consumer IDs, etc., 
> and these are embedded in MBeans names making it impossible to parse out 
> individual bits from MBeans.
> Perhaps a pipe character should be used to avoid the conflict. 
> This looks like a major blocker because it means nobody can write Kafka 0.8.x 
> monitoring tools unless they are doing it for themselves AND do not use 
> dashes AND do not use underscores.
> See: http://search-hadoop.com/m/4TaT4lonIW



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

Reply via email to