You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Joel Koshy (JIRA)" <ji...@apache.org> on 2012/09/06 20:35:09 UTC

[jira] [Commented] (KAFKA-203) Improve Kafka internal metrics

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

Joel Koshy commented on KAFKA-203:
----------------------------------

I think this patch looks great and this list of stats is a good start. I
have some minor comments:

1) Rebase - the latest patch applies cleanly to r1378264.

2) The following are just my preferences on naming. What you have should be
   fine, but we should make sure the stat names are as intuitive as
   possible. We should come up with a naming convention for stats and add it
   to our coding convention.

   a) Some timer stats may be better named. E.g., SimpleConsumer
   ConsumerRequestTime will include both request rate and request duration
   which is not very intuitive. OTOH I'm having trouble thinking of a naming
   convention: I would suggest just ConsumerRequestStats - but the size stat
   would be outside then.

   b) Partition.scala:
      ISRExpandRate -> ISRExpandEventRate
      ISRShrinkRate -> ISRShrinkEventRate

   c) Log.scala:
      "LogSegments" -> "NumLogSegments"

   d) ConsumerTopicStat.scala:
      "Total" -> "AllTopics" Also, what if there's a topic called "Total"?
      :) We may want to name this label such that it is an illegal topic
      name (KAFKA-495) - say, "All/Topics".

   e) SimpleConsumer.scala:
      "ConsumerRequestTime" -> see above.

   f) FileMessageSet.scala:
      "LogFlush" -> "LogFlushStats"

   g) RequestChannel.scala:

      i) Instead of "regular" and "follower" how about "consumer" and
      "replica"?

      ii) endRequestTracking -> updateRequestMetrics

      iii) responseComplet (typo)

      iv) For timing stats, may be better to include the unit as part of the
      metric names (e.g., TotalTimeNs).

      v) SendTime -> ResponseSendTime(Ns)

      vi) May be useful to add a comment that simply lays out the phases to
      make the code clearer:
      /* received (start time) -> in queue (queue time) -> dequeued for
      api-local processing -> [api remote processing] -> send response */

   h) AsyncProducerStats.scala:
      DroppedEvent -> DroppedEventsPerSec
      Resentevent -> ResendEventsPerSec
      resents -> resends
      FailedSend -> FailedSendsPerSec
      (or maybe we should just follow a convention: <stat>Rate which
      defaults to <stat> per sec)
      FailedSendtRate (typo)

   i) KafkaApis.scala
      byteInRate -> bytesInRate; byteOutRate -> bytesOutRate
      ExpiresPerSecond -> ExpirationsPerSec

   j) KafkaRequestHandlers.scala
      MessageInPerSec -> IncomingMessagesPerSec

3) There are some places (SimpleConsumer, FileMessageSet, SyncProducer)
   where you use metrics timers. Instead of this:

   val timer = newTimer(...)
   ...
   val ctx = timer.time()
   try {
     // do something
   }
   finally {
     ctx.stop()
   }

   You can use the following equivalent pattern:
   val timer = new KafkaTimer(underlying)
   timer.time {
     // do something
   }

4) ZookeeperConsumerConnector: These JMX operations are actually useful to
   consumers right?

5) DefaultEventHandler: should byte rate be updated here or only after
   sending? Although it does seem useful to have the global byte rate even
   for those that are subsequently dropped.

6) SyncProducer.scala: use KafkaTimer. Also, same comment on naming for
   timers described above.

7) AbstractFetcherThread.scala: FetcherLagMetrics.lock unused.

8) KafkaApis.scala:
   a) Line 108 unused
   b) One caveat in removing the per key ProducerRequestPurgatory stats is
   if there is a key that has an intermittently slow follower you won't be
   able to narrow it down very easily (since the entire request will
   expire). OTOH you will have that stat available from the follower - it's
   just that you will need to "search" for the follower that is causing the
   expirations. So I think it's fine to remove it as it makes the code a lot
   simpler.

9) Pool.scala: good idea.

                
> Improve Kafka internal metrics
> ------------------------------
>
>                 Key: KAFKA-203
>                 URL: https://issues.apache.org/jira/browse/KAFKA-203
>             Project: Kafka
>          Issue Type: New Feature
>          Components: core
>    Affects Versions: 0.8
>            Reporter: Jay Kreps
>            Assignee: Jun Rao
>              Labels: tools
>         Attachments: kafka-203_v1.patch
>
>
> Currently metrics in kafka are using old-school JMX directly. This makes adding metrics a pain. It would be good to do one of the following:
> 1. Convert to Coda Hale's metrics package (https://github.com/codahale/metrics)
> 2. Write a simple metrics package
> The new metrics package should make metrics easier to add and work with and package up the common logic of keeping windowed gauges, histograms, counters, etc. JMX should be just one output of this.
> The advantage of the Coda Hale package is that it exists so we don't need to write it. The downsides are (1) introduces another client dependency which causes conflicts, and (2) seems a bit heavy on design. The good news is that the metrics-core package doesn't seem to bring in a lot of dependencies which is nice, though the scala wrapper seems to want scala 2.9. I am also a little skeptical of the approach for histograms--it does sampling instead of bucketing though that may be okay.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira