[
https://issues.apache.org/jira/browse/KAFKA-6320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16300890#comment-16300890
]
ASF GitHub Bot commented on KAFKA-6320:
---------------------------------------
junrao commented on a change in pull request #4351: KAFKA-6320: move ZK metrics
in KafkaHealthCheck to ZookeeperClient
URL: https://github.com/apache/kafka/pull/4351#discussion_r158420526
##########
File path: core/src/main/scala/kafka/zookeeper/ZooKeeperClient.scala
##########
@@ -54,10 +58,47 @@ class ZooKeeperClient(connectString: String,
private val inFlightRequests = new Semaphore(maxInFlightRequests)
private val stateChangeHandlers = new ConcurrentHashMap[String,
StateChangeHandler]().asScala
+ private val metricNames = Set[String]()
+
+ // The state map has to be created before creating ZooKeeper since it's
needed in the ZooKeeper callback.
+ private val stateToMeterMap = {
+ import KeeperState._
+ val stateToEventTypeMap = Map(
+ Disconnected -> "Disconnects",
+ SyncConnected -> "SyncConnects",
+ AuthFailed -> "AuthFailures",
+ ConnectedReadOnly -> "ReadOnlyConnects",
+ SaslAuthenticated -> "SaslAuthentications",
+ Expired -> "Expires"
+ )
+ stateToEventTypeMap.map { case (state, eventType) =>
+ val name = s"ZooKeeper${eventType}PerSec"
+ metricNames += name
+ state -> newMeter(name, eventType.toLowerCase(Locale.ROOT),
TimeUnit.SECONDS)
+ }
+ }
+
info(s"Initializing a new session to $connectString.")
@volatile private var zooKeeper = new ZooKeeper(connectString,
sessionTimeoutMs, ZooKeeperClientWatcher)
+
+ private val sessionStateGauge =
+ newGauge("SessionState", new Gauge[String] {
+ override def value: String =
+ Option(zooKeeper.getState.toString).getOrElse("DISCONNECTED")
+ })
+
+ metricNames += "SessionState"
+
waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS)
+
+ /**
+ * This is added to preserve the original metric name in JMX
+ */
+ override def metricName(name: String, metricTags:
scala.collection.Map[String, String]): MetricName = {
+ explicitMetricName("kafka.server", "KafkaHealthcheck", name, metricTags)
Review comment:
Good point. Added.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> move ZK metrics in KafkaHealthCheck to ZookeeperClient
> ------------------------------------------------------
>
> Key: KAFKA-6320
> URL: https://issues.apache.org/jira/browse/KAFKA-6320
> Project: Kafka
> Issue Type: Sub-task
> Affects Versions: 1.0.0
> Reporter: Jun Rao
> Assignee: Jun Rao
> Fix For: 1.1.0
>
>
> In KAFKA-5473, we will be de-commissioning the usage of KafkaHealthCheck. So,
> we need to move the ZK metrics SessionState and ZooKeeper${eventType}PerSec
> in that class to somewhere else (e.g. ZookeeperClient).
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)