> On Aug. 1, 2014, 11:33 p.m., Guozhang Wang wrote: > > To be more precisely, in the old clients we have four stats classes: > > > > 1. ProducerTopicStats > > 2. ProducerRequestStats > > 3. ProducerStats > > 4. FetchRequestAndResponseStats > > 5. ConsumerTopicStats > > > > Among them: > > > > 1) is accessed by Producer and DefaultEventHandler > > 2) is accessed by SyncProducer > > 3) is accessed by DefaultEventHandler > > 4) is accessed by SimpleConsumer > > 5) is accessed by ConsumerIterator, PartitionTopicInfo, and > > ZookeeperConsumerConnector > > > > All of the above modules can have or already have a close() call. Since for > > all of the metrics, we create new instances through client-ids. And hence, > > upon closing all these above modules we can just pass in the client id > > string to the Stats factories to close the metrics, and by doing so we do > > not need to keep track of the metrics owner. > > Jiangjie Qin wrote: > Thank you very much for the review, Guozhang. The reason I passed in the > metricOwner is because in this approach I have a global metric registry in > KafkaMetricsGroup. That means whennever a metric is created with > newGauge(),newMeter(),newTimer(),newHistogram(), a metric owner has to be > specified. The same metric owner is needed for deregister the metric from > yammer. > In those five stats, they do the following two things: > 1. create a kafka metric object(e.g. ConsumerTopicsMetrics) and put them > into their own registry. > 2. create yammer metrics and register them to yammer. > While the kafka metrics can be removed easily with a client-id, deleting > the yammer metrics needs more name translation. It could also potentially > make the interaction with KafkaMetricsGroup less unified as there are many > other stand alone metrics not using the stats factory. > Jun also pointed out another issue that the current approach with a > global yammer metrics map has insufficient isolation between objects and is > somewhat hacky. Instead of a global metric map, he suggested to create a > single metric registry for each instance and deregister the metrics in the > registry when the instance exits. Without a global registry, maybe the metric > owner is no longer needed to be passed in. > Thanks!
To make this simple, how about the following approach. a. We introduce KafkaMetricsGroup.removeMetric() as in KAFKA-687. b. Then, we add the logic to remove client metrics based on the client id when the clients are closed. This has the side effect that if two clients somehow use the same client id by mistake, closing one client will mean that the other client's metrics will disappear (the other client will still be functioning). However, this is probably not a big issue since with the same client id, the two clients' metrics are already mixed up and are not particularly useful. c. In addition to the stats that Guozhang listed above, we need to handle FetcherLagStats. - Jun ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/24196/#review49401 ----------------------------------------------------------- On Aug. 1, 2014, 9:26 p.m., Jiangjie Qin wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/24196/ > ----------------------------------------------------------- > > (Updated Aug. 1, 2014, 9:26 p.m.) > > > Review request for kafka. > > > Bugs: KAFKA-1567 > https://issues.apache.org/jira/browse/KAFKA-1567 > > > Repository: kafka > > > Description > ------- > > fix for metric memory leaking issue. > > > Diffs > ----- > > core/src/main/scala/kafka/client/ClientUtils.scala > ce7ede3f6d60e756e252257bd8c6fedc21f21e1c > core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala > b9e2bea7b442a19bcebd1b350d39541a8c9dd068 > core/src/main/scala/kafka/consumer/ConsumerIterator.scala > ac491b4da2583ef7227c67f5b8bc0fd731d705c3 > core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala > ff5f470f7aa304917d3295fcb7702291ce7fe0b5 > core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala > 875eeeb73cba5bd034349f5e7b6e16dfdf544254 > core/src/main/scala/kafka/consumer/KafkaStream.scala > 805e91677034edcca74c74dab01a6d6d437ae442 > core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala > 9c779ce072fb0a3426e96fd77f383cce6991a0f9 > core/src/main/scala/kafka/consumer/SimpleConsumer.scala > 0e64632210385ef63c2ad3445b55ac4f37a63df2 > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala > 65f518d47c7555c42c4bff39c211814831f4b8b6 > core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala > a20ab90165cc7ebb1cf44078efe23a53938c8df6 > core/src/main/scala/kafka/producer/Producer.scala > 4798481d573bbdce0ba39035c50f4c4411ad0469 > core/src/main/scala/kafka/producer/ProducerRequestStats.scala > 96942205a6a461e122e003add1ab9bcebde1fe16 > core/src/main/scala/kafka/producer/ProducerStats.scala > e1610d3c602fb0f5f4cc237cb8b4e0d168a41530 > core/src/main/scala/kafka/producer/ProducerTopicStats.scala > ed209f4773dedb09e9a34005e6849730229aa6e9 > core/src/main/scala/kafka/producer/SyncProducer.scala > 489f0077512d9a69be81649c490274964290fa40 > core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala > d8ac915de31a26d7aa67760d69373975cacd0c9d > core/src/main/scala/kafka/producer/async/ProducerSendThread.scala > 42e9c741c2dcef756416832f11d37678cb7710ee > core/src/main/scala/kafka/server/AbstractFetcherManager.scala > 9390edf37dae9815d4a3790fdd31dac422382698 > core/src/main/scala/kafka/server/AbstractFetcherThread.scala > 3b15254f32252cf824d7a292889ac7662d73ada1 > gradle.properties 4827769a3f8e34f0fe7e783eb58e44d4db04859b > > Diff: https://reviews.apache.org/r/24196/diff/ > > > Testing > ------- > > > Thanks, > > Jiangjie Qin > >