Index: core/src/main/scala/kafka/server/ReplicaManager.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/ReplicaManager.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/ReplicaManager.scala	(revision )
@@ -34,7 +34,6 @@
 import scala.collection.mutable.HashMap
 import scala.collection.Map
 import scala.collection.Set
-import scala.Some
 
 import org.I0Itec.zkclient.ZkClient
 import com.yammer.metrics.core.Gauge
@@ -87,8 +86,8 @@
       def value = underReplicatedPartitionCount()
     }
   )
-  val isrExpandRate = newMeter("IsrExpandsPerSec",  "expands", TimeUnit.SECONDS)
+  val isrExpandRate = newMeter("IsrExpandsPerSec", "expands", TimeUnit.SECONDS)
-  val isrShrinkRate = newMeter("IsrShrinksPerSec",  "shrinks", TimeUnit.SECONDS)
+  val isrShrinkRate = newMeter("IsrShrinksPerSec", "shrinks", TimeUnit.SECONDS)
 
   def underReplicatedPartitionCount(): Int = {
       getLeaderPartitions().count(_.isUnderReplicated)
@@ -248,7 +247,7 @@
         val partitionDataAndOffsetInfo =
           try {
             val (fetchInfo, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId)
-            BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes)
+            BrokerTopicStats.getBrokerTopicStats(new TaggableInfo("topic" -> topic)).bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes)
             BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes)
             if (isFetchFromFollower) {
               debug("Partition [%s,%d] received fetch request from follower %d"
@@ -268,7 +267,7 @@
                 fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage))
               new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata)
             case t: Throwable =>
-              BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
+              BrokerTopicStats.getBrokerTopicStats(new TaggableInfo("topic" -> topic)).failedFetchRequestRate.mark()
               BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark()
               error("Error when processing fetch request for partition [%s,%d] offset %d from %s with correlation id %d. Possible cause: %s"
                 .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId, t.getMessage))
Index: core/src/main/scala/kafka/server/ReplicaFetcherManager.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/ReplicaFetcherManager.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/ReplicaFetcherManager.scala	(revision )
@@ -18,17 +18,17 @@
 package kafka.server
 
 import kafka.cluster.Broker
+import kafka.common.TaggableInfo
 
 class ReplicaFetcherManager(private val brokerConfig: KafkaConfig, private val replicaMgr: ReplicaManager)
         extends AbstractFetcherManager("ReplicaFetcherManager on broker " + brokerConfig.brokerId,
-                                       "Replica", brokerConfig.numReplicaFetchers) {
+          new TaggableInfo("source" -> "Replica"), brokerConfig.numReplicaFetchers) {
 
   override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = {
-    new ReplicaFetcherThread("ReplicaFetcherThread-%d-%d".format(fetcherId, sourceBroker.id), sourceBroker, brokerConfig, replicaMgr)
+    new ReplicaFetcherThread(new TaggableInfo("threadName" -> "ReplicaFetcherThread", "fetcherId" -> fetcherId.toString, "sourceBrokerId" -> sourceBroker.id.toString), sourceBroker, brokerConfig, replicaMgr)
   }
 
-  def shutdown() {
-    info("shutting down")
+  def shutdown() {    info("shutting down")
     closeAllFetchers()
     info("shutdown completed")
   }  
\ No newline at end of file
Index: core/src/test/scala/unit/kafka/integration/FetcherTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/integration/FetcherTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/integration/FetcherTest.scala	(revision )
@@ -19,6 +19,8 @@
 
 import java.util.concurrent._
 import java.util.concurrent.atomic._
+import kafka.common.TaggableInfo
+
 import scala.collection._
 import junit.framework.Assert._
 
@@ -48,7 +50,7 @@
                                                            new AtomicLong(0),
                                                            new AtomicLong(0),
                                                            new AtomicInteger(0),
-                                                           ""))
+                                                           new TaggableInfo("clientId"->"")))
 
   var fetcher: ConsumerFetcherManager = null
 
@@ -56,7 +58,7 @@
     super.setUp
     createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers)
 
-    fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient)
+    fetcher = new ConsumerFetcherManager(new ConsumerId("consumer1"), new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient)
     fetcher.stopConnections()
     fetcher.startConnections(topicInfos, cluster)
   }
Index: core/src/main/scala/kafka/server/DelayedRequestKey.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/DelayedRequestKey.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/DelayedRequestKey.scala	(revision )
@@ -6,7 +6,7 @@
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -17,22 +17,21 @@
 
 package kafka.server
 
-import kafka.common.TopicAndPartition
+import kafka.common.{TopicAllInfo, TaggableInfo, Taggable, TopicAndPartition}
 
 /**
  * Keys used for delayed request metrics recording
  */
 trait DelayedRequestKey {
-  def keyLabel: String
+  def keyLabel: Taggable
 }
 
 object DelayedRequestKey {
-  val globalLabel = "All"
+  val globalLabel = TopicAllInfo()
 }
 
 case class TopicPartitionRequestKey(topic: String, partition: Int) extends DelayedRequestKey {
-
   def this(topicAndPartition: TopicAndPartition) = this(topicAndPartition.topic, topicAndPartition.partition)
 
-  override def keyLabel = "%s-%d".format(topic, partition)
+  override def keyLabel = new TaggableInfo("topic" -> topic, "partitionId" -> partition.toString)
 }
Index: core/src/main/scala/kafka/javaapi/OffsetRequest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/javaapi/OffsetRequest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/javaapi/OffsetRequest.scala	(revision )
@@ -20,7 +20,6 @@
 import kafka.common.TopicAndPartition
 import kafka.api.{Request, PartitionOffsetRequestInfo}
 import scala.collection.mutable
-import java.nio.ByteBuffer
 
 
 class OffsetRequest(requestInfo: java.util.Map[TopicAndPartition, PartitionOffsetRequestInfo],
Index: core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala	(revision )
@@ -17,6 +17,7 @@
 
 package kafka.log4j
 
+import kafka.common.{TaggableInfo, BrokerSpecificInfo}
 import kafka.consumer.SimpleConsumer
 import kafka.server.{KafkaConfig, KafkaServer}
 import kafka.utils.{TestUtils, Utils, Logging}
@@ -59,7 +60,7 @@
     logDirZk = new File(logDirZkPath)
     config = new KafkaConfig(propsZk)
     server = TestUtils.createServer(config)
-    simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64 * 1024, "")
+    simpleConsumerZk = new SimpleConsumer(new BrokerSpecificInfo("localhost", portZk), 1000000, 64 * 1024, new TaggableInfo("clientId"->""))
   }
 
   @After
Index: core/src/main/scala/kafka/network/RequestChannel.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/network/RequestChannel.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/network/RequestChannel.scala	(revision )
@@ -22,7 +22,7 @@
 import com.yammer.metrics.core.Gauge
 import java.nio.ByteBuffer
 import kafka.api._
-import kafka.common.TopicAndPartition
+import kafka.common.{TaggableInfo, Taggable, TopicAndPartition}
 import kafka.utils.{Logging, SystemTime}
 import kafka.message.ByteBufferMessageSet
 import java.net._
@@ -125,12 +125,12 @@
     def value = responseQueues.foldLeft(0) {(total, q) => total + q.size()}
   })
 
-  for(i <- 0 until numProcessors) {
+  for (i <- 0 until numProcessors) {
     newGauge(
-      "Processor-" + i + "-ResponseQueueSize",
+      "ResponseQueueSize",
       new Gauge[Int] {
         def value = responseQueues(i).size()
-      }
+      }, Map("ProcessorNum" -> i.toString)
     )
   }
 
@@ -186,25 +186,25 @@
 }
 
 object RequestMetrics {
-  val metricsMap = new scala.collection.mutable.HashMap[String, RequestMetrics]
-  val consumerFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "-Consumer"
-  val followFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "-Follower"
+  val metricsMap = new scala.collection.mutable.HashMap[Taggable, RequestMetrics]
+  val consumerFetchMetricName = new TaggableInfo("request" -> "FetchConsumer")
+  val followFetchMetricName = new TaggableInfo("request" -> "FetchFollower")
   (RequestKeys.keyToNameAndDeserializerMap.values.map(e => e._1)
     ++ List(consumerFetchMetricName, followFetchMetricName)).foreach(name => metricsMap.put(name, new RequestMetrics(name)))
 }
 
-class RequestMetrics(name: String) extends KafkaMetricsGroup {
-  val requestRate = newMeter(name + "-RequestsPerSec",  "requests", TimeUnit.SECONDS)
+class RequestMetrics(name: Taggable) extends KafkaMetricsGroup {
+  val requestRate = newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, name.toTags)
   // time a request spent in a request queue
-  val requestQueueTimeHist = newHistogram(name + "-RequestQueueTimeMs")
+  val requestQueueTimeHist = newHistogram("RequestQueueTimeMs", biased = true, name.toTags)
   // time a request takes to be processed at the local broker
-  val localTimeHist = newHistogram(name + "-LocalTimeMs")
+  val localTimeHist = newHistogram("LocalTimeMs", biased = true, name.toTags)
   // time a request takes to wait on remote brokers (only relevant to fetch and produce requests)
-  val remoteTimeHist = newHistogram(name + "-RemoteTimeMs")
+  val remoteTimeHist = newHistogram("RemoteTimeMs", biased = true, name.toTags)
   // time a response spent in a response queue
-  val responseQueueTimeHist = newHistogram(name + "-ResponseQueueTimeMs")
+  val responseQueueTimeHist = newHistogram("ResponseQueueTimeMs", biased = true, name.toTags)
   // time to send the response to the requester
-  val responseSendTimeHist = newHistogram(name + "-ResponseSendTimeMs")
-  val totalTimeHist = newHistogram(name + "-TotalTimeMs")
+  val responseSendTimeHist = newHistogram("ResponseSendTimeMs", biased = true, name.toTags)
+  val totalTimeHist = newHistogram("TotalTimeMs", biased = true, name.toTags)
 }
 
Index: core/src/main/scala/kafka/log/LogCleaner.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/log/LogCleaner.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/log/LogCleaner.scala	(revision )
@@ -23,11 +23,9 @@
 import kafka.metrics.KafkaMetricsGroup
 
 import scala.collection._
-import scala.math
 import java.nio._
 import java.util.Date
 import java.io.File
-import java.lang.IllegalStateException
 import java.util.concurrent.CountDownLatch
 import java.util.concurrent.TimeUnit
 
@@ -78,7 +76,7 @@
   private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, 
                                         checkIntervalMs = 300, 
                                         throttleDown = true, 
-                                        "cleaner-io",
+                                        "CleanerIO",
                                         "bytes",
                                         time = time)
   
@@ -86,12 +84,12 @@
   private val cleaners = (0 until config.numThreads).map(new CleanerThread(_))
   
   /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */
-  newGauge("max-buffer-utilization-percent", 
+  newGauge("MaxBufferUtilizationPercent",
            new Gauge[Int] {
              def value: Int = cleaners.map(_.lastStats).map(100 * _.bufferUtilization).max.toInt
            })
   /* a metric to track the recopy rate of each thread's last cleaning */
-  newGauge("cleaner-recopy-percent", 
+  newGauge("CleanerRecopyPercent",
            new Gauge[Int] {
              def value: Int = {
                val stats = cleaners.map(_.lastStats)
@@ -100,11 +98,10 @@
              }
            })
   /* a metric to track the maximum cleaning time for the last cleaning from each thread */
-  newGauge("max-clean-time-secs",
+  newGauge("MaxCleanTimeSecs",
            new Gauge[Int] {
              def value: Int = cleaners.map(_.lastStats).map(_.elapsedSecs).max.toInt
-           })
+           })  
-  
   /**
    * Start the background cleaning
    */
\ No newline at end of file
Index: core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala	(revision )
@@ -27,10 +27,12 @@
 import kafka.api._
 import java.text.SimpleDateFormat
 import java.util.Date
-import kafka.common.{ErrorMapping, TopicAndPartition}
+import kafka.common.{TaggableInfo, BrokerSpecificInfo, ErrorMapping, TopicAndPartition}
 import kafka.utils._
 import kafka.consumer.{ConsumerConfig, Whitelist, SimpleConsumer}
 
+import scala.collection.mutable
+
 /**
  *  For verifying the consistency among replicas.
  *
@@ -237,7 +239,7 @@
   private def setInitialOffsets() {
     for ((brokerId, topicAndPartitions) <- leadersPerBroker) {
       val broker = brokerMap(brokerId)
-      val consumer = new SimpleConsumer(broker.host, broker.port, 10000, 100000, ReplicaVerificationTool.clientId)
+      val consumer = new SimpleConsumer(new BrokerSpecificInfo(broker.host, broker.port), 10000, 100000, new TaggableInfo("clientId"->ReplicaVerificationTool.clientId))
       val initialOffsetMap: Map[TopicAndPartition, PartitionOffsetRequestInfo] =
         topicAndPartitions.map(topicAndPartition => topicAndPartition -> PartitionOffsetRequestInfo(initialOffsetTime, 1)).toMap
       val offsetRequest = OffsetRequest(initialOffsetMap)
@@ -339,11 +341,10 @@
                              replicaBuffer: ReplicaBuffer, socketTimeout: Int, socketBufferSize: Int,
                              fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean)
   extends ShutdownableThread(name) {
-  val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, ReplicaVerificationTool.clientId)
+  val simpleConsumer = new SimpleConsumer(new BrokerSpecificInfo(sourceBroker.host, sourceBroker.port), socketTimeout, socketBufferSize, TaggableInfo(mutable.LinkedHashMap("clientId" -> ReplicaVerificationTool.clientId)))
   val fetchRequestBuilder = new FetchRequestBuilder().
           clientId(ReplicaVerificationTool.clientId).
-          replicaId(Request.DebuggingConsumerId).
-          maxWait(maxWait).
+          replicaId(Request.DebuggingConsumerId).          maxWait(maxWait).
           minBytes(minBytes)
 
   override def doWork() {
\ No newline at end of file
Index: core/src/main/scala/kafka/common/ClientIdAndTopic.scala
===================================================================
--- core/src/main/scala/kafka/common/ClientIdAndTopic.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/common/TopicInfo.scala	(revision )
@@ -1,5 +1,7 @@
 package kafka.common
 
+import scala.collection.mutable
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -8,7 +10,7 @@
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -17,11 +19,13 @@
  * limitations under the License.
  */
 
-/**
- * Convenience case class since (clientId, topic) pairs are used in the creation
- * of many Stats objects.
- */
-case class ClientIdAndTopic(clientId: String, topic: String) {
-  override def toString = "%s-%s".format(clientId, topic)
+trait TopicInfo extends Taggable {
 }
 
+case class TopicSpecificInfo(topic: String) extends TopicInfo {
+  override def toTags = mutable.LinkedHashMap("topic" -> topic)
+}
+
+case class TopicAllInfo() extends TopicInfo {
+  override def toTags = mutable.LinkedHashMap("allTopics" -> "true")
+}
Index: core/src/main/scala/kafka/consumer/KafkaStream.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/KafkaStream.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/KafkaStream.scala	(revision )
@@ -19,6 +19,7 @@
 
 
 import java.util.concurrent.BlockingQueue
+import kafka.common.Taggable
 import kafka.serializer.Decoder
 import kafka.message.MessageAndMetadata
 
@@ -26,7 +27,7 @@
                         consumerTimeoutMs: Int,
                         private val keyDecoder: Decoder[K],
                         private val valueDecoder: Decoder[V],
-                        val clientId: String)
+                        val clientId: Taggable)
    extends Iterable[MessageAndMetadata[K,V]] with java.lang.Iterable[MessageAndMetadata[K,V]] {
 
   private val iter: ConsumerIterator[K,V] =
Index: core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala	(revision )
@@ -17,7 +17,7 @@
 
 package kafka.api
 
-import kafka.common.Topic
+import kafka.common.{BrokerSpecificInfo, TaggableInfo, Topic}
 import org.apache.kafka.common.errors.{InvalidTopicException,NotEnoughReplicasException}
 import org.scalatest.junit.JUnit3Suite
 import org.junit.Test
@@ -64,8 +64,8 @@
     super.setUp()
 
     // TODO: we need to migrate to new consumers when 0.9 is final
-    consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "")
-    consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "")
+    consumer1 = new SimpleConsumer(new BrokerSpecificInfo("localhost", configs(0).port), 100, 1024 * 1024, new TaggableInfo("clientId" -> ""))
+    consumer2 = new SimpleConsumer(new BrokerSpecificInfo("localhost", configs(1).port), 100, 1024 * 1024, new TaggableInfo("clientId" -> ""))
 
     producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize)
     producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize)
Index: core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala	(revision )
@@ -6,7 +6,7 @@
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -21,7 +21,8 @@
 import java.util.concurrent.TimeUnit
 
 import com.yammer.metrics.Metrics
-import com.yammer.metrics.core.{Gauge, MetricName}
+import com.yammer.metrics.core._
+import kafka.common.Taggable
 import kafka.consumer.{ConsumerTopicStatsRegistry, FetchRequestAndResponseStatsRegistry}
 import kafka.producer.{ProducerRequestStatsRegistry, ProducerStatsRegistry, ProducerTopicStatsRegistry}
 import kafka.utils.Logging
@@ -35,32 +36,65 @@
    * Creates a new MetricName object for gauges, meters, etc. created for this
    * metrics group.
    * @param name Descriptive name of the metric.
+   * @param tags Additional attributes which mBean will have.
    * @return Sanitized metric name object.
    */
-  private def metricName(name: String) = {
+  private def metricName(name: String, tags: scala.collection.Map[String, String] = Map.empty) = {
     val klass = this.getClass
     val pkg = if (klass.getPackage == null) "" else klass.getPackage.getName
     val simpleName = klass.getSimpleName.replaceAll("\\$$", "")
-    new MetricName(pkg, simpleName, name)
+
+    explicitMetricName(pkg, simpleName, name, tags)
   }
 
-  def newGauge[T](name: String, metric: Gauge[T]) =
-    Metrics.defaultRegistry().newGauge(metricName(name), metric)
 
-  def newMeter(name: String, eventType: String, timeUnit: TimeUnit) =
-    Metrics.defaultRegistry().newMeter(metricName(name), eventType, timeUnit)
+  private def explicitMetricName(group: String, typeName: String, name: String, tags: scala.collection.Map[String, String] = Map.empty) = {
+    val nameBuilder: StringBuilder = new StringBuilder
 
-  def newHistogram(name: String, biased: Boolean = true) =
-    Metrics.defaultRegistry().newHistogram(metricName(name), biased)
+    nameBuilder.append(group)
 
-  def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit) =
-    Metrics.defaultRegistry().newTimer(metricName(name), durationUnit, rateUnit)
+    nameBuilder.append(":type=")
 
-  def removeMetric(name: String) =
-    Metrics.defaultRegistry().removeMetric(metricName(name))
+    nameBuilder.append(typeName)
 
+    if (name.length > 0) {
+      nameBuilder.append(",name=")
+      nameBuilder.append(name)
-}
+    }
 
+    if (tags.nonEmpty) {
+      val tagsString = tags
+        .filter(_._2 != "").map(kv => "%s=%s".format(kv._1, kv._2))
+        .mkString(",")
+
+      if (tagsString != "") {
+        nameBuilder.append(",")
+      }
+
+      nameBuilder.append(tagsString)
+    }
+
+    new MetricName(group, typeName, name, null, nameBuilder.toString())
+  }
+
+  def newGauge[T](name: String, metric: Gauge[T], tags: scala.collection.Map[String, String] = Map.empty) =
+    Metrics.defaultRegistry().newGauge(metricName(name, tags), metric)
+
+  def newMeter(name: String, eventType: String, timeUnit: TimeUnit, tags: scala.collection.Map[String, String] = Map.empty) =
+    Metrics.defaultRegistry().newMeter(metricName(name, tags), eventType, timeUnit)
+
+  def newHistogram(name: String, biased: Boolean = true, tags: scala.collection.Map[String, String] = Map.empty) =
+    Metrics.defaultRegistry().newHistogram(metricName(name, tags), biased)
+
+  def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit, tags: scala.collection.Map[String, String] = Map.empty) =
+    Metrics.defaultRegistry().newTimer(metricName(name, tags), durationUnit, rateUnit)
+
+  def removeMetric(name: String, tags: scala.collection.Map[String, String] = Map.empty) =
+    Metrics.defaultRegistry().removeMetric(metricName(name, tags))
+
+
+}
+
 object KafkaMetricsGroup extends KafkaMetricsGroup with Logging {
   /**
    * To make sure all the metrics be de-registered after consumer/producer close, the metric names should be
@@ -68,100 +102,101 @@
    */
   private val consumerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName](
     // kafka.consumer.ZookeeperConsumerConnector
-    new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-FetchQueueSize"),
-    new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-KafkaCommitsPerSec"),
-    new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-ZooKeeperCommitsPerSec"),
-    new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-RebalanceRateAndTime"),
-    new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-OwnedPartitionsCount"),
-    new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "AllTopicsOwnedPartitionsCount"),
+    explicitMetricName("kafka.consumer", "ZookeeperConsumerConnector", "FetchQueueSize"),
+    explicitMetricName("kafka.consumer", "ZookeeperConsumerConnector", "KafkaCommitsPerSec"),
+    explicitMetricName("kafka.consumer", "ZookeeperConsumerConnector", "ZooKeeperCommitsPerSec"),
+    explicitMetricName("kafka.consumer", "ZookeeperConsumerConnector", "RebalanceRateAndTime"),
+    explicitMetricName("kafka.consumer", "ZookeeperConsumerConnector", "OwnedPartitionsCount"),
+    explicitMetricName("kafka.consumer", "ZookeeperConsumerConnector", "OwnedPartitionsCount"),
 
     // kafka.consumer.ConsumerFetcherManager
-    new MetricName("kafka.consumer", "ConsumerFetcherManager", "-MaxLag"),
-    new MetricName("kafka.consumer", "ConsumerFetcherManager", "-MinFetchRate"),
+    explicitMetricName("kafka.consumer", "ConsumerFetcherManager", "MaxLag"),
+    explicitMetricName("kafka.consumer", "ConsumerFetcherManager", "MinFetchRate"),
 
     // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread
-    new MetricName("kafka.server", "FetcherLagMetrics", "-ConsumerLag"),
+    explicitMetricName("kafka.server", "FetcherLagMetrics", "ConsumerLag"),
 
     // kafka.consumer.ConsumerTopicStats <-- kafka.consumer.{ConsumerIterator, PartitionTopicInfo}
-    new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-MessagesPerSec"),
-    new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-AllTopicsMessagesPerSec"),
+    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", "MessagesPerSec"),
+    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", "MessagesPerSec"),
 
     // kafka.consumer.ConsumerTopicStats
-    new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-BytesPerSec"),
-    new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-AllTopicsBytesPerSec"),
+    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"),
+    explicitMetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"),
 
     // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread
-    new MetricName("kafka.server", "FetcherStats", "-BytesPerSec"),
-    new MetricName("kafka.server", "FetcherStats", "-RequestsPerSec"),
+    explicitMetricName("kafka.server", "FetcherStats", "BytesPerSec"),
+    explicitMetricName("kafka.server", "FetcherStats", "RequestsPerSec"),
 
     // kafka.consumer.FetchRequestAndResponseStats <-- kafka.consumer.SimpleConsumer
-    new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-FetchResponseSize"),
-    new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-FetchRequestRateAndTimeMs"),
-    new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-AllBrokersFetchResponseSize"),
-    new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-AllBrokersFetchRequestRateAndTimeMs"),
+    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.
      */
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestRateAndTimeMs"),
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestSize"),
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestRateAndTimeMs"),
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestSize")
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"),
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize"),
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"),
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize")
   )
 
-  private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName] (
+  private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName](
     // kafka.producer.ProducerStats <-- DefaultEventHandler <-- Producer
-    new MetricName("kafka.producer", "ProducerStats", "-SerializationErrorsPerSec"),
-    new MetricName("kafka.producer", "ProducerStats", "-ResendsPerSec"),
-    new MetricName("kafka.producer", "ProducerStats", "-FailedSendsPerSec"),
+    explicitMetricName("kafka.producer", "ProducerStats", "SerializationErrorsPerSec"),
+    explicitMetricName("kafka.producer", "ProducerStats", "ResendsPerSec"),
+    explicitMetricName("kafka.producer", "ProducerStats", "FailedSendsPerSec"),
 
     // kafka.producer.ProducerSendThread
-    new MetricName("kafka.producer.async", "ProducerSendThread", "-ProducerQueueSize"),
+    explicitMetricName("kafka.producer.async", "ProducerSendThread", "ProducerQueueSize"),
 
     // kafka.producer.ProducerTopicStats <-- kafka.producer.{Producer, async.DefaultEventHandler}
-    new MetricName("kafka.producer", "ProducerTopicMetrics", "-MessagesPerSec"),
-    new MetricName("kafka.producer", "ProducerTopicMetrics", "-DroppedMessagesPerSec"),
-    new MetricName("kafka.producer", "ProducerTopicMetrics", "-BytesPerSec"),
-    new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsMessagesPerSec"),
-    new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsDroppedMessagesPerSec"),
-    new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsBytesPerSec"),
+    explicitMetricName("kafka.producer", "ProducerTopicMetrics", "MessagesPerSec"),
+    explicitMetricName("kafka.producer", "ProducerTopicMetrics", "DroppedMessagesPerSec"),
+    explicitMetricName("kafka.producer", "ProducerTopicMetrics", "BytesPerSec"),
+    explicitMetricName("kafka.producer", "ProducerTopicMetrics", "MessagesPerSec"),
+    explicitMetricName("kafka.producer", "ProducerTopicMetrics", "DroppedMessagesPerSec"),
+    explicitMetricName("kafka.producer", "ProducerTopicMetrics", "BytesPerSec"),
 
     // kafka.producer.ProducerRequestStats <-- SyncProducer
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestRateAndTimeMs"),
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestSize"),
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestRateAndTimeMs"),
-    new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestSize")
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"),
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize"),
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"),
+    explicitMetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize")
   )
 
-  def removeAllConsumerMetrics(clientId: String) {
+  def removeAllConsumerMetrics(clientId: Taggable) {
     FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId)
     ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId)
     ProducerRequestStatsRegistry.removeProducerRequestStats(clientId)
     removeAllMetricsInList(KafkaMetricsGroup.consumerMetricNameList, clientId)
   }
 
-  def removeAllProducerMetrics(clientId: String) {
+  def removeAllProducerMetrics(clientId: Taggable) {
     ProducerRequestStatsRegistry.removeProducerRequestStats(clientId)
     ProducerTopicStatsRegistry.removeProducerTopicStats(clientId)
     ProducerStatsRegistry.removeProducerStats(clientId)
     removeAllMetricsInList(KafkaMetricsGroup.producerMetricNameList, clientId)
   }
 
-  private def removeAllMetricsInList(metricNameList: immutable.List[MetricName], clientId: String) {
+  private def removeAllMetricsInList(metricNameList: immutable.List[MetricName], clientId: Taggable) {
     metricNameList.foreach(metric => {
-      val pattern = (clientId + ".*" + metric.getName +".*").r
+      val pattern = (".*" + clientId + ".*").r
       val registeredMetrics = scala.collection.JavaConversions.asScalaSet(Metrics.defaultRegistry().allMetrics().keySet())
       for (registeredMetric <- registeredMetrics) {
         if (registeredMetric.getGroup == metric.getGroup &&
+          registeredMetric.getName == metric.getName &&
           registeredMetric.getType == metric.getType) {
-          pattern.findFirstIn(registeredMetric.getName) match {
+          pattern.findFirstIn(registeredMetric.getMBeanName) match {
             case Some(_) => {
               val beforeRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size
               Metrics.defaultRegistry().removeMetric(registeredMetric)
               val afterRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size
               trace("Removing metric %s. Metrics registry size reduced from %d to %d".format(
-                  registeredMetric, beforeRemovalSize, afterRemovalSize))
+                registeredMetric, beforeRemovalSize, afterRemovalSize))
             }
             case _ =>
           }
Index: core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala	(revision )
@@ -19,28 +19,28 @@
 
 import java.util.concurrent.TimeUnit
 
-import kafka.common.ClientIdAndBroker
+import kafka.common.{Taggable, BrokerAllInfo, BrokerInfo, ClientIdAndBroker}
 import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
 import kafka.utils.Pool
 
 class FetchRequestAndResponseMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup {
-  val requestTimer = new KafkaTimer(newTimer(metricId + "FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
-  val requestSizeHist = newHistogram(metricId + "FetchResponseSize")
+  val requestTimer = new KafkaTimer(newTimer("FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, metricId.toTags))
+  val requestSizeHist = newHistogram("FetchResponseSize", biased = true, metricId.toTags)
 }
 
 /**
  * Tracks metrics of the requests made by a given consumer client to all brokers, and the responses obtained from the brokers.
  * @param clientId ClientId of the given consumer
  */
-class FetchRequestAndResponseStats(clientId: String) {
+class FetchRequestAndResponseStats(clientId: Taggable) {
   private val valueFactory = (k: ClientIdAndBroker) => new FetchRequestAndResponseMetrics(k)
   private val stats = new Pool[ClientIdAndBroker, FetchRequestAndResponseMetrics](Some(valueFactory))
-  private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAndBroker(clientId, "AllBrokers"))
+  private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAndBroker(clientId, new BrokerAllInfo()))
 
   def getFetchRequestAndResponseAllBrokersStats(): FetchRequestAndResponseMetrics = allBrokersStats
 
-  def getFetchRequestAndResponseStats(brokerInfo: String): FetchRequestAndResponseMetrics = {
-    stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo + "-"))
+  def getFetchRequestAndResponseStats(brokerInfo: BrokerInfo): FetchRequestAndResponseMetrics = {
+    stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo))
   }
 }
 
@@ -48,18 +48,18 @@
  * Stores the fetch request and response stats information of each consumer client in a (clientId -> FetchRequestAndResponseStats) map.
  */
 object FetchRequestAndResponseStatsRegistry {
-  private val valueFactory = (k: String) => new FetchRequestAndResponseStats(k)
-  private val globalStats = new Pool[String, FetchRequestAndResponseStats](Some(valueFactory))
+  private val valueFactory = (k: Taggable) => new FetchRequestAndResponseStats(k)
+  private val globalStats = new Pool[Taggable, FetchRequestAndResponseStats](Some(valueFactory))
 
-  def getFetchRequestAndResponseStats(clientId: String) = {
+  def getFetchRequestAndResponseStats(clientId: Taggable) = {
     globalStats.getAndMaybePut(clientId)
   }
 
-  def removeConsumerFetchRequestAndResponseStats(clientId: String) {
-    val pattern = (clientId + "-ConsumerFetcherThread.*").r
+  def removeConsumerFetchRequestAndResponseStats(clientId: Taggable) {
+    val pattern = (clientId + ".*ConsumerFetcherThread.*").r
     val keys = globalStats.keys
     for (key <- keys) {
-      pattern.findFirstIn(key) match {
+      pattern.findFirstIn(key.toString) match {
         case Some(_) => globalStats.remove(key)
         case _ =>
       }
Index: core/src/main/scala/kafka/consumer/TopicCount.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/TopicCount.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/TopicCount.scala	(revision )
@@ -20,7 +20,7 @@
 import scala.collection._
 import org.I0Itec.zkclient.ZkClient
 import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, Utils}
-import kafka.common.KafkaException
+import kafka.common.{Taggable, KafkaException}
 
 private[kafka] trait TopicCount {
 
@@ -30,10 +30,10 @@
 
 }
 
-case class ConsumerThreadId(consumer: String, threadId: Int) extends Ordered[ConsumerThreadId] {
-  override def toString = "%s-%d".format(consumer, threadId)
+case class ConsumerThreadId(consumerId: ConsumerId, threadId: Int) extends Ordered[ConsumerThreadId] with Taggable {
+  override def toTags = consumerId.toTags ++ mutable.LinkedHashMap("threadId" -> threadId.toString)
 
-  def compare(that: ConsumerThreadId) = toString.compare(that.toString)
+  def compare(that: ConsumerThreadId) = threadId.toString.compare(that.threadId.toString)
 }
 
 private[kafka] object TopicCount extends Logging {
@@ -43,20 +43,20 @@
 
   def makeThreadId(consumerIdString: String, threadId: Int) = consumerIdString + "-" + threadId
 
-  def makeConsumerThreadIdsPerTopic(consumerIdString: String,
+  def makeConsumerThreadIdsPerTopic(consumerId: ConsumerId,
                                     topicCountMap: Map[String,  Int]) = {
     val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[ConsumerThreadId]]()
     for ((topic, nConsumers) <- topicCountMap) {
       val consumerSet = new mutable.HashSet[ConsumerThreadId]
       assert(nConsumers >= 1)
       for (i <- 0 until nConsumers)
-        consumerSet += ConsumerThreadId(consumerIdString, i)
+        consumerSet += ConsumerThreadId(consumerId, i)
       consumerThreadIdsPerTopicMap.put(topic, consumerSet)
     }
     consumerThreadIdsPerTopicMap
   }
 
-  def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient, excludeInternalTopics: Boolean) : TopicCount = {
+  def constructTopicCount(group: String, consumerId: ConsumerId, zkClient: ZkClient, excludeInternalTopics: Boolean) : TopicCount = {
     val dirs = new ZKGroupDirs(group)
     val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)._1
     var subscriptionPattern: String = null
@@ -98,24 +98,24 @@
     }
   }
 
-  def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) =
-    new StaticTopicCount(consumerIdString, topicCount)
+  def constructTopicCount(consumerId: ConsumerId, topicCount: Map[String, Int]) =
+    new StaticTopicCount(consumerId, topicCount)
 
-  def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient, excludeInternalTopics: Boolean) =
-    new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams, excludeInternalTopics)
+  def constructTopicCount(consumerId: ConsumerId, filter: TopicFilter, numStreams: Int, zkClient: ZkClient, excludeInternalTopics: Boolean) =
+    new WildcardTopicCount(zkClient, consumerId, filter, numStreams, excludeInternalTopics)
 
 }
 
-private[kafka] class StaticTopicCount(val consumerIdString: String,
+private[kafka] class StaticTopicCount(val consumerId: ConsumerId,
                                 val topicCountMap: Map[String, Int])
                                 extends TopicCount {
 
-  def getConsumerThreadIdsPerTopic = TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap)
+  def getConsumerThreadIdsPerTopic = TopicCount.makeConsumerThreadIdsPerTopic(consumerId, topicCountMap)
 
   override def equals(obj: Any): Boolean = {
     obj match {
       case null => false
-      case n: StaticTopicCount => consumerIdString == n.consumerIdString && topicCountMap == n.topicCountMap
+      case n: StaticTopicCount => consumerId.toString == n.consumerId.toString && topicCountMap == n.topicCountMap
       case _ => false
     }
   }
@@ -126,14 +126,14 @@
 }
 
 private[kafka] class WildcardTopicCount(zkClient: ZkClient,
-                                        consumerIdString: String,
+                                        consumerId: ConsumerId,
                                         topicFilter: TopicFilter,
                                         numStreams: Int,
                                         excludeInternalTopics: Boolean) extends TopicCount {
   def getConsumerThreadIdsPerTopic = {
     val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath)
                          .filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics))
-    TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*))
+    TopicCount.makeConsumerThreadIdsPerTopic(consumerId, Map(wildcardTopics.map((_, numStreams)): _*))
   }
 
   def getTopicCountMap = Map(Utils.JSONEscapeString(topicFilter.regex) -> numStreams)
Index: core/src/main/scala/kafka/consumer/ConsumerFetcherThreadId.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/ConsumerFetcherThreadId.scala	(revision )
+++ core/src/main/scala/kafka/consumer/ConsumerFetcherThreadId.scala	(revision )
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.consumer
+
+import kafka.common.Taggable
+
+import scala.collection.mutable
+
+class ConsumerFetcherThreadId(threadName: String, fetcherId: Int, sourceBrokerId: Int, consumerId: ConsumerId) extends Taggable {
+  override def toTags = mutable.LinkedHashMap("threadName" -> threadName, "fetcherId" -> fetcherId.toString, "sourceBrokerId" -> sourceBrokerId.toString) ++ consumerId.toTags
+}
Index: core/src/main/scala/kafka/producer/ProducerRequestStats.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/producer/ProducerRequestStats.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/producer/ProducerRequestStats.scala	(revision )
@@ -19,26 +19,26 @@
 import kafka.metrics.{KafkaTimer, KafkaMetricsGroup}
 import java.util.concurrent.TimeUnit
 import kafka.utils.Pool
-import kafka.common.ClientIdAndBroker
+import kafka.common.{Taggable, BrokerInfo, BrokerAllInfo, ClientIdAndBroker}
 
 class ProducerRequestMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup {
-  val requestTimer = new KafkaTimer(newTimer(metricId + "ProducerRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
-  val requestSizeHist = newHistogram(metricId + "ProducerRequestSize")
+  val requestTimer = new KafkaTimer(newTimer("ProducerRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, metricId.toTags))
+  val requestSizeHist = newHistogram("ProducerRequestSize", biased = true, metricId.toTags)
 }
 
 /**
  * Tracks metrics of requests made by a given producer client to all brokers.
  * @param clientId ClientId of the given producer
  */
-class ProducerRequestStats(clientId: String) {
+class ProducerRequestStats(clientId: Taggable) {
   private val valueFactory = (k: ClientIdAndBroker) => new ProducerRequestMetrics(k)
   private val stats = new Pool[ClientIdAndBroker, ProducerRequestMetrics](Some(valueFactory))
-  private val allBrokersStats = new ProducerRequestMetrics(new ClientIdAndBroker(clientId, "AllBrokers"))
+  private val allBrokersStats = new ProducerRequestMetrics(new ClientIdAndBroker(clientId, new BrokerAllInfo()))
 
   def getProducerRequestAllBrokersStats(): ProducerRequestMetrics = allBrokersStats
 
-  def getProducerRequestStats(brokerInfo: String): ProducerRequestMetrics = {
-    stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo + "-"))
+  def getProducerRequestStats(brokerInfo: BrokerInfo): ProducerRequestMetrics = {
+    stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo))
   }
 }
 
@@ -46,14 +46,14 @@
  * Stores the request stats information of each producer client in a (clientId -> ProducerRequestStats) map.
  */
 object ProducerRequestStatsRegistry {
-  private val valueFactory = (k: String) => new ProducerRequestStats(k)
-  private val globalStats = new Pool[String, ProducerRequestStats](Some(valueFactory))
+  private val valueFactory = (k: Taggable) => new ProducerRequestStats(k)
+  private val globalStats = new Pool[Taggable, ProducerRequestStats](Some(valueFactory))
 
-  def getProducerRequestStats(clientId: String) = {
+  def getProducerRequestStats(clientId: Taggable) = {
     globalStats.getAndMaybePut(clientId)
   }
 
-  def removeProducerRequestStats(clientId: String) {
+  def removeProducerRequestStats(clientId: Taggable) {
     globalStats.remove(clientId)
   }
 }
Index: core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala	(revision )
@@ -20,15 +20,13 @@
 import org.I0Itec.zkclient.ZkClient
 import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager}
 import kafka.cluster.{Cluster, Broker}
-import scala.collection.immutable
-import scala.collection.Map
+import scala.collection.{immutable, mutable}
 import collection.mutable.HashMap
-import scala.collection.mutable
 import java.util.concurrent.locks.ReentrantLock
 import kafka.utils.Utils.inLock
 import kafka.utils.ZkUtils._
 import kafka.utils.{ShutdownableThread, SystemTime}
-import kafka.common.TopicAndPartition
+import kafka.common.{TaggableInfo, TopicAndPartition}
 import kafka.client.ClientUtils
 import java.util.concurrent.atomic.AtomicInteger
 
@@ -37,11 +35,11 @@
  *  Once ConsumerFetcherManager is created, startConnections() and stopAllConnections() can be called repeatedly
  *  until shutdown() is called.
  */
-class ConsumerFetcherManager(private val consumerIdString: String,
+class ConsumerFetcherManager(private val consumerId: ConsumerId,
                              private val config: ConsumerConfig,
                              private val zkClient : ZkClient)
         extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds),
-                                       config.clientId, config.numConsumerFetchers) {
+                                       new TaggableInfo("clientId" -> config.clientId) , config.numConsumerFetchers) {
   private var partitionMap: immutable.Map[TopicAndPartition, PartitionTopicInfo] = null
   private var cluster: Cluster = null
   private val noLeaderPartitionSet = new mutable.HashSet[TopicAndPartition]
@@ -115,17 +113,15 @@
   }
 
   override def createFetcherThread(fetcherId: Int, sourceBroker: Broker): AbstractFetcherThread = {
-    new ConsumerFetcherThread(
-      "ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id),
+    new ConsumerFetcherThread(new ConsumerFetcherThreadId("ConsumerFetcherThread", fetcherId, sourceBroker.id, consumerId),
       config, sourceBroker, partitionMap, this)
   }
 
   def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) {
-    leaderFinderThread = new LeaderFinderThread(consumerIdString + "-leader-finder-thread")
+    leaderFinderThread = new LeaderFinderThread(consumerId + "-leader-finder-thread")
     leaderFinderThread.start()
 
-    inLock(lock) {
-      partitionMap = topicInfos.map(tpi => (TopicAndPartition(tpi.topic, tpi.partitionId), tpi)).toMap
+    inLock(lock) {      partitionMap = topicInfos.map(tpi => (TopicAndPartition(tpi.topic, tpi.partitionId), tpi)).toMap
       this.cluster = cluster
       noLeaderPartitionSet ++= topicInfos.map(tpi => TopicAndPartition(tpi.topic, tpi.partitionId))
       cond.signalAll()
\ No newline at end of file
Index: core/src/main/scala/kafka/log/Log.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/log/Log.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/log/Log.scala	(revision )
@@ -27,7 +27,7 @@
 import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
 import java.util.concurrent.atomic._
 import java.text.NumberFormat
-import scala.collection.JavaConversions
+import scala.collection.{mutable, JavaConversions}
 
 import com.yammer.metrics.core.Gauge
 
@@ -73,20 +73,32 @@
 
   info("Completed load of log %s with log end offset %d".format(name, logEndOffset))
 
-  newGauge(name + "-" + "NumLogSegments",
-           new Gauge[Int] { def value = numberOfSegments })
 
-  newGauge(name + "-" + "LogStartOffset",
-           new Gauge[Long] { def value = logStartOffset })
+  val metricTags = buildMetricTags(topicAndPartition)
 
-  newGauge(name + "-" + "LogEndOffset",
-           new Gauge[Long] { def value = logEndOffset })
+  newGauge("NumLogSegments",
+    new Gauge[Int] {
+      def value = numberOfSegments
+    }, metricTags)
-           
+
-  newGauge(name + "-" + "Size", 
-           new Gauge[Long] {def value = size})
 
+  newGauge("LogEndOffset",
+    new Gauge[Long] {
+      def value = logEndOffset
+    }, metricTags)
+
+
+  newGauge("Size",
+    new Gauge[Long] {
+      def value = size
+    }, metricTags)
+
+  private def buildMetricTags(topicPartition: TopicAndPartition) = {
+    mutable.LinkedHashMap("topic" -> topicAndPartition.topic, "partitionId" -> topicAndPartition.partition.toString)
+  }
+
   /** The name of this log */
-  def name  = dir.getName()
+  def name = dir.getName
 
   /* Load the log segments from the log files on disk */
   private def loadSegments() {
@@ -153,7 +165,7 @@
 
     if(logSegments.size == 0) {
       // no existing segments, create a new mutable segment beginning at offset 0
-      segments.put(0, new LogSegment(dir = dir, 
+      segments.put(0L, new LogSegment(dir = dir,
                                      startOffset = 0,
                                      indexIntervalBytes = config.indexInterval, 
                                      maxIndexSize = config.maxIndexSize,
@@ -274,7 +286,7 @@
           if(MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) {
             // we record the original message set size instead of trimmed size
             // to be consistent with pre-compression bytesRejectedRate recording
-            BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
+            BrokerTopicStats.getBrokerTopicStats(new TaggableInfo("topic" -> topicAndPartition.topic)).bytesRejectedRate.mark(messages.sizeInBytes)
             BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
             throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
               .format(MessageSet.entrySize(messageAndOffset.message), config.maxMessageSize))
@@ -359,7 +371,7 @@
       // Check if the message sizes are valid.
       val messageSize = MessageSet.entrySize(m)
       if(messageSize > config.maxMessageSize) {
-        BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
+        BrokerTopicStats.getBrokerTopicStats(new TaggableInfo("topic" -> topicAndPartition.topic)).bytesRejectedRate.mark(messages.sizeInBytes)
         BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
         throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
           .format(messageSize, config.maxMessageSize))
Index: core/src/test/scala/unit/kafka/producer/ProducerTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/producer/ProducerTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/producer/ProducerTest.scala	(revision )
@@ -34,7 +34,7 @@
 import org.junit.Assert.assertTrue
 import org.junit.Assert.assertFalse
 import org.junit.Assert.assertEquals
-import kafka.common.{ErrorMapping, FailedToSendMessageException}
+import kafka.common.{TaggableInfo, BrokerSpecificInfo, ErrorMapping, FailedToSendMessageException}
 import kafka.serializer.StringEncoder
 
 class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{
@@ -67,8 +67,8 @@
     props.put("host", "localhost")
     props.put("port", port1.toString)
 
-    consumer1 = new SimpleConsumer("localhost", port1, 1000000, 64*1024, "")
-    consumer2 = new SimpleConsumer("localhost", port2, 100, 64*1024, "")
+    consumer1 = new SimpleConsumer(new BrokerSpecificInfo("localhost", port1), 1000000, 64*1024, new TaggableInfo("clientId"->""))
+    consumer2 = new SimpleConsumer(new BrokerSpecificInfo("localhost", port2), 100, 64*1024, new TaggableInfo("clientId"->""))
 
     // temporarily set request handler logger to a higher level
     requestHandlerLogger.setLevel(Level.FATAL)
Index: core/src/main/scala/kafka/producer/async/ProducerSendThread.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/producer/async/ProducerSendThread.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/producer/async/ProducerSendThread.scala	(revision )
@@ -24,6 +24,8 @@
 import kafka.metrics.KafkaMetricsGroup
 import com.yammer.metrics.core.Gauge
 
+import scala.collection.mutable
+
 class ProducerSendThread[K,V](val threadName: String,
                               val queue: BlockingQueue[KeyedMessage[K,V]],
                               val handler: EventHandler[K,V],
@@ -34,10 +36,10 @@
   private val shutdownLatch = new CountDownLatch(1)
   private val shutdownCommand = new KeyedMessage[K,V]("shutdown", null.asInstanceOf[K], null.asInstanceOf[V])
 
-  newGauge(clientId + "-ProducerQueueSize",
+  newGauge("ProducerQueueSize",
           new Gauge[Int] {
             def value = queue.size
-          })
+          }, mutable.LinkedHashMap("clientId" -> clientId))
 
   override def run {
     try {
Index: core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala	(revision )
@@ -19,7 +19,6 @@
 import org.apache.kafka.common.requests._
 import kafka.api.ApiUtils._
 import kafka.network.RequestChannel.Response
-import scala.Some
 
 object JoinGroupRequestAndHeader {
   def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = {
@@ -35,7 +34,7 @@
                                      override val correlationId: Int,
                                      override val clientId: String,
                                      override val body: JoinGroupRequest)
-  extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) {
+  extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey).toString, Some(RequestKeys.JoinGroupKey)) {
 
   override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
     val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
Index: core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala	(revision )
@@ -19,6 +19,7 @@
 
 import java.util.concurrent._
 import java.util.concurrent.atomic._
+import kafka.common.Taggable
 import kafka.message._
 import kafka.utils.Logging
 
@@ -28,11 +29,10 @@
                          private val consumedOffset: AtomicLong,
                          private val fetchedOffset: AtomicLong,
                          private val fetchSize: AtomicInteger,
-                         private val clientId: String) extends Logging {
+                         private val clientId: Taggable) extends Logging {
 
   debug("initial consumer offset of " + this + " is " + consumedOffset.get)
   debug("initial fetch offset of " + this + " is " + fetchedOffset.get)
-
   private val consumerTopicStats = ConsumerTopicStatsRegistry.getConsumerTopicStat(clientId)
 
   def getConsumeOffset() = consumedOffset.get
\ No newline at end of file
Index: core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala	(revision )
@@ -24,5 +24,5 @@
 }
 
 case class JoinGroupResponseAndHeader(override val correlationId: Int, override val body: JoinGroupResponse)
-  extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) {
+  extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey).toString, None) {
 }
Index: core/src/main/scala/kafka/consumer/SimpleConsumer.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/SimpleConsumer.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/SimpleConsumer.scala	(revision )
@@ -19,24 +19,20 @@
 
 import kafka.api._
 import kafka.network._
-import kafka.utils._
-import kafka.common.{ErrorMapping, TopicAndPartition}
-import org.apache.kafka.common.utils.Utils._
+import kafka.utils.{Logging, threadsafe}
+import kafka.common._
+import org.apache.kafka.common.utils.Utils
 
 /**
  * A consumer of kafka messages
  */
 @threadsafe
-class SimpleConsumer(val host: String,
-                     val port: Int,
+class SimpleConsumer(val brokerInfo: BrokerSpecificInfo,
                      val soTimeout: Int,
                      val bufferSize: Int,
-                     val clientId: String) extends Logging {
-
-  ConsumerConfig.validateClientId(clientId)
+                     val clientId: Taggable) extends Logging {
   private val lock = new Object()
-  private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout)
-  val brokerInfo = "host_%s-port_%s".format(host, port)
+  private val blockingChannel = new BlockingChannel(brokerInfo.host, brokerInfo.port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout)
   private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId)
   private var isClosed = false
 
@@ -47,7 +43,7 @@
   }
 
   private def disconnect() = {
-    debug("Disconnecting from " + formatAddress(host, port))
+    debug("Disconnecting from " + Utils.formatAddress(brokerInfo.host, brokerInfo.port))
     blockingChannel.disconnect()
   }
 
@@ -160,7 +156,7 @@
    */
   def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = {
     val request = OffsetRequest(requestInfo = Map(topicAndPartition -> PartitionOffsetRequestInfo(earliestOrLatest, 1)),
-                                clientId = clientId,
+                                clientId = clientId.toString,
                                 replicaId = consumerId)
     val partitionErrorAndOffset = getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition)
     val offset = partitionErrorAndOffset.error match {
Index: core/src/main/scala/kafka/tools/GetOffsetShell.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/tools/GetOffsetShell.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/tools/GetOffsetShell.scala	(revision )
@@ -21,7 +21,7 @@
 import kafka.consumer._
 import joptsimple._
 import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest}
-import kafka.common.TopicAndPartition
+import kafka.common.{TaggableInfo, BrokerSpecificInfo, TopicAndPartition}
 import kafka.client.ClientUtils
 import kafka.utils.{ToolsUtils, CommandLineUtils}
 
@@ -93,7 +93,7 @@
         case Some(metadata) =>
           metadata.leader match {
             case Some(leader) =>
-              val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId)
+              val consumer = new SimpleConsumer(new BrokerSpecificInfo(leader.host, leader.port), 10000, 100000, new TaggableInfo("clientId"->clientId))
               val topicAndPartition = TopicAndPartition(topic, partitionId)
               val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets)))
               val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets
Index: core/src/main/scala/kafka/producer/ProducerTopicStats.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/producer/ProducerTopicStats.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/producer/ProducerTopicStats.scala	(revision )
@@ -17,31 +17,31 @@
 package kafka.producer
 
 import kafka.metrics.KafkaMetricsGroup
-import kafka.common.ClientIdAndTopic
+import kafka.common.{Taggable, ClientIdAndTopic, ClientIdAllTopics, ClientIdTopic}
 import kafka.utils.{Pool, threadsafe}
 import java.util.concurrent.TimeUnit
 
 
 @threadsafe
-class ProducerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup {
-  val messageRate = newMeter(metricId + "MessagesPerSec", "messages", TimeUnit.SECONDS)
-  val byteRate = newMeter(metricId + "BytesPerSec", "bytes", TimeUnit.SECONDS)
-  val droppedMessageRate = newMeter(metricId + "DroppedMessagesPerSec", "drops", TimeUnit.SECONDS)
+class ProducerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup {
+  val messageRate = newMeter("MessagesPerSec", "messages", TimeUnit.SECONDS, metricId.toTags)
+  val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, metricId.toTags)
+  val droppedMessageRate = newMeter("DroppedMessagesPerSec", "drops", TimeUnit.SECONDS, metricId.toTags)
 }
 
 /**
  * Tracks metrics for each topic the given producer client has produced data to.
  * @param clientId The clientId of the given producer client.
  */
-class ProducerTopicStats(clientId: String) {
-  private val valueFactory = (k: ClientIdAndTopic) => new ProducerTopicMetrics(k)
-  private val stats = new Pool[ClientIdAndTopic, ProducerTopicMetrics](Some(valueFactory))
-  private val allTopicsStats = new ProducerTopicMetrics(new ClientIdAndTopic(clientId, "AllTopics")) // to differentiate from a topic named AllTopics
+class ProducerTopicStats(clientId: Taggable) {
+  private val valueFactory = (k: ClientIdTopic) => new ProducerTopicMetrics(k)
+  private val stats = new Pool[ClientIdTopic, ProducerTopicMetrics](Some(valueFactory))
+  private val allTopicsStats = new ProducerTopicMetrics(new ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics
 
   def getProducerAllTopicsStats(): ProducerTopicMetrics = allTopicsStats
 
   def getProducerTopicStats(topic: String): ProducerTopicMetrics = {
-    stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic + "-"))
+    stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic))
   }
 }
 
@@ -49,14 +49,14 @@
  * Stores the topic stats information of each producer client in a (clientId -> ProducerTopicStats) map.
  */
 object ProducerTopicStatsRegistry {
-  private val valueFactory = (k: String) => new ProducerTopicStats(k)
-  private val globalStats = new Pool[String, ProducerTopicStats](Some(valueFactory))
+  private val valueFactory = (k: Taggable) => new ProducerTopicStats(k)
+  private val globalStats = new Pool[Taggable, ProducerTopicStats](Some(valueFactory))
 
-  def getProducerTopicStats(clientId: String) = {
+  def getProducerTopicStats(clientId: Taggable) = {
     globalStats.getAndMaybePut(clientId)
   }
 
-  def removeProducerTopicStats(clientId: String) {
+  def removeProducerTopicStats(clientId: Taggable) {
     globalStats.remove(clientId)
   }
 }
Index: core/src/main/scala/kafka/api/RequestKeys.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/api/RequestKeys.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/api/RequestKeys.scala	(revision )
@@ -17,7 +17,7 @@
 
 package kafka.api
 
-import kafka.common.KafkaException
+import kafka.common.{TaggableInfo, Taggable, KafkaException}
 import java.nio.ByteBuffer
 
 object RequestKeys {
@@ -35,23 +35,24 @@
   val JoinGroupKey: Short = 11
   val HeartbeatKey: Short = 12
 
-  val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]=
-    Map(ProduceKey -> ("Produce", ProducerRequest.readFrom),
-        FetchKey -> ("Fetch", FetchRequest.readFrom),
-        OffsetsKey -> ("Offsets", OffsetRequest.readFrom),
-        MetadataKey -> ("Metadata", TopicMetadataRequest.readFrom),
-        LeaderAndIsrKey -> ("LeaderAndIsr", LeaderAndIsrRequest.readFrom),
-        StopReplicaKey -> ("StopReplica", StopReplicaRequest.readFrom),
-        UpdateMetadataKey -> ("UpdateMetadata", UpdateMetadataRequest.readFrom),
-        ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom),
-        OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom),
-        OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom),
-        ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom),
-        JoinGroupKey -> ("JoinGroup", JoinGroupRequestAndHeader.readFrom),
-        HeartbeatKey -> ("Heartbeat", HeartbeatRequestAndHeader.readFrom)
+  val keyToNameAndDeserializerMap: Map[Short, (Taggable, (ByteBuffer) => RequestOrResponse)] =
+    Map(ProduceKey ->(new TaggableInfo("request" -> "Produce"), ProducerRequest.readFrom),
+      FetchKey ->(new TaggableInfo("request" -> "Fetch"), FetchRequest.readFrom),
+      OffsetsKey ->(new TaggableInfo("request" -> "Offsets"), OffsetRequest.readFrom),
+      MetadataKey ->(new TaggableInfo("request" -> "Metadata"), TopicMetadataRequest.readFrom),
+      LeaderAndIsrKey ->(new TaggableInfo("request" -> "LeaderAndIsr"), LeaderAndIsrRequest.readFrom),
+      StopReplicaKey ->(new TaggableInfo("request" -> "StopReplica"), StopReplicaRequest.readFrom),
+      UpdateMetadataKey ->(new TaggableInfo("request" -> "UpdateMetadata"), UpdateMetadataRequest.readFrom),
+      ControlledShutdownKey ->(new TaggableInfo("request" -> "ControlledShutdown"), ControlledShutdownRequest.readFrom),
+      OffsetCommitKey ->(new TaggableInfo("request" -> "OffsetCommit"), OffsetCommitRequest.readFrom),
+      OffsetFetchKey ->(new TaggableInfo("request" -> "OffsetFetch"), OffsetFetchRequest.readFrom),
+      ConsumerMetadataKey ->(new TaggableInfo("request" -> "ConsumerMetadata"), ConsumerMetadataRequest.readFrom),
+      JoinGroupKey ->(new TaggableInfo("request" -> "JoinGroup"), JoinGroupRequestAndHeader.readFrom),
+      HeartbeatKey ->(new TaggableInfo("request" -> "Heartbeat"), HeartbeatRequestAndHeader.readFrom)
     )
 
-  def nameForKey(key: Short): String = {
+
+  def nameForKey(key: Short): Taggable = {
     keyToNameAndDeserializerMap.get(key) match {
       case Some(nameAndSerializer) => nameAndSerializer._1
       case None => throw new KafkaException("Wrong request type %d".format(key))
Index: core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala	(revision )
@@ -64,11 +64,10 @@
   @Test
   def testWildcardTopicCountGetTopicCountMapEscapeJson() {
     def getTopicCountMapKey(regex: String): String = {
-      val topicCount = new WildcardTopicCount(null, "consumerId", new Whitelist(regex), 1, true)
+      val topicCount = new WildcardTopicCount(null, new ConsumerId("consumerId"), new Whitelist(regex), 1, true)
       topicCount.getTopicCountMap.head._1
     }
-    //lets make sure that the JSON strings are escaping as we expect
-    //if they are not then when they get saved to zookeeper and read back out they will be broken on parse
+    //lets make sure that the JSON strings are escaping as we expect    //if they are not then when they get saved to zookeeper and read back out they will be broken on parse
     assertEquals("-\\\"-", getTopicCountMapKey("-\"-"))
     assertEquals("-\\\\-", getTopicCountMapKey("-\\-"))
     assertEquals("-\\/-", getTopicCountMapKey("-/-"))
\ No newline at end of file
Index: core/src/main/scala/kafka/consumer/ConsumerId.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/ConsumerId.scala	(revision )
+++ core/src/main/scala/kafka/consumer/ConsumerId.scala	(revision )
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.consumer
+
+import kafka.common.{Taggable}
+
+import scala.collection.mutable
+
+case class ConsumerId(consumerTags: mutable.LinkedHashMap[String, String]) extends Taggable {
+  def this(consumerTagsString: String) = {
+    this(ConsumerId.mkMap(consumerTagsString))
+  }
+
+  override def toTags = consumerTags
+
+
+}
+
+object ConsumerId {
+  def mkMap(string: String) = {
+    val pairs = string.split(",").map { pair =>
+      pair.split("=") match {
+        case Array(k, v) => (k.trim, v.trim)
+        case _ => ("value", pair)
+      }
+    }
+    collection.mutable.LinkedHashMap(pairs: _*)
+  }
+}
Index: core/src/main/scala/kafka/common/ClientIdTopic.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/common/ClientIdTopic.scala	(revision )
+++ core/src/main/scala/kafka/common/ClientIdTopic.scala	(revision )
@@ -0,0 +1,35 @@
+package kafka.common
+
+import scala.collection.mutable
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Convenience case class since (clientId, topic) pairs are used in the creation
+ * of many Stats objects.
+ */
+trait ClientIdTopic extends Taggable {
+}
+
+case class ClientIdAndTopic(clientId: Taggable, topic: String) extends ClientIdTopic {
+  override def toTags = clientId.toTags ++ mutable.LinkedHashMap("topic" -> topic)
+}
+
+case class ClientIdAllTopics(clientId: Taggable) extends ClientIdTopic {
+  override def toTags = clientId.toTags ++ TopicAllInfo().toTags
+}
Index: core/src/main/scala/kafka/common/BrokerInfo.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/common/BrokerInfo.scala	(revision )
+++ core/src/main/scala/kafka/common/BrokerInfo.scala	(revision )
@@ -0,0 +1,31 @@
+package kafka.common
+
+import scala.collection.mutable
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+trait BrokerInfo extends Taggable {
+}
+
+case class BrokerSpecificInfo(host: String, port: Int) extends BrokerInfo {
+  override def toTags = mutable.LinkedHashMap("brokerHost" -> host, "brokerPort" -> port.toString)
+}
+
+case class BrokerAllInfo() extends BrokerInfo {
+  override def toTags = mutable.LinkedHashMap("allBrokers" -> "true")
+}
Index: core/src/main/scala/kafka/consumer/PartitionAssignor.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/PartitionAssignor.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/PartitionAssignor.scala	(revision )
@@ -39,7 +39,7 @@
   }
 }
 
-class AssignmentContext(group: String, val consumerId: String, excludeInternalTopics: Boolean, zkClient: ZkClient) {
+class AssignmentContext(group: String, val consumerId: ConsumerId, excludeInternalTopics: Boolean, zkClient: ZkClient) {
   val myTopicThreadIds: collection.Map[String, collection.Set[ConsumerThreadId]] = {
     val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkClient, excludeInternalTopics)
     myTopicCount.getConsumerThreadIdsPerTopic
@@ -101,7 +101,7 @@
 
     allTopicPartitions.foreach(topicPartition => {
       val threadId = threadAssignor.next()
-      if (threadId.consumer == ctx.consumerId)
+      if (threadId.consumerId == ctx.consumerId)
         partitionOwnershipDecision += (topicPartition -> threadId)
     })
 
Index: core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala	(revision )
@@ -20,7 +20,7 @@
 import org.I0Itec.zkclient.ZkClient
 import kafka.consumer.{SimpleConsumer, ConsumerConfig}
 import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest}
-import kafka.common.{TopicAndPartition, KafkaException}
+import kafka.common.{TaggableInfo, BrokerSpecificInfo, TopicAndPartition, KafkaException}
 import kafka.utils.{ZKGroupTopicDirs, ZkUtils, ZKStringSerializer, Utils}
 
 
@@ -65,7 +65,7 @@
 
       ZkUtils.getBrokerInfo(zkClient, broker) match {
         case Some(brokerInfo) =>
-          val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024, "UpdateOffsetsInZk")
+          val consumer = new SimpleConsumer(new BrokerSpecificInfo(brokerInfo.host, brokerInfo.port), 10000, 100 * 1024, new TaggableInfo("clientId"->"UpdateOffsetsInZk"))
           val topicAndPartition = TopicAndPartition(topic, partition)
           val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1)))
           val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head
Index: core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala	(revision )
@@ -45,8 +45,8 @@
   private val topicMetadataToRefresh = Set.empty[String]
   private val sendPartitionPerTopicCache = HashMap.empty[String, Int]
 
-  private val producerStats = ProducerStatsRegistry.getProducerStats(config.clientId)
-  private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId)
+  private val producerStats = ProducerStatsRegistry.getProducerStats(new TaggableInfo("clientId"->config.clientId))
+  private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(new TaggableInfo("clientId"->config.clientId))
 
   def handle(events: Seq[KeyedMessage[K,V]]) {
     val serializedData = serialize(events)
Index: core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala	(revision )
@@ -17,6 +17,7 @@
 
 package kafka.server
 
+import kafka.common.Taggable
 import kafka.metrics.KafkaMetricsGroup
 import kafka.utils.Pool
 import kafka.network.{BoundedByteBufferSend, RequestChannel}
@@ -30,12 +31,12 @@
   extends RequestPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) {
   this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId)
 
-  private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup {
-    val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS)
+  private class DelayedProducerRequestMetrics(keyLabel: Taggable = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup {
+    val expiredRequestMeter = newMeter("ExpiresPerSecond", "requests", TimeUnit.SECONDS, keyLabel.toTags)
   }
 
   private val producerRequestMetricsForKey = {
-    val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-")
+    val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel)
     new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory))
   }
 
Index: core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala	(revision )
@@ -17,6 +17,7 @@
 
 package kafka.integration
 
+import kafka.common.{TaggableInfo, BrokerSpecificInfo}
 import kafka.consumer.SimpleConsumer
 import org.scalatest.junit.JUnit3Suite
 import kafka.producer.Producer
@@ -35,7 +36,7 @@
         encoder = classOf[StringEncoder].getName,
         keyEncoder = classOf[StringEncoder].getName,
         partitioner = classOf[StaticPartitioner].getName)
-      consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "")
+      consumer = new SimpleConsumer(new BrokerSpecificInfo(host, port), 1000000, 64*1024, new TaggableInfo("clientId"->""))
     }
 
    override def tearDown() {
Index: core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/ReplicaFetcherThread.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/ReplicaFetcherThread.scala	(revision )
@@ -22,9 +22,9 @@
 import kafka.log.LogConfig
 import kafka.message.ByteBufferMessageSet
 import kafka.api.{OffsetRequest, FetchResponsePartitionData}
-import kafka.common.{KafkaStorageException, TopicAndPartition}
+import kafka.common.{Taggable, KafkaStorageException, TopicAndPartition}
 
-class ReplicaFetcherThread(name:String,
+class ReplicaFetcherThread(name: Taggable,
                            sourceBroker: Broker,
                            brokerConfig: KafkaConfig,
                            replicaMgr: ReplicaManager)
Index: core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala	(revision )
@@ -23,12 +23,11 @@
 import kafka.utils._
 import kafka.consumer.SimpleConsumer
 import kafka.api.{OffsetFetchResponse, OffsetFetchRequest, OffsetRequest}
-import kafka.common.{OffsetMetadataAndError, ErrorMapping, BrokerNotAvailableException, TopicAndPartition}
+import kafka.common._
 import scala.collection._
 import kafka.client.ClientUtils
 import kafka.network.BlockingChannel
 import kafka.api.PartitionOffsetRequestInfo
-import scala.Some
 import org.I0Itec.zkclient.exception.ZkNoNodeException
 
 object ConsumerOffsetChecker extends Logging {
@@ -46,7 +45,7 @@
               val brokerInfo = m.asInstanceOf[Map[String, Any]]
               val host = brokerInfo.get("host").get.asInstanceOf[String]
               val port = brokerInfo.get("port").get.asInstanceOf[Int]
-              Some(new SimpleConsumer(host, port, 10000, 100000, "ConsumerOffsetChecker"))
+              Some(new SimpleConsumer(new BrokerSpecificInfo(host, port), 10000, 100000, new TaggableInfo("clientId"->"ConsumerOffsetChecker")))
             case None =>
               throw new BrokerNotAvailableException("Broker id %d does not exist".format(bid))
           }
@@ -101,7 +100,7 @@
     for ((bid, consumerOpt) <- consumerMap)
       consumerOpt match {
         case Some(consumer) =>
-          println("%s -> %s:%d".format(bid, consumer.host, consumer.port))
+          println("%s -> %s".format(bid, consumer.brokerInfo))
         case None => // ignore
       }
   }
Index: core/src/main/scala/kafka/producer/Producer.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/producer/Producer.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/producer/Producer.scala	(revision )
@@ -18,8 +18,7 @@
 
 import java.util.concurrent.atomic.AtomicBoolean
 import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
-
-import kafka.common.QueueFullException
+import kafka.common.{TaggableInfo, QueueFullException}
 import kafka.metrics._
 import kafka.producer.async.{DefaultEventHandler, EventHandler, ProducerSendThread}
 import kafka.serializer.Encoder
@@ -50,7 +49,7 @@
       producerSendThread.start()
   }
 
-  private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId)
+  private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(new TaggableInfo("clientId"->config.clientId))
 
   KafkaMetricsReporter.startReporters(config.props)
 
@@ -127,7 +126,7 @@
       if(canShutdown) {
         info("Shutting down producer")
         val startTime = System.nanoTime()
-        KafkaMetricsGroup.removeAllProducerMetrics(config.clientId)
+        KafkaMetricsGroup.removeAllProducerMetrics(new TaggableInfo("clientId"->config.clientId))
         if (producerSendThread != null)
           producerSendThread.shutdown
         eventHandler.close
Index: core/src/main/scala/kafka/admin/AdminUtils.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/admin/AdminUtils.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/admin/AdminUtils.scala	(revision )
@@ -31,7 +31,6 @@
 import kafka.common._
 import scala.Predef._
 import collection.Map
-import scala.Some
 import collection.Set
 
 object AdminUtils extends Logging {
Index: core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala	(revision )
@@ -20,6 +20,8 @@
 
 import java.util.concurrent._
 import java.util.concurrent.atomic._
+import kafka.common.TaggableInfo
+
 import scala.collection._
 import junit.framework.Assert._
 
@@ -54,7 +56,7 @@
                                                            new AtomicLong(consumedOffset),
                                                            new AtomicLong(0),
                                                            new AtomicInteger(0),
-                                                           ""))
+                                                           new TaggableInfo("clientId"->"")))
   val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer0))
 
   override def setUp() {
@@ -76,7 +78,7 @@
                                                     consumerConfig.consumerTimeoutMs,
                                                     new StringDecoder(), 
                                                     new StringDecoder(),
-                                                    clientId = "")
+                                                    clientId = new TaggableInfo("clientId"->""))
     val receivedMessages = (0 until 5).map(i => iter.next.message).toList
 
     assertFalse(iter.hasNext)
@@ -99,7 +101,7 @@
       ConsumerConfig.ConsumerTimeoutMs,
       new FailDecoder(),
       new FailDecoder(),
-      clientId = "")
+      clientId = new TaggableInfo("clientId"->""))
 
     val receivedMessages = (0 until 5).map{ i =>
       assertTrue(iter.hasNext)
Index: core/src/main/scala/kafka/server/FetchRequestPurgatory.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/FetchRequestPurgatory.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/FetchRequestPurgatory.scala	(revision )
@@ -23,6 +23,8 @@
 
 import java.util.concurrent.TimeUnit
 
+import scala.collection.mutable
+
 /**
  * The purgatory holding delayed fetch requests
  */
@@ -31,9 +33,8 @@
   this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId)
 
   private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup {
-    private val metricPrefix = if (forFollower) "Follower" else "Consumer"
-
-    val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS)
+    private val metricTags = mutable.LinkedHashMap("source" -> (if (forFollower) "Follower" else "Consumer"))
+    val expiredRequestMeter = newMeter("ExpiresPerSecond", "requests", TimeUnit.SECONDS, metricTags)
   }
 
   private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true)
\ No newline at end of file
Index: core/src/main/scala/kafka/tools/SimpleConsumerShell.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/tools/SimpleConsumerShell.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/tools/SimpleConsumerShell.scala	(revision )
@@ -24,7 +24,7 @@
 import kafka.api.{OffsetRequest, FetchRequestBuilder, Request}
 import kafka.cluster.Broker
 import scala.collection.JavaConversions._
-import kafka.common.TopicAndPartition
+import kafka.common.{TaggableInfo, BrokerSpecificInfo, TopicAndPartition}
 
 /**
  * Command line program to dump out messages to standard out using the simple consumer
@@ -167,8 +167,9 @@
       System.exit(1)
     }
     if (startingOffset < 0) {
-      val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, ConsumerConfig.SocketTimeout,
-                                              ConsumerConfig.SocketBufferSize, clientId)
+      ConsumerConfig.validateClientId(clientId)
+      val simpleConsumer = new SimpleConsumer(new BrokerSpecificInfo(fetchTargetBroker.host, fetchTargetBroker.port), ConsumerConfig.SocketTimeout,
+                                              ConsumerConfig.SocketBufferSize, new TaggableInfo("clientId"->clientId))
       try {
         startingOffset = simpleConsumer.earliestOrLatestOffset(TopicAndPartition(topic, partitionId), startingOffset,
                                                                Request.DebuggingConsumerId)
@@ -189,7 +190,8 @@
     val replicaString = if(replicaId > 0) "leader" else "replica"
     info("Starting simple consumer shell to partition [%s, %d], %s [%d], host and port: [%s, %d], from offset [%d]"
                  .format(topic, partitionId, replicaString, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset))
-    val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024, clientId)
+    ConsumerConfig.validateClientId(clientId)
+    val simpleConsumer = new SimpleConsumer(new BrokerSpecificInfo(fetchTargetBroker.host, fetchTargetBroker.port), 10000, 64*1024, new TaggableInfo("clientId"->clientId))
     val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() {
       def run() {
         var offset = startingOffset
Index: core/src/main/scala/kafka/producer/SyncProducer.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/producer/SyncProducer.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/producer/SyncProducer.scala	(revision )
@@ -18,12 +18,14 @@
 package kafka.producer
 
 import kafka.api._
+import kafka.common.{TaggableInfo, BrokerSpecificInfo}
 import kafka.network.{BlockingChannel, BoundedByteBufferSend, Receive}
 import kafka.utils._
 import java.util.Random
 
 import org.apache.kafka.common.utils.Utils._
 
+
 object SyncProducer {
   val RequestKey: Short = 0
   val randomGenerator = new Random
@@ -39,8 +41,8 @@
   @volatile private var shutdown: Boolean = false
   private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize,
     config.sendBufferBytes, config.requestTimeoutMs)
-  val brokerInfo = "host_%s-port_%s".format(config.host, config.port)
-  val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(config.clientId)
+  val brokerInfo = new BrokerSpecificInfo(config.host, config.port)
+  val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(new TaggableInfo("clientId"->config.clientId))
 
   trace("Instantiating Scala Sync Producer")
 
Index: core/src/main/scala/kafka/server/AbstractFetcherManager.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/AbstractFetcherManager.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/AbstractFetcherManager.scala	(revision )
@@ -23,10 +23,10 @@
 import kafka.utils.{Utils, Logging}
 import kafka.cluster.Broker
 import kafka.metrics.KafkaMetricsGroup
-import kafka.common.TopicAndPartition
+import kafka.common.{Taggable, TopicAndPartition}
 import com.yammer.metrics.core.Gauge
 
-abstract class AbstractFetcherManager(protected val name: String, metricPrefix: String, numFetchers: Int = 1)
+abstract class AbstractFetcherManager(protected val name: String, metricPrefix: Taggable, numFetchers: Int = 1)
   extends Logging with KafkaMetricsGroup {
   // map of (source broker_id, fetcher_id per source broker) => fetcher
   private val fetcherThreadMap = new mutable.HashMap[BrokerAndFetcherId, AbstractFetcherThread]
@@ -34,7 +34,7 @@
   this.logIdent = "[" + name + "] "
 
   newGauge(
-    metricPrefix + "-MaxLag",
+    "MaxLag",
     new Gauge[Long] {
       // current max lag across all fetchers/topics/partitions
       def value = fetcherThreadMap.foldLeft(0L)((curMaxAll, fetcherThreadMapEntry) => {
@@ -42,11 +42,11 @@
           curMaxThread.max(fetcherLagStatsEntry._2.lag)
         }).max(curMaxAll)
       })
-    }
+    }, metricPrefix.toTags
   )
 
   newGauge(
-    metricPrefix + "-MinFetchRate",
+    "MinFetchRate",
     {
       new Gauge[Double] {
         // current min fetch rate across all fetchers/topics/partitions
@@ -59,11 +59,10 @@
           })
         }
       }
-    }
+    }, metricPrefix.toTags
   )
 
-  private def getFetcherId(topic: String, partitionId: Int) : Int = {
-    Utils.abs(31 * topic.hashCode() + partitionId) % numFetchers
+  private def getFetcherId(topic: String, partitionId: Int) : Int = {    Utils.abs(31 * topic.hashCode() + partitionId) % numFetchers
   }
 
   // to be defined in subclass to create a specific fetcher
\ No newline at end of file
Index: core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/integration/kafka/api/ProducerSendTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/integration/kafka/api/ProducerSendTest.scala	(revision )
@@ -17,8 +17,7 @@
 
 package kafka.api.test
 
-import java.lang.{Integer, IllegalArgumentException}
-
+import kafka.common.{TaggableInfo, BrokerSpecificInfo}
 import org.apache.kafka.clients.producer._
 import org.scalatest.junit.JUnit3Suite
 import org.junit.Test
@@ -51,8 +50,8 @@
     super.setUp()
 
     // TODO: we need to migrate to new consumers when 0.9 is final
-    consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "")
-    consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "")
+    consumer1 = new SimpleConsumer(new BrokerSpecificInfo("localhost", configs(0).port), 100, 1024*1024, new TaggableInfo("clientId"->""))
+    consumer2 = new SimpleConsumer(new BrokerSpecificInfo("localhost", configs(1).port), 100, 1024*1024, new TaggableInfo("clientId"->""))
   }
 
   override def tearDown() {
Index: core/src/main/scala/kafka/producer/ProducerStats.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/producer/ProducerStats.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/producer/ProducerStats.scala	(revision )
@@ -16,28 +16,29 @@
  */
 package kafka.producer
 
+import kafka.common.Taggable
 import kafka.metrics.KafkaMetricsGroup
 import java.util.concurrent.TimeUnit
 import kafka.utils.Pool
 
-class ProducerStats(clientId: String) extends KafkaMetricsGroup {
-  val serializationErrorRate = newMeter(clientId + "-SerializationErrorsPerSec",  "errors", TimeUnit.SECONDS)
-  val resendRate = newMeter(clientId + "-ResendsPerSec",  "resends", TimeUnit.SECONDS)
-  val failedSendRate = newMeter(clientId + "-FailedSendsPerSec",  "failed sends", TimeUnit.SECONDS)
+class ProducerStats(clientId: Taggable) extends KafkaMetricsGroup {
+  val serializationErrorRate = newMeter("SerializationErrorsPerSec", "errors", TimeUnit.SECONDS, clientId.toTags)
+  val resendRate = newMeter("ResendsPerSec", "resends", TimeUnit.SECONDS, clientId.toTags)
+  val failedSendRate = newMeter("FailedSendsPerSec" , "failed sends", TimeUnit.SECONDS, clientId.toTags)
 }
 
 /**
  * Stores metrics of serialization and message sending activity of each producer client in a (clientId -> ProducerStats) map.
  */
 object ProducerStatsRegistry {
-  private val valueFactory = (k: String) => new ProducerStats(k)
-  private val statsRegistry = new Pool[String, ProducerStats](Some(valueFactory))
+  private val valueFactory = (k: Taggable) => new ProducerStats(k)
+  private val statsRegistry = new Pool[Taggable, ProducerStats](Some(valueFactory))
 
-  def getProducerStats(clientId: String) = {
+  def getProducerStats(clientId: Taggable) = {
     statsRegistry.getAndMaybePut(clientId)
   }
 
-  def removeProducerStats(clientId: String) {
+  def removeProducerStats(clientId: Taggable) {
     statsRegistry.remove(clientId)
   }
 }
Index: core/src/main/scala/kafka/common/Taggable.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/common/Taggable.scala	(revision )
+++ core/src/main/scala/kafka/common/Taggable.scala	(revision )
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.common
+
+import scala.collection.mutable
+
+trait Taggable {
+  def toTags: scala.collection.Map[String, String] = new mutable.LinkedHashMap[String, String]
+
+  override def toString: String = {
+    toTags
+    .filter(_._2 != "").map(kv => "%s=%s".format(kv._1, kv._2))
+    .mkString(",")
+  }
+
+  override def hashCode(): Int = toString.hashCode()
+
+  override def equals(obj: scala.Any): Boolean = toString.equals(obj.toString)
+}
+
+case class TaggableInfo(tags: mutable.LinkedHashMap[String, String]) extends Taggable {
+  override def toTags = tags
+
+  def this(tags1: Taggable, tags2: Taggable) = {
+    this(mutable.LinkedHashMap[String, String]((tags1.toTags ++ tags2.toTags).iterator.toSeq: _*))
+  }
+
+  def this(entries: (String, String)*) = this(mutable.LinkedHashMap(entries: _*))
+}
Index: core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala	(revision )
@@ -20,27 +20,27 @@
 import kafka.utils.{Pool, threadsafe, Logging}
 import java.util.concurrent.TimeUnit
 import kafka.metrics.KafkaMetricsGroup
-import kafka.common.ClientIdAndTopic
+import kafka.common.{Taggable, ClientIdAllTopics, ClientIdAndTopic, ClientIdTopic}
 
 @threadsafe
-class ConsumerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup {
-  val messageRate = newMeter(metricId + "MessagesPerSec",  "messages", TimeUnit.SECONDS)
-  val byteRate = newMeter(metricId + "BytesPerSec",  "bytes", TimeUnit.SECONDS)
+class ConsumerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup {
+  val messageRate = newMeter("MessagesPerSec", "messages", TimeUnit.SECONDS, metricId.toTags)
+  val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, metricId.toTags)
 }
 
 /**
  * Tracks metrics for each topic the given consumer client has consumed data from.
  * @param clientId The clientId of the given consumer client.
  */
-class ConsumerTopicStats(clientId: String) extends Logging {
-  private val valueFactory = (k: ClientIdAndTopic) => new ConsumerTopicMetrics(k)
-  private val stats = new Pool[ClientIdAndTopic, ConsumerTopicMetrics](Some(valueFactory))
-  private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAndTopic(clientId, "AllTopics")) // to differentiate from a topic named AllTopics
+class ConsumerTopicStats(clientId: Taggable) extends Logging {
+  private val valueFactory = (k: ClientIdTopic) => new ConsumerTopicMetrics(k)
+  private val stats = new Pool[ClientIdTopic, ConsumerTopicMetrics](Some(valueFactory))
+  private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics
 
   def getConsumerAllTopicStats(): ConsumerTopicMetrics = allTopicStats
 
   def getConsumerTopicStats(topic: String): ConsumerTopicMetrics = {
-    stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic + "-"))
+    stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic))
   }
 }
 
@@ -48,14 +48,14 @@
  * Stores the topic stats information of each consumer client in a (clientId -> ConsumerTopicStats) map.
  */
 object ConsumerTopicStatsRegistry {
-  private val valueFactory = (k: String) => new ConsumerTopicStats(k)
-  private val globalStats = new Pool[String, ConsumerTopicStats](Some(valueFactory))
+  private val valueFactory = (k: Taggable) => new ConsumerTopicStats(k)
+  private val globalStats = new Pool[Taggable, ConsumerTopicStats](Some(valueFactory))
 
-  def getConsumerTopicStat(clientId: String) = {
+  def getConsumerTopicStat(clientId: Taggable) = {
     globalStats.getAndMaybePut(clientId)
   }
 
-  def removeConsumerTopicStat(clientId: String) {
+  def removeConsumerTopicStat(clientId: Taggable) {
     globalStats.remove(clientId)
   }
 }
\ No newline at end of file
Index: core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala	(revision )
@@ -19,6 +19,7 @@
 
 import java.util.{Properties, Collection, ArrayList}
 
+import kafka.common.{TaggableInfo, BrokerSpecificInfo}
 import org.scalatest.junit.JUnit3Suite
 import org.junit.runners.Parameterized
 import org.junit.runner.RunWith
@@ -34,9 +35,7 @@
 import kafka.zk.ZooKeeperTestHarness
 import kafka.utils.{Utils, TestUtils}
 
-import scala.Array
 
-
 @RunWith(value = classOf[Parameterized])
 class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooKeeperTestHarness {
   private val brokerId = 0
@@ -76,7 +75,7 @@
     props.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000")
     props.put(ProducerConfig.LINGER_MS_CONFIG, "200")
     var producer = new KafkaProducer(props)
-    val consumer = new SimpleConsumer("localhost", port, 100, 1024*1024, "")
+    val consumer = new SimpleConsumer(new BrokerSpecificInfo("localhost", port), 100, 1024 * 1024, new TaggableInfo("clientId"->""))
 
     try {
       // create topic
Index: core/src/main/scala/kafka/utils/ZkUtils.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/utils/ZkUtils.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/utils/ZkUtils.scala	(revision )
@@ -18,7 +18,7 @@
 package kafka.utils
 
 import kafka.cluster.{Broker, Cluster}
-import kafka.consumer.{ConsumerThreadId, TopicCount}
+import kafka.consumer.{ConsumerId, ConsumerThreadId, TopicCount}
 import org.I0Itec.zkclient.ZkClient
 import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException,
   ZkMarshallingError, ZkBadVersionException}
@@ -30,10 +30,8 @@
 import kafka.common.{KafkaException, NoEpochForPartitionException}
 import kafka.controller.ReassignedPartitionsContext
 import kafka.controller.KafkaController
-import scala.Some
 import kafka.controller.LeaderIsrAndControllerEpoch
 import kafka.common.TopicAndPartition
-import scala.collection
 
 object ZkUtils extends Logging {
   val ConsumersPath = "/consumers"
@@ -663,7 +661,7 @@
     val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir)
     val consumersPerTopicMap = new mutable.HashMap[String, List[ConsumerThreadId]]
     for (consumer <- consumers) {
-      val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient, excludeInternalTopics)
+      val topicCount = TopicCount.constructTopicCount(group, new ConsumerId(consumer), zkClient, excludeInternalTopics)
       for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) {
         for (consumerThreadId <- consumerThreadIdSet)
           consumersPerTopicMap.get(topic) match {
Index: core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala	(revision )
@@ -24,5 +24,5 @@
 }
 
 case class HeartbeatResponseAndHeader(override val correlationId: Int, override val body: HeartbeatResponse)
-  extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) {
+  extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey).toString, None) {
 }
Index: core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/server/LogOffsetTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/server/LogOffsetTest.scala	(revision )
@@ -28,7 +28,7 @@
 import kafka.admin.AdminUtils
 import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest}
 import kafka.utils.TestUtils._
-import kafka.common.{ErrorMapping, TopicAndPartition}
+import kafka.common.{TaggableInfo, BrokerSpecificInfo, ErrorMapping, TopicAndPartition}
 import org.junit.After
 import org.junit.Before
 import org.junit.Test
@@ -51,7 +51,7 @@
     logDir = new File(logDirPath)
     time = new MockTime()
     server = TestUtils.createServer(new KafkaConfig(config), time)
-    simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "")
+    simpleConsumer = new SimpleConsumer(new BrokerSpecificInfo("localhost", brokerPort), 1000000, 64*1024, new TaggableInfo("clientId"->""))
   }
 
   @After
Index: core/src/main/scala/kafka/common/ClientIdAndBroker.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/common/ClientIdAndBroker.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/common/ClientIdAndBroker.scala	(revision )
@@ -8,7 +8,7 @@
  * (the "License"); you may not use this file except in compliance with
  * the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -21,6 +21,6 @@
  * Convenience case class since (clientId, brokerInfo) pairs are used to create
  * SyncProducer Request Stats and SimpleConsumer Request and Response Stats.
  */
-case class ClientIdAndBroker(clientId: String, brokerInfo: String) {
-  override def toString = "%s-%s".format(clientId, brokerInfo)
+case class ClientIdAndBroker(clientId: Taggable, brokerInfo: BrokerInfo) extends Taggable {
+  override def toTags = clientId.toTags ++ brokerInfo.toTags
 }
Index: core/src/main/scala/kafka/server/KafkaApis.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/KafkaApis.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/KafkaApis.scala	(revision )
@@ -256,9 +256,10 @@
         val numAppendedMessages = if (info.firstOffset == -1L || info.lastOffset == -1L) 0 else (info.lastOffset - info.firstOffset + 1)
 
         // update stats for successfully appended bytes and messages as bytesInRate and messageInRate
-        BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes)
+        val topic: TaggableInfo = new TaggableInfo("topic" -> topicAndPartition.topic)
+        BrokerTopicStats.getBrokerTopicStats(topic).bytesInRate.mark(messages.sizeInBytes)
         BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes)
-        BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages)
+        BrokerTopicStats.getBrokerTopicStats(topic).messagesInRate.mark(numAppendedMessages)
         BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages)
 
         trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d"
@@ -289,7 +290,7 @@
             producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nere.getMessage))
           new ProduceResult(topicAndPartition, nere)
         case e: Throwable =>
-          BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark()
+          BrokerTopicStats.getBrokerTopicStats(new TaggableInfo("topic" -> topicAndPartition.topic)).failedProduceRequestRate.mark()
           BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark()
           error("Error processing ProducerRequest with correlation id %d from client %s on partition %s"
             .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e)
Index: core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala	(revision )
@@ -20,10 +20,10 @@
 import java.net.URI
 import java.text.SimpleDateFormat
 import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest}
-import kafka.consumer.SimpleConsumer
+import kafka.consumer.{ConsumerConfig, SimpleConsumer}
 import kafka.utils._
 import org.apache.log4j.Logger
-import kafka.common.TopicAndPartition
+import kafka.common.{TaggableInfo, BrokerSpecificInfo, TopicAndPartition}
 
 
 /**
@@ -42,7 +42,8 @@
         println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec")
     }
 
-    val consumer = new SimpleConsumer(config.url.getHost, config.url.getPort, 30*1000, 2*config.fetchSize, config.clientId)
+    ConsumerConfig.validateClientId(config.clientId)
+    val consumer = new SimpleConsumer(new BrokerSpecificInfo(config.url.getHost, config.url.getPort), 30 * 1000, 2 * config.fetchSize, new TaggableInfo("clientId"->config.clientId))
 
     // reset to latest or smallest offset
     val topicAndPartition = TopicAndPartition(config.topic, config.partition)
Index: core/src/main/scala/kafka/server/KafkaRequestHandler.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/KafkaRequestHandler.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/KafkaRequestHandler.scala	(revision )
@@ -17,6 +17,7 @@
 
 package kafka.server
 
+import kafka.common.{TopicAllInfo, Taggable}
 import kafka.network._
 import kafka.utils._
 import kafka.metrics.KafkaMetricsGroup
@@ -93,23 +94,23 @@
   }
 }
 
-class BrokerTopicMetrics(name: String) extends KafkaMetricsGroup {
-  val messagesInRate = newMeter(name + "MessagesInPerSec",  "messages", TimeUnit.SECONDS)
-  val bytesInRate = newMeter(name + "BytesInPerSec",  "bytes", TimeUnit.SECONDS)
-  val bytesOutRate = newMeter(name + "BytesOutPerSec",  "bytes", TimeUnit.SECONDS)
-  val bytesRejectedRate = newMeter(name + "BytesRejectedPerSec",  "bytes", TimeUnit.SECONDS)
-  val failedProduceRequestRate = newMeter(name + "FailedProduceRequestsPerSec",  "requests", TimeUnit.SECONDS)
-  val failedFetchRequestRate = newMeter(name + "FailedFetchRequestsPerSec",  "requests", TimeUnit.SECONDS)
+class BrokerTopicMetrics(name: Taggable) extends KafkaMetricsGroup {
+  val messagesInRate = newMeter("MessagesInPerSec", "messages", TimeUnit.SECONDS, name.toTags)
+  val bytesInRate = newMeter("BytesInPerSec", "bytes", TimeUnit.SECONDS, name.toTags)
+  val bytesOutRate = newMeter("BytesOutPerSec", "bytes", TimeUnit.SECONDS, name.toTags)
+  val bytesRejectedRate = newMeter("BytesRejectedPerSec", "bytes", TimeUnit.SECONDS, name.toTags)
+  val failedProduceRequestRate = newMeter("FailedProduceRequestsPerSec", "requests", TimeUnit.SECONDS, name.toTags)
+  val failedFetchRequestRate = newMeter("FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS, name.toTags)
 }
 
 object BrokerTopicStats extends Logging {
-  private val valueFactory = (k: String) => new BrokerTopicMetrics(k)
-  private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory))
-  private val allTopicsStats = new BrokerTopicMetrics("AllTopics")
+  private val valueFactory = (k: Taggable) => new BrokerTopicMetrics(k)
+  private val stats = new Pool[Taggable, BrokerTopicMetrics](Some(valueFactory))
+  private val allTopicsStats = new BrokerTopicMetrics(TopicAllInfo())
 
   def getBrokerAllTopicsStats(): BrokerTopicMetrics = allTopicsStats
 
-  def getBrokerTopicStats(topic: String): BrokerTopicMetrics = {
-    stats.getAndMaybePut(topic + "-")
+  def getBrokerTopicStats(topic: Taggable): BrokerTopicMetrics = {
+    stats.getAndMaybePut(topic)
   }
 }
Index: core/src/main/scala/kafka/cluster/Partition.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/cluster/Partition.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/cluster/Partition.scala	(revision )
@@ -29,12 +29,13 @@
 import java.io.IOException
 import java.util.concurrent.locks.ReentrantReadWriteLock
 import kafka.utils.Utils.{inReadLock,inWriteLock}
-import scala.Some
 import scala.collection.immutable.Set
 
 import com.yammer.metrics.core.Gauge
 
+import scala.collection.mutable
 
+
 /**
  * Data structure that represents a topic partition. The leader maintains the AR, ISR, CUR, RAR
  */
@@ -63,12 +64,12 @@
   private def isReplicaLocal(replicaId: Int) : Boolean = (replicaId == localBrokerId)
 
   newGauge(
-    topic + "-" + partitionId + "-UnderReplicated",
+    "UnderReplicated",
     new Gauge[Int] {
       def value = {
         if (isUnderReplicated) 1 else 0
       }
-    }
+    }, mutable.LinkedHashMap("topic" -> topic, "partitionId" -> partitionId.toString)
   )
 
   def isUnderReplicated(): Boolean = {
Index: core/src/main/scala/kafka/network/SocketServer.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/network/SocketServer.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/network/SocketServer.scala	(revision )
@@ -26,7 +26,7 @@
 
 import scala.collection._
 
-import kafka.common.KafkaException
+import kafka.common.{KafkaException}
 import kafka.metrics.KafkaMetricsGroup
 import kafka.utils._
 import com.yammer.metrics.core.{Gauge, Meter}
@@ -67,7 +67,7 @@
                                     time, 
                                     maxRequestSize, 
                                     aggregateIdleMeter,
-                                    newMeter("NetworkProcessor-" + i + "-IdlePercent", "percent", TimeUnit.NANOSECONDS),
+                                    newMeter("IdlePercent", "percent", TimeUnit.NANOSECONDS, Map("NetworkProcessorNum" -> i.toString)),
                                     numProcessorThreads, 
                                     requestChannel,
                                     quotas,
Index: core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala	(revision )
@@ -27,7 +27,7 @@
 import org.scalatest.junit.JUnit3Suite
 import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest}
 import kafka.utils.TestUtils._
-import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition}
+import kafka.common._
 import scala.util.Random
 import scala.collection._
 
@@ -50,7 +50,7 @@
     logDir = new File(logDirPath)
     time = new MockTime()
     server = TestUtils.createServer(new KafkaConfig(config), time)
-    simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024, "test-client")
+    simpleConsumer = new SimpleConsumer(new BrokerSpecificInfo("localhost", brokerPort), 1000000, 64*1024, new TaggableInfo("clientId"->"test-client"))
     val consumerMetadataRequest = ConsumerMetadataRequest(group)
     Stream.continually {
       val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest)
Index: core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala	(revision )
@@ -104,24 +104,26 @@
   private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null
 
   // useful for tracking migration of consumers to store offsets in kafka
-  private val kafkaCommitMeter = newMeter(config.clientId + "-KafkaCommitsPerSec", "commits", TimeUnit.SECONDS)
-  private val zkCommitMeter = newMeter(config.clientId + "-ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS)
-  private val rebalanceTimer = new KafkaTimer(newTimer(config.clientId + "-RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
+  val metricTags = Map("clientId" -> config.clientId)
+  private val kafkaCommitMeter = newMeter("KafkaCommitsPerSec", "commits", TimeUnit.SECONDS, metricTags)
+  private val zkCommitMeter = newMeter("ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS, metricTags)
+  private val rebalanceTimer = new KafkaTimer(newTimer("RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, metricTags))
 
-  val consumerIdString = {
-    var consumerUuid : String = null
+  val consumerId = {
+    var consumerUuid: mutable.LinkedHashMap[String, String] = null
     config.consumerId match {
       case Some(consumerId) // for testing only
-      => consumerUuid = consumerId
+      => consumerUuid = mutable.LinkedHashMap("consumerId" -> consumerId)
       case None // generate unique consumerId automatically
       => val uuid = UUID.randomUUID()
-      consumerUuid = "%s-%d-%s".format(
-        InetAddress.getLocalHost.getHostName, System.currentTimeMillis,
-        uuid.getMostSignificantBits().toHexString.substring(0,8))
+        consumerUuid = mutable.LinkedHashMap(
+          "consumerHostName" -> InetAddress.getLocalHost.getHostName,
+          "timestamp" -> System.currentTimeMillis.toString,
+          "uuid" -> uuid.getMostSignificantBits().toHexString.substring(0, 8))
     }
-    config.groupId + "_" + consumerUuid
+    ConsumerId(mutable.LinkedHashMap((mutable.LinkedHashMap("groupId" -> config.groupId) ++ consumerUuid).iterator.toSeq:_*))
   }
-  this.logIdent = "[" + consumerIdString + "], "
+  this.logIdent = "[" + consumerId.toString + "], "
 
   connectZk()
   createFetcher()
@@ -162,7 +164,7 @@
 
   private def createFetcher() {
     if (enableFetcher)
-      fetcher = Some(new ConsumerFetcherManager(consumerIdString, config, zkClient))
+      fetcher = Some(new ConsumerFetcherManager(consumerId, config, zkClient))
   }
 
   private def connectZk() {
@@ -185,7 +187,7 @@
     if (canShutdown) {
       info("ZKConsumerConnector shutting down")
       val startTime = System.nanoTime()
-      KafkaMetricsGroup.removeAllConsumerMetrics(config.clientId)
+      KafkaMetricsGroup.removeAllConsumerMetrics(new TaggableInfo("clientId"->config.clientId))
       rebalanceLock synchronized {
         if (wildcardTopicWatcher != null)
           wildcardTopicWatcher.shutdown()
@@ -220,7 +222,7 @@
     if (topicCountMap == null)
       throw new RuntimeException("topicCountMap is null")
 
-    val topicCount = TopicCount.constructTopicCount(consumerIdString, topicCountMap)
+    val topicCount = TopicCount.constructTopicCount(consumerId, topicCountMap)
 
     val topicThreadIds = topicCount.getConsumerThreadIdsPerTopic
 
@@ -229,13 +231,13 @@
       threadIdSet.map(_ => {
         val queue =  new LinkedBlockingQueue[FetchedDataChunk](config.queuedMaxMessages)
         val stream = new KafkaStream[K,V](
-          queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, config.clientId)
+          queue, config.consumerTimeoutMs, keyDecoder, valueDecoder, new TaggableInfo("clientId"->config.clientId))
         (queue, stream)
       })
     ).flatten.toList
 
     val dirs = new ZKGroupDirs(config.groupId)
-    registerConsumerInZK(dirs, consumerIdString, topicCount)
+    registerConsumerInZK(dirs, consumerId, topicCount)
     reinitializeConsumer(topicCount, queuesAndStreams)
 
     loadBalancerListener.kafkaMessageAndMetadataStreams.asInstanceOf[Map[String, List[KafkaStream[K,V]]]]
@@ -244,15 +246,15 @@
   // this API is used by unit tests only
   def getTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]] = topicRegistry
 
-  private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) {
-    info("begin registering consumer " + consumerIdString + " in ZK")
+  private def registerConsumerInZK(dirs: ZKGroupDirs, consumerId: ConsumerId, topicCount: TopicCount) {
+    info("begin registering consumer " + consumerId + " in ZK")
     val timestamp = SystemTime.milliseconds.toString
     val consumerRegistrationInfo = Json.encode(Map("version" -> 1, "subscription" -> topicCount.getTopicCountMap, "pattern" -> topicCount.pattern,
                                                   "timestamp" -> timestamp))
 
-    createEphemeralPathExpectConflictHandleZKBug(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, consumerRegistrationInfo, null,
+    createEphemeralPathExpectConflictHandleZKBug(zkClient, dirs.consumerRegistryDir + "/" + consumerId, consumerRegistrationInfo, null,
                                                  (consumerZKString, consumer) => true, config.zkSessionTimeoutMs)
-    info("end registering consumer " + consumerIdString + " in ZK")
+    info("end registering consumer " + consumerId + " in ZK")
   }
 
   private def sendShutdownToAllQueues() = {
@@ -451,7 +453,7 @@
 
 
   class ZKSessionExpireListener(val dirs: ZKGroupDirs,
-                                 val consumerIdString: String,
+                                 val consumerId: ConsumerId,
                                  val topicCount: TopicCount,
                                  val loadBalancerListener: ZKRebalancerListener)
     extends IZkStateListener {
@@ -474,9 +476,9 @@
        *  connection for us. We need to release the ownership of the current consumer and re-register this
        *  consumer in the consumer registry and trigger a rebalance.
        */
-      info("ZK expired; release old broker parition ownership; re-register consumer " + consumerIdString)
+      info("ZK expired; release old broker parition ownership; re-register consumer " + consumerId)
       loadBalancerListener.resetState()
-      registerConsumerInZK(dirs, consumerIdString, topicCount)
+      registerConsumerInZK(dirs, consumerId, topicCount)
       // explicitly trigger load balancing for this consumer
       loadBalancerListener.syncedRebalance()
       // There is no need to resubscribe to child and state changes.
@@ -507,7 +509,7 @@
     }
   }
 
-  class ZKRebalancerListener(val group: String, val consumerIdString: String,
+  class ZKRebalancerListener(val group: String, val consumerId: ConsumerId,
                              val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_,_]]])
     extends IZkChildListener {
 
@@ -518,16 +520,16 @@
     private val cond = lock.newCondition()
     
     @volatile private var allTopicsOwnedPartitionsCount = 0
-    newGauge(config.clientId + "-" + config.groupId + "-AllTopicsOwnedPartitionsCount", new Gauge[Int] {
+    newGauge("OwnedPartitionsCount", new Gauge[Int] {
       def value() = allTopicsOwnedPartitionsCount
-    })
+    }, Map("clientId" -> config.clientId, "groupId" -> config.groupId) ++ TopicAllInfo().toTags)
 
-    private def ownedPartitionsCountMetricName(topic: String) =
-      "%s-%s-%s-OwnedPartitionsCount".format(config.clientId, config.groupId, topic)
+    private def ownedPartitionsCountMetricTags(topic: String) =
+      Map("clientId" -> config.clientId, "groupId" -> config.groupId, "topic" -> topic)
 
-    private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") {
+    private val watcherExecutorThread = new Thread(consumerId + "_watcher_executor") {
       override def run() {
-        info("starting watcher executor thread for consumer " + consumerIdString)
+        info("starting watcher executor thread for consumer " + consumerId)
         var doRebalance = false
         while (!isShuttingDown.get) {
           try {
@@ -546,7 +548,7 @@
             case t: Throwable => error("error during syncedRebalance", t)
           }
         }
-        info("stopping watcher executor thread for consumer " + consumerIdString)
+        info("stopping watcher executor thread for consumer " + consumerId)
       }
     }
     watcherExecutorThread.start()
@@ -567,7 +569,7 @@
       val topicDirs = new ZKGroupTopicDirs(group, topic)
       val znode = topicDirs.consumerOwnerDir + "/" + partition
       deletePath(zkClient, znode)
-      debug("Consumer " + consumerIdString + " releasing " + znode)
+      debug("Consumer " + consumerId + " releasing " + znode)
     }
 
     private def releasePartitionOwnership(localTopicRegistry: Pool[String, Pool[Int, PartitionTopicInfo]])= {
@@ -576,7 +578,7 @@
         for(partition <- infos.keys) {
           deletePartitionOwnershipFromZK(topic, partition)
         }
-        removeMetric(ownedPartitionsCountMetricName(topic))
+        removeMetric("OwnedPartitionsCount", ownedPartitionsCountMetricTags(topic))
         localTopicRegistry.remove(topic)
       }
       allTopicsOwnedPartitionsCount = 0
@@ -593,7 +595,7 @@
             return
           } else {
             for (i <- 0 until config.rebalanceMaxRetries) {
-              info("begin rebalancing consumer " + consumerIdString + " try #" + i)
+              info("begin rebalancing consumer " + consumerId + " try #" + i)
               var done = false
               var cluster: Cluster = null
               try {
@@ -607,7 +609,7 @@
                     **/
                   info("exception during rebalance ", e)
               }
-              info("end rebalancing consumer " + consumerIdString + " try #" + i)
+              info("end rebalancing consumer " + consumerId + " try #" + i)
               if (done) {
                 return
               } else {
@@ -623,12 +625,12 @@
         }
       }
 
-      throw new ConsumerRebalanceFailedException(consumerIdString + " can't rebalance after " + config.rebalanceMaxRetries +" retries")
+      throw new ConsumerRebalanceFailedException(consumerId + " can't rebalance after " + config.rebalanceMaxRetries +" retries")
     }
 
     private def rebalance(cluster: Cluster): Boolean = {
       val myTopicThreadIdsMap = TopicCount.constructTopicCount(
-        group, consumerIdString, zkClient, config.excludeInternalTopics).getConsumerThreadIdsPerTopic
+        group, consumerId, zkClient, config.excludeInternalTopics).getConsumerThreadIdsPerTopic
       val brokers = getAllBrokersInCluster(zkClient)
       if (brokers.size == 0) {
         // This can happen in a rare case when there are no brokers available in the cluster when the consumer is started.
@@ -649,7 +651,7 @@
 
         releasePartitionOwnership(topicRegistry)
 
-        val assignmentContext = new AssignmentContext(group, consumerIdString, config.excludeInternalTopics, zkClient)
+        val assignmentContext = new AssignmentContext(group, consumerId, config.excludeInternalTopics, zkClient)
         val partitionOwnershipDecision = partitionAssignor.assign(assignmentContext)
         val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]](
           valueFactory = Some((topic: String) => new Pool[Int, PartitionTopicInfo]))
@@ -679,9 +681,9 @@
 
             partitionOwnershipDecision.view.groupBy { case(topicPartition, consumerThreadId) => topicPartition.topic }
                                       .foreach { case (topic, partitionThreadPairs) =>
-              newGauge(ownedPartitionsCountMetricName(topic), new Gauge[Int] {
+              newGauge("OwnedPartitionsCount", new Gauge[Int] {
                 def value() = partitionThreadPairs.size
-              })
+              }, ownedPartitionsCountMetricTags(topic))
             }
 
             topicRegistry = currentTopicRegistry
@@ -747,7 +749,7 @@
       for (partitionInfos <- topicRegistry.values)
         for (partition <- partitionInfos.values)
           allPartitionInfos ::= partition
-      info("Consumer " + consumerIdString + " selected partitions : " +
+      info("Consumer " + consumerId + " selected partitions : " +
         allPartitionInfos.sortWith((s,t) => s.partitionId < t.partitionId).map(_.toString).mkString(","))
 
       fetcher match {
@@ -801,7 +803,7 @@
                                                  consumedOffset,
                                                  fetchedOffset,
                                                  new AtomicInteger(config.fetchMessageMaxBytes),
-                                                 config.clientId)
+                                                 new TaggableInfo("clientId" -> config.clientId))
       partTopicInfoMap.put(partition, partTopicInfo)
       debug(partTopicInfo + " selected new offset " + offset)
       checkpointedZkOffsets.put(TopicAndPartition(topic, partition), offset)
@@ -818,13 +820,13 @@
     if (loadBalancerListener == null) {
       val topicStreamsMap = new mutable.HashMap[String,List[KafkaStream[K,V]]]
       loadBalancerListener = new ZKRebalancerListener(
-        config.groupId, consumerIdString, topicStreamsMap.asInstanceOf[scala.collection.mutable.Map[String, List[KafkaStream[_,_]]]])
+        config.groupId, consumerId, topicStreamsMap.asInstanceOf[scala.collection.mutable.Map[String, List[KafkaStream[_,_]]]])
     }
 
     // create listener for session expired event if not exist yet
     if (sessionExpirationListener == null)
       sessionExpirationListener = new ZKSessionExpireListener(
-        dirs, consumerIdString, topicCount, loadBalancerListener)
+        dirs, consumerId, topicCount, loadBalancerListener)
 
     // create listener for topic partition change event if not exist yet
     if (topicPartitionChangeListener == null)
@@ -863,10 +865,11 @@
       topicThreadIdAndQueues.put(topicThreadId, q)
       debug("Adding topicThreadId %s and queue %s to topicThreadIdAndQueues data structure".format(topicThreadId, q.toString))
       newGauge(
-        config.clientId + "-" + config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize",
+        "FetchQueueSize",
         new Gauge[Int] {
           def value = q.size
-        }
+        },
+        Map("clientId" -> config.clientId, "topicThreadId" -> topicThreadId._1) ++ topicThreadId._2.toTags
       )
     })
 
@@ -910,7 +913,7 @@
                                           config.consumerTimeoutMs,
                                           keyDecoder,
                                           valueDecoder,
-                                          config.clientId)
+                                          new TaggableInfo("clientId"->config.clientId))
         (queue, stream)
     }).toList
 
@@ -920,10 +923,10 @@
         .filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics))
 
     private val wildcardTopicCount = TopicCount.constructTopicCount(
-      consumerIdString, topicFilter, numStreams, zkClient, config.excludeInternalTopics)
+      consumerId, topicFilter, numStreams, zkClient, config.excludeInternalTopics)
 
     val dirs = new ZKGroupDirs(config.groupId)
-    registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount)
+    registerConsumerInZK(dirs, consumerId, wildcardTopicCount)
     reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams)
 
     /*
Index: core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala	(revision )
@@ -16,6 +16,7 @@
  */
 package kafka.server
 
+import kafka.common.{TaggableInfo, BrokerSpecificInfo}
 import kafka.zk.ZooKeeperTestHarness
 import kafka.consumer.SimpleConsumer
 import kafka.producer._
@@ -74,7 +75,7 @@
     producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromConfigs(Seq(config)),
       encoder = classOf[StringEncoder].getName,
       keyEncoder = classOf[IntEncoder].getName)
-    val consumer = new SimpleConsumer(host, port, 1000000, 64*1024, "")
+    val consumer = new SimpleConsumer(new BrokerSpecificInfo(host, port), 1000000, 64*1024, new TaggableInfo("clientId"->""))
 
     var fetchedMessage: ByteBufferMessageSet = null
     while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
Index: core/src/main/scala/kafka/tools/MirrorMaker.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/tools/MirrorMaker.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/tools/MirrorMaker.scala	(revision )
@@ -196,7 +196,7 @@
     // time should be discounted by # threads.
     private val waitPut = newMeter("MirrorMaker-DataChannel-WaitOnPut", "percent", TimeUnit.NANOSECONDS)
     private val waitTake = newMeter("MirrorMaker-DataChannel-WaitOnTake", "percent", TimeUnit.NANOSECONDS)
-    private val channelSizeHist = newHistogram("MirrorMaker-DataChannel-Size")
+    private val channelSizeHist = newHistogram("MirrorMaker-DataChannel-Size", true)
 
     def put(record: ProducerRecord) {
       // If the key of the message is empty, use round-robin to select the queue
Index: core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala	(revision )
@@ -19,7 +19,6 @@
 import org.apache.kafka.common.requests.{HeartbeatResponse, HeartbeatRequest}
 import kafka.api.ApiUtils._
 import kafka.network.RequestChannel.Response
-import scala.Some
 
 object HeartbeatRequestAndHeader {
   def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = {
@@ -35,7 +34,7 @@
                                      override val correlationId: Int,
                                      override val clientId: String,
                                      override val body: HeartbeatRequest)
-  extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) {
+  extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey).toString, Some(RequestKeys.HeartbeatKey)) {
 
   override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
     val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
Index: core/src/main/scala/kafka/log/LogCleanerManager.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/log/LogCleanerManager.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/log/LogCleanerManager.scala	(revision )
@@ -59,7 +59,7 @@
   
   /* a gauge for tracking the cleanable ratio of the dirtiest log */
   @volatile private var dirtiestLogCleanableRatio = 0.0
-  newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt })
+  newGauge("MaxDirtyPercent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt })
 
   /**
    * @return the position processed for all logs.
Index: core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala	(revision )
@@ -141,7 +141,7 @@
     EasyMock.expectLastCall().anyTimes()
 
     scenario.subscriptions.foreach { case(consumerId, subscriptionInfo) =>
-      EasyMock.expect(zkClient.readData("/consumers/%s/ids/%s".format(scenario.group, consumerId), new Stat()))
+      EasyMock.expect(zkClient.readData("/consumers/%s/ids/%s".format(scenario.group, new ConsumerId(consumerId)), new Stat()))
               .andReturn(subscriptionInfo.registrationString)
       EasyMock.expectLastCall().anyTimes()
     }
@@ -163,7 +163,7 @@
   private def assignAndVerify(scenario: Scenario, assignor: PartitionAssignor, zkClient: ZkClient,
                               verifyAssignmentIsUniform: Boolean = false) {
     val assignments = scenario.subscriptions.map{ case(consumer, subscription)  =>
-      val ctx = new AssignmentContext("g1", consumer, excludeInternalTopics = true, zkClient)
+      val ctx = new AssignmentContext("g1", new ConsumerId(consumer), excludeInternalTopics = true, zkClient)
       assignor.assign(ctx)
     }
 
Index: core/src/main/scala/kafka/consumer/ConsumerIterator.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/ConsumerIterator.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/ConsumerIterator.scala	(revision )
@@ -17,12 +17,12 @@
 
 package kafka.consumer
 
-import kafka.utils.{IteratorTemplate, Logging, Utils}
+import kafka.utils.{IteratorTemplate, Logging}
 import java.util.concurrent.{TimeUnit, BlockingQueue}
 import kafka.serializer.Decoder
 import java.util.concurrent.atomic.AtomicReference
 import kafka.message.{MessageAndOffset, MessageAndMetadata}
-import kafka.common.{KafkaException, MessageSizeTooLargeException}
+import kafka.common.{Taggable, KafkaException, MessageSizeTooLargeException}
 
 
 /**
@@ -34,7 +34,7 @@
                              consumerTimeoutMs: Int,
                              private val keyDecoder: Decoder[K],
                              private val valueDecoder: Decoder[V],
-                             val clientId: String)
+                             val clientId: Taggable)
   extends IteratorTemplate[MessageAndMetadata[K, V]] with Logging {
 
   private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null)
Index: core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala	(revision )
@@ -17,6 +17,7 @@
 
 package kafka.javaapi.consumer
 
+import kafka.common.{TaggableInfo, BrokerSpecificInfo}
 import kafka.utils.threadsafe
 import kafka.javaapi.FetchResponse
 import kafka.javaapi.OffsetRequest
@@ -31,7 +32,7 @@
                      val bufferSize: Int,
                      val clientId: String) {
 
-  private val underlying = new kafka.consumer.SimpleConsumer(host, port, soTimeout, bufferSize, clientId)
+  private val underlying = new kafka.consumer.SimpleConsumer(new BrokerSpecificInfo(host, port), soTimeout, bufferSize, new TaggableInfo("clientId"->clientId))
 
   /**
    *  Fetch a set of messages from a topic. This version of the fetch method
Index: core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala	(revision )
@@ -109,8 +109,8 @@
 
     // also check partition ownership
     val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
-    val expected_1 = List( ("0", "group1_consumer1-0"),
-                           ("1", "group1_consumer1-0"))
+    val expected_1 = List( ("0", "groupId=group1,consumerId=consumer1,threadId=0"),
+                           ("1", "groupId=group1,consumerId=consumer1,threadId=0"))
     assertEquals(expected_1, actual_1)
 
     // commit consumed offsets
@@ -134,8 +134,8 @@
 
     // also check partition ownership
     val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir)
-    val expected_2 = List( ("0", "group1_consumer1-0"),
-                           ("1", "group1_consumer2-0"))
+    val expected_2 = List( ("0", "groupId=group1,consumerId=consumer1,threadId=0"),
+                           ("1", "groupId=group1,consumerId=consumer2,threadId=0"))
     assertEquals(expected_2, actual_2)
 
     // create a consumer with empty map
@@ -197,8 +197,8 @@
 
     // also check partition ownership
     val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
-    val expected_1 = List( ("0", "group1_consumer1-0"),
-                           ("1", "group1_consumer1-0"))
+    val expected_1 = List( ("0", "groupId=group1,consumerId=consumer1,threadId=0"),
+                           ("1", "groupId=group1,consumerId=consumer1,threadId=0"))
     assertEquals(expected_1, actual_1)
 
     // commit consumed offsets
@@ -222,8 +222,8 @@
 
     // also check partition ownership
     val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir)
-    val expected_2 = List( ("0", "group1_consumer1-0"),
-                           ("1", "group1_consumer2-0"))
+    val expected_2 = List( ("0", "groupId=group1,consumerId=consumer1,threadId=0"),
+                           ("1", "groupId=group1,consumerId=consumer2,threadId=0"))
     assertEquals(expected_2, actual_2)
 
     // create a consumer with empty map
@@ -268,8 +268,8 @@
 
     // also check partition ownership
     val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir)
-    val expected_2 = List( ("0", "group1_consumer0-0"),
-                           ("1", "group1_consumer0-0"))
+    val expected_2 = List( ("0", "groupId=group1,consumerId=consumer0,threadId=0"),
+                           ("1", "groupId=group1,consumerId=consumer0,threadId=0"))
     assertEquals(expected_2, actual_2)
 
     zkConsumerConnector1.shutdown
@@ -336,7 +336,7 @@
 
     // also check partition ownership
     val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
-    val expected_1 = List( ("0", "group1_consumer1-0"))
+    val expected_1 = List( ("0", "groupId=group1,consumerId=consumer1,threadId=0"))
     assertEquals(expected_1, actual_1)
 
     val receivedMessages1 = getMessages(nMessages, topicMessageStreams1)
Index: core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala	(revision )
@@ -21,16 +21,16 @@
 import kafka.server.AbstractFetcherThread
 import kafka.message.ByteBufferMessageSet
 import kafka.api.{Request, OffsetRequest, FetchResponsePartitionData}
-import kafka.common.TopicAndPartition
+import kafka.common.{TaggableInfo, TopicAndPartition}
 
 
-class ConsumerFetcherThread(name: String,
+class ConsumerFetcherThread(consumerFetcherThreadId: ConsumerFetcherThreadId,
                             val config: ConsumerConfig,
                             sourceBroker: Broker,
                             partitionMap: Map[TopicAndPartition, PartitionTopicInfo],
                             val consumerFetcherManager: ConsumerFetcherManager)
-        extends AbstractFetcherThread(name = name, 
-                                      clientId = config.clientId + "-" + name,
+        extends AbstractFetcherThread(name = consumerFetcherThreadId,
+                                      clientId = new TaggableInfo(new TaggableInfo("clientId" -> config.clientId), consumerFetcherThreadId),
                                       sourceBroker = sourceBroker,
                                       socketTimeout = config.socketTimeoutMs,
                                       socketBufferSize = config.socketReceiveBufferBytes,
Index: core/src/main/scala/kafka/server/AbstractFetcherThread.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/server/AbstractFetcherThread.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/server/AbstractFetcherThread.scala	(revision )
@@ -21,14 +21,12 @@
 import kafka.utils.{Pool, ShutdownableThread}
 import kafka.consumer.{PartitionTopicInfo, SimpleConsumer}
 import kafka.api.{FetchRequest, FetchResponse, FetchResponsePartitionData, FetchRequestBuilder}
-import kafka.common.{KafkaException, ClientIdAndBroker, TopicAndPartition, ErrorMapping}
+import kafka.common._
 import kafka.utils.Utils.inLock
 import kafka.message.{InvalidMessageException, ByteBufferMessageSet, MessageAndOffset}
 import kafka.metrics.KafkaMetricsGroup
 
-import scala.collection.mutable
-import scala.collection.Set
-import scala.collection.Map
+import scala.collection.{mutable, Set, Map}
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.locks.ReentrantLock
 import java.util.concurrent.atomic.AtomicLong
@@ -38,20 +36,23 @@
 /**
  *  Abstract class for fetching data from multiple partitions from the same broker.
  */
-abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int,
+abstract class AbstractFetcherThread(name: Taggable, clientId: Taggable, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int,
                                      fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1,
                                      isInterruptible: Boolean = true)
-  extends ShutdownableThread(name, isInterruptible) {
+  extends ShutdownableThread(name.toString, isInterruptible) {
   private val partitionMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map
   private val partitionMapLock = new ReentrantLock
   private val partitionMapCond = partitionMapLock.newCondition()
-  val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, clientId)
-  private val brokerInfo = "host_%s-port_%s".format(sourceBroker.host, sourceBroker.port)
+
+  private val brokerInfo = new BrokerSpecificInfo(sourceBroker.host, sourceBroker.port)
   private val metricId = new ClientIdAndBroker(clientId, brokerInfo)
+
+  val simpleConsumer = new SimpleConsumer(brokerInfo, socketTimeout, socketBufferSize, clientId)
+
   val fetcherStats = new FetcherStats(metricId)
   val fetcherLagStats = new FetcherLagStats(metricId)
   val fetchRequestBuilder = new FetchRequestBuilder().
-          clientId(clientId).
+          clientId(clientId.toString).
           replicaId(fetcherBrokerId).
           maxWait(maxWait).
           minBytes(minBytes)
@@ -207,10 +208,10 @@
 class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMetricsGroup {
   private[this] val lagVal = new AtomicLong(-1L)
   newGauge(
-    metricId + "-ConsumerLag",
+    "ConsumerLag",
     new Gauge[Long] {
       def value = lagVal.get
-    }
+    }, metricId.toTags
   )
 
   def lag_=(newLag: Long) {
@@ -230,11 +231,11 @@
 }
 
 class FetcherStats(metricId: ClientIdAndBroker) extends KafkaMetricsGroup {
-  val requestRate = newMeter(metricId + "-RequestsPerSec", "requests", TimeUnit.SECONDS)
-  val byteRate = newMeter(metricId + "-BytesPerSec", "bytes", TimeUnit.SECONDS)
+  val requestRate = newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, metricId.toTags)
+  val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, metricId.toTags)
 }
 
-case class ClientIdBrokerTopicPartition(clientId: String, brokerInfo: String, topic: String, partitionId: Int) {
-  override def toString = "%s-%s-%s-%d".format(clientId, brokerInfo, topic, partitionId)
+case class ClientIdBrokerTopicPartition(clientId: Taggable, brokerInfo: BrokerInfo, topic: String, partitionId: Int) extends Taggable {
+  override def toTags = clientId.toTags ++ mutable.LinkedHashMap("topic" -> topic, "partitionId" -> partitionId.toString) ++ brokerInfo.toTags
 }
 
Index: core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
IDEA additional info:
Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
<+>UTF-8
===================================================================
--- core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala	(revision eb7ac9eb7eebc4e0655b65e07cae594e61a6c05e)
+++ core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala	(revision )
@@ -19,9 +19,6 @@
 import kafka.api._
 import java.nio.ByteBuffer
 import scala.collection.mutable
-import kafka.network.{BoundedByteBufferSend, RequestChannel}
-import kafka.common.ErrorMapping
-import kafka.network.RequestChannel.Response
 
 class TopicMetadataRequest(val versionId: Short,
                            val correlationId: Int,
