Monitor Kafka with JMX

Apache Kafka® brokers and clients report many internal metrics. JMX is the default reporter, though you can add any pluggable reporter.

You can deploy Confluent Control Center for out-of-the-box Kafka cluster monitoring so you don’t have to build your own monitoring system. Control Center makes it easy to manage the entire Confluent Platform. Control Center is a web-based application that allows you to manage your cluster and to alert on triggers. Additionally, Control Center measures how long messages take to be delivered, and determines the source of any issues in your cluster.

Tip

  • Health+: Consider monitoring and managing your environment with Confluent Health+. Ensure the health of your clusters and minimize business disruption with intelligent alerts, monitoring, and proactive support based on best practices created by the inventors Apache Kafka®.
  • For on-premises Confluent Platform: for a practical guide to optimizing your Kafka deployment for various service goals including throughput, latency, durability and availability, and useful metrics to monitor for performance and cluster health for on-prem Kafka clusters, see the Optimizing Your Apache Kafka Deployment whitepaper.
  • For clusters in Confluent Cloud: for a practical guide to configuring, monitoring, and optimizing your Kafka client applications, see Developing Client Applications on Confluent Cloud.

See also

For an example that showcases how to monitor Kafka client application and Confluent Cloud metrics, and steps through various failure scenarios to see how they are reflected in the provided metrics, see the Observability for Apache Kafka® Clients to Confluent Cloud demo.

Server Metrics

Broker Metrics

Confluent Control Center and Confluent Cloud monitors the following important operational broker metrics aggregated across the cluster, and per broker or per topic where applicable. Control Center provides built-in dashboards for viewing these metrics, and Confluent recommends you set alerts at least on the first three.

kafka.server:type=KafkaServer,name=TotalDiskReadBytes
The total number of bytes read by the broker process, including reads from all disks. The total doesn’t include reads from page cache.
kafka.server:type=KafkaServer,name=TotalDiskWriteBytes
The total number of bytes written by the broker process, including writes from all disks.
kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount
Number of partitions whose in-sync replicas count is less than minIsr.
kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions
Number of under-replicated partitions (| ISR | < | current replicas |). Replicas that are added as part of a reassignment will not count toward this value. Alert if value is greater than 0.
kafka.server:type=ReplicaManager,name=ReassigningPartitions
Number of reassigning partitions.
kafka.cluster:type=Partition,topic={topic},name=UnderMinIsr,partition={partition}
Number of partitions whose in-sync replicas count is less than minIsr. These partitions will be unavailable to producers who use acks=all.
kafka.controller:type=KafkaController,name=OfflinePartitionsCount
Number of partitions that don’t have an active leader and are hence not writable or readable. Alert if value is greater than 0.
kafka.controller:type=KafkaController,name=ActiveControllerCount
Number of active controllers in the cluster. Alert if the aggregated sum across all brokers in the cluster is anything other than 1 because there should be exactly one controller per cluster.
kafka.controller:type=KafkaController,name=GlobalPartitionCount
Number of partitions across all topics in the cluster.
kafka.server:type=BrokerTopicMetrics,name=BytesInPerSec
Byte-in rate from clients.
kafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec
Byte-out rate from clients.
kafka.server:type=BrokerTopicMetrics,name=ReplicationBytesInPerSec
Byte-in rate from other brokers.
kafka.server:type=BrokerTopicMetrics,name=ReplicationBytesOutPerSec
Byte-out rate to other brokers.
kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower},version=([0-9]+)
The request rate. version refers to the API version of the request type. To get the total count for a specific request type, make sure that JMX monitoring tools aggregate across different versions.
kafka.server:type=BrokerTopicMetrics,name=TotalProduceRequestsPerSec
Produce request rate.
kafka.server:type=BrokerTopicMetrics,name=TotalFetchRequestsPerSec
Fetch request rate.
kafka.server:type=BrokerTopicMetrics,name=FailedProduceRequestsPerSec
Produce request rate for requests that failed.
kafka.server:type=BrokerTopicMetrics,name=FailedFetchRequestsPerSec
Fetch request rate for requests that failed.
kafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesInPerSec
Incoming byte rate of reassignment traffic.
kafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesOutPerSec
Outgoing byte rate of reassignment traffic.
kafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMs
Leader election rate and latency.
kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec
Unclean leader election rate.
kafka.server:type=ReplicaManager,name=PartitionCount
Number of partitions on this broker. This should be mostly even across all brokers.
kafka.server:type=ReplicaManager,name=LeaderCount
Number of leaders on this broker. This should be mostly even across all brokers. If not, set auto.leader.rebalance.enable to true on all brokers in the cluster.
kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent
Average fraction of time the request handler threads are idle. Values are between 0 (all resources are used) and 1 (all resources are available).
kafka.server:type=Produce,name=DelayQueueSize
Number of producer clients currently being throttled. The value can be any number greater than or equal to 0.
kafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercent
Average fraction of time the network processor threads are idle. Values are between 0 (all resources are used) and 1 (all resources are available).
kafka.network:type=RequestChannel,name=RequestQueueSize
Size of the request queue. A congested request queue will not be able to process incoming or outgoing requests.
kafka.network:type=RequestChannel,name=ResponseQueueSize
Size of the response queue. The response queue is unbounded. A congested response queue can result in delayed response times and memory pressure on the broker.
kafka.server:type=socket-server-metrics,listener={listener_name},networkProcessor={#},name=connection-count
Number of currently open connections to the broker.
kafka.server:type=socket-server-metrics,listener={listener_name},networkProcessor={#},name=connection-creation-rate
Number of new connections established per second.
kafka.network:type=RequestMetrics,name=TotalTimeMs,request={Produce|FetchConsumer|FetchFollower}
Total time in milliseconds to serve the specified request.
kafka.network:type=RequestMetrics,name=RequestQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request waits in the request queue.
kafka.network:type=RequestMetrics,name=LocalTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request is processed at the leader.
kafka.network:type=RequestMetrics,name=RemoteTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request waits for the follower. This is non-zero for produce requests when acks=all.
kafka.network:type=RequestMetrics,name=ResponseQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request waits in the response queue.
kafka.network:type=RequestMetrics,name=ResponseSendTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time to send the response.

Here are other available metrics you may optionally observe on a Kafka broker.

kafka.server:type=BrokerTopicMetrics,name=MessagesInPerSec
Aggregate incoming message rate.
kafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs
Log flush rate and time.
kafka.server:type=ReplicaManager,name=IsrShrinksPerSec
If a broker goes down, ISR for some of the partitions will shrink. When that broker is up again, ISR will be expanded once the replicas are fully caught up. Other than that, the expected value for both ISR shrink rate and expansion rate is 0.
kafka.server:type=ReplicaManager,name=IsrExpandsPerSec
When a broker is brought up after a failure, it starts catching up by reading from the leader. Once it is caught up, it gets added back to the ISR.
kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)

Lag in number of messages per follower replica. This is useful to know if the replica is slow or has stopped replicating from the leader and if the associated brokers need to be removed from the In-Sync Replicas list.

Important

This metric is internal to the cluster and does not represent the Kafka client application’s consumer lag.

kafka.server:type=DelayedOperationPurgatory,delayedOperation=Produce,name=PurgatorySize
Number of requests waiting in the producer purgatory. This should be non-zero when acks=all is used on the producer.
kafka.server:type=DelayedOperationPurgatory,delayedOperation=Fetch,name=PurgatorySize
Number of requests waiting in the fetch purgatory. This is high if consumers use a large value for fetch.wait.max.ms .

ZooKeeper Metrics

Confluent Control Center monitors the following important operational broker metrics relating to ZooKeeper. We expose counts for ZooKeeper state transitions, which can help to spot problems, e.g., with broker sessions to ZooKeeper. The metrics currently show the rate of transitions per second for each one of the possible states. Here is the list of the counters we expose, one for each possible ZooKeeper client states.

kafka.server:type=SessionExpireListener,name=ZooKeeperDisconnectsPerSec
ZooKeeper client is currently disconnected from the ensemble. The client lost its previous connection to a server and it is currently trying to reconnect. The session is not necessarily expired. Note that this metric tells you if the broker is disconnecting, but not if ZooKeeper is down. If you are checking system health, ZooKeeperExpiresPerSec is a better metric to help you determine this.
kafka.server:type=SessionExpireListener,name=ZooKeeperExpiresPerSec

The ZooKeeper session has expired. When a session expires, we can have leader changes and even a new controller. It is important to keep an eye on the number of such events across a Kafka cluster and if the overall number is high, then we have a few recommendations:

  1. Check the health of your network
  2. Check for garbage collection issues and tune it accordingly
  3. If necessary, increase the session time out by setting the value of zookeeper.session.timeout.ms.

Here are other available ZooKeeper metrics you may optionally observe on a Kafka broker.

kafka.server:type=SessionExpireListener,name=ZooKeeperSyncConnectsPerSec
ZooKeeper client is connected to the ensemble and ready to execute operations.
kafka.server:type=SessionExpireListener,name=ZooKeeperAuthFailuresPerSec
An attempt to connect to the ensemble failed because the client has not provided correct credentials.
kafka.server:type=SessionExpireListener,name=ZooKeeperReadOnlyConnectsPerSec
The server the client is connected to is currently LOOKING, which means that it is neither FOLLOWING nor LEADING. Consequently, the client can only read the ZooKeeper state, but not make any changes (create, delete, or set the data of znodes).
kafka.server:type=SessionExpireListener,name=ZooKeeperSaslAuthenticationsPerSec
Client has successfully authenticated.
kafka.server:type=SessionExpireListener,name=SessionState
Connection status of broker’s ZooKeeper session. Expected value is CONNECTED.

Producer Metrics

Starting with Kafka version 3.1.1, the producer exposes the following metrics.

MBean: kafka.producer:type=producer-metrics,client-id=([-.\w]+)

waiting-threads
The number of user threads blocked waiting for buffer memory to enqueue their records.
buffer-total-bytes
The maximum amount of buffer memory the client can use whether or not it is available.
buffer-available-bytes
The total amount of buffer memory that is not being used, either unallocated or in the free list.
bufferpool-wait-time
The fraction of time an appender waits for space allocation.
bufferpool-wait-time-ns-total
The total time in nanoseconds an appender waits for space allocation in nanoseconds.
flush-time-ns-total
The total time in nanoseconds the producer spent in Producer.flush.
txn-init-time-ns-total
The total time in nanoseconds that the producer spent initializing transactions for exactly-once semantics.
txn-begin-time-ns-total
The total time in nanoseconds the producer spent in beginTransaction for exactly-once semantics.
txn-send-offsets-time-ns-total
The total time the producer spent sending offsets to transactions in nanoseconds for exactly-once semantics.
txn-commit-time-ns-total
The total time in nanoseconds the producer spent committing transactions for exactly-once semantics.
txn-abort-time-ns-total
The total time in nanoseconds the producer spent aborting transactions for exactly-once semantics.

Global Request Metrics

Starting with 0.8.2, the producer exposes the following metrics:

MBean: kafka.producer:type=producer-metrics,client-id=([-.\w]+)

batch-size-avg
The average number of bytes sent per partition per-request.
batch-size-max
The max number of bytes sent per partition per-request.
batch-split-rate
The average number of batch splits per second.
batch-split-total
The total number of batch splits.
compression-rate-avg
The average compression rate of record batches, defined as the average ratio of the compressed batch size over the uncompressed size.
incoming-byte-rate
The average number of incoming bytes received per second from all servers.
metadata-age
The age, in seconds, of the current producer metadata being used.
outgoing-byte-rate
The average number of bytes sent per second to all servers.
produce-throttle-time-avg
The average time in milliseconds a request was throttled by a broker.
produce-throttle-time-max
The maximum time in milliseconds a request was throttled by a broker.
record-error-rate
The average per-second number of record sends that resulted in errors.
record-error-total
The total number of record sends that resulted in errors.
record-queue-time-avg
The average time in milliseconds record batches spent in the send buffer.
record-queue-time-max
The maximum time in milliseconds record batches spent in the send buffer.
record-retry-rate
The average per-second number of retried record sends.
record-retry-total
The total number of retried record sends.
record-send-rate
The average number of records sent per second.
record-send-total
The total number of records sent.
record-size-avg
The average record size.
record-size-max
The maximum record size.
records-per-request-avg
The average number of records per request.
request-rate
The average number of requests sent per second.
requests-in-flight
The current number of in-flight requests awaiting a response.

Global Connection Metrics

MBean: kafka.producer:type=producer-metrics,client-id=([-.\w]+)

connection-close-rate
Connections closed per second in the window.
connection-count
The current number of active connections.
connection-creation-rate
New connections established per second in the window.
io-ratio
The fraction of time the I/O thread spent doing I/O.
io-time-ns-avg
The average length of time for I/O per select call in nanoseconds.
io-wait-ratio
The fraction of time the I/O thread spent waiting.
io-wait-time-ns-avg
The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.
select-rate
Number of times the I/O layer checked for new I/O to perform per second.

Per-Broker Metrics

MBean: kafka.producer:type=producer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)

Besides the producer global request metrics, the following metrics are also available per broker:

incoming-byte-rate
The average number of bytes received per second from the broker.
outgoing-byte-rate
The average number of bytes sent per second to the broker.
request-size-avg
The average size of all requests in the window for a broker.
request-size-max
The maximum size of any request sent in the window for a broker.
request-rate
The average number of requests sent per second to the broker.
response-rate
The average number of responses received per second from the broker.

Per-Topic Metrics

MBean: kafka.producer:type=producer-topic-metrics,client-id=”{client-id}”,topic=”{topic}”

Besides the producer global request metrics, the following metrics are also available per topic:

byte-rate
The average number of bytes sent per second for a topic.
byte-total
The total number of bytes sent for a topic.
compression-rate
The average compression rate of record batches for a topic, defined as the average ratio of the compressed batch size divided by the uncompressed size.
record-error-rate
The average per-second number of record sends that resulted in errors for a topic.
record-error-total
The total number of record sends that resulted in errors for a topic.
record-retry-rate
The average per-second number of retried record sends for a topic.
record-retry-total
The total number of retried record sends for a topic.
record-send-rate
The average number of records sent per second for a topic.
record-send-total
The total number of records sent for a topic.

Audit Metrics

confluent-audit-metrics:name=audit-log-rate-per-minute
The number of audit logs created per minute. This metric is useful in cases where you need to know the number of audit logs created.
confluent-audit-metrics:name=fallback-rate-per-minute
The rate of audit log fallbacks per minute. This metric is useful in cases where you need to know the fallback rate of your audit logs.

Authorizer Metrics

confluent-authorizer-metrics:name=authorization-request-rate-per-minute
The number of authorization requests per minute. This metric is useful in cases where you need to know the exact number of authorization requests per minute.
confluent-authorizer-metrics:name=authorization-allowed-rate-per-minute
The number of authorizations allowed per minute. This metric is useful in cases where you need to know the rate of authorizations allowed per minute.
confluent-authorizer-metrics:name=authorization-denied-rate-per-minute
The number of authorizations denied per minute. This metric is useful in cases where you need to know the rate of authorizations denied per minute.

RBAC and LDAP Health Metrics

confluent.metadata:type=LdapGroupManager,name=failure-start-seconds-ago
The number of seconds since the last failed attempt to process metadata from the LDAP server. This is reset to zero on the next successful metadata refresh. This metric is available on brokers in the metadata cluster if LDAP group-based authorization is enabled. Alert if value is greater than zero.
confluent.metadata:type=KafkaAuthStore,name=writer-failure-start-seconds-ago
The number of seconds since the last failure in the writer that updates authentication or authorization metadata on topics in the metadata cluster. This is reset to zero after the next successful metadata update. This metric is available on brokers in the metadata cluster. Alert if value is greater than zero.
confluent.metadata:type=KafkaAuthStore,name=reader-failure-start-seconds-ago
The number of seconds since the last failure in the consumer that processes authentication or authorization metadata from the topics in the metadata cluster. This is reset to zero after the next successful metadata refresh. This metric is available on all brokers configured to use RBAC. Alert if value is greater than zero.
confluent.metadata:type=KafkaAuthStore,name=remote-failure-start-seconds-ago
The number of seconds since the last failure in the metadata service, for example, due to LDAP refresh failures for a long duration. This is reset to zero when notification of successful refresh from the metadata service is processed. This metric is available on all brokers configured to use RBAC. Alert if value is greater than zero.
confluent.metadata:type=KafkaAuthStore,name=active-writer-count
Number of active writers in the metadata cluster. Alert if the sum is any number other than one because there should be exactly one writer in the metadata cluster.
confluent.metadata:type=KafkaAuthStore,name=metadata-status,topic=([-.\w]+),partition=([0-9]+)
Current status of metadata on each metadata topic partition. Value may be UNKNOWN, INITIALIZING, INITIALIZED or FAILED.
confluent.metadata:type=KafkaAuthStore,name=record-send-rate,topic=([-.\w]+),partition=([0-9]+)
The average number of records sent per second to the metadata topic partitions.
confluent.metadata:type=KafkaAuthStore,name=record-error-rate,topic=([-.\w]+),partition=([0-9]+)
The average number of record send attempts per second to the metadata topic partitions that failed.
confluent-auth-store-metrics:name=rbac-role-bindings-count
The number of role bindings defined. This metric is useful in cases where you need to know the exact number of role bindings that exist.
confluent-auth-store-metrics:name=rbac-access-rules-count
The number of RBAC access rules defined. This metric is useful in cases where you need to know the exact number of RBAC access rules that exist. Access rules allow or deny access to specific resources within a specific scope, unlike role bindings, which assign an RBAC role for a specific resource to a specific principal.
confluent-auth-store-metrics:name=acl-access-rules-count
The number of ACL access rules defined. This metric is useful in cases where you need to know the exact number of ACLs that exist.

Consumer Metrics

Starting with Kafka 3.1.1, the consumer exposes the following metrics:

MBean: kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)

committed-time-ns-total
The cumulative sum of time in nanoseconds elapsed during calls to Consumer.committed.
commit-sync-time-ns-total
The cumulative sum of time in nanoseconds elapsed during calls to Consumer.commitSync.

Starting with Kafka 2.4.0, the consumer exposes the following metrics:

MBean: kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)

time-between-poll-avg
The average delay between invocations of poll().
time-between-poll-max
The max delay between invocations of poll().
last-poll-seconds-ago
The number of seconds since the last poll() invocation.
poll-idle-ratio-avg
The average fraction of time the consumer’s poll() is idle as opposed to waiting for the user code to process records.

Fetch Metrics

Starting with Kafka 0.9.0.0, the consumer exposes the following metrics:

MBean: kafka.consumer:type=consumer-fetch-manager-metrics,client-id=”{client-id}”

bytes-consumed-rate
The average number of bytes consumed per second.
bytes-consumed-total
The total number of bytes consumed.
fetch-latency-avg
The average time taken for a fetch request.
fetch-latency-max
The max time taken for a fetch request.
fetch-rate
The number of fetch requests per second.
fetch-size-avg
The average number of bytes fetched per request.
fetch-size-max
The maximum number of bytes fetched per request.
fetch-throttle-time-avg
The average throttle time in milliseconds. When quotas are enabled, the broker may delay fetch requests in order to throttle a consumer which has exceeded its limit. This metric indicates how throttling time has been added to fetch requests on average.
fetch-throttle-time-max
The maximum throttle time in milliseconds.
fetch-total
The total number of fetch requests.
records-consumed-rate
The average number of records consumed per second.
records-consumed-total
The total number of records consumed.
records-lag-max
The maximum lag in terms of number of records for any partition in this window. An increasing value over time is your best indication that the consumer group is not keeping up with the producers.
records-lead-min
The minimum lead in terms of number of records for any partition in this window.
records-per-request-avg
The average number of records in each request.

Topic-level Fetch Metrics

MBean: kafka.consumer:type=consumer-fetch-manager-metrics,client-id=”{client-id}”,topic=”{topic}”

bytes-consumed-rate
The average number of bytes consumed per second for a specific topic.
bytes-consumed-total
The total number of bytes consumed for a specific topic.
fetch-size-avg
The average number of bytes fetched per request for a specific topic.
fetch-size-max
The maximum number of bytes fetched per request for a specific topic.
records-consumed-rate
The average number of records consumed per second for a specific topic.
records-consumed-total
The total number of records consumed for a specific topic.
records-per-request-avg
The average number of records in each request for a specific topic.

Partition-level Fetch Metrics

MBean: kafka.consumer:type=consumer-fetch-manager-metrics,partition=”{partition}”,topic=”{topic}”,client-id=”{client-id}

preferred-read-replica
The current read replica for the partition, or -1 if reading from leader.
records-lag
The latest lag of the partition.
records-lag-avg
The average lag of the partition.
records-lag-max
The max lag of the partition.
records-lead
The latest lead of the partition.
records-lead-avg
The average lead of the partition.
records-lead-min
The min lead of the partition.

Consumer Group Metrics

MBean: kafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)

assigned-partitions
The number of partitions currently assigned to this consumer.
commit-latency-avg
The average time taken for a commit request.
commit-latency-max
The max time taken for a commit request.
commit-rate
The number of commit calls per second.
heartbeat-rate
The average number of heartbeats per second. After a rebalance, the consumer sends heartbeats to the coordinator to keep itself active in the group. You can control this using the heartbeat.interval.ms setting for the consumer. You may see a lower rate than configured if the processing loop is taking more time to handle message batches. Usually this is OK as long as you see no increase in the join rate.
heartbeat-response-time-max
The max time taken to receive a response to a heartbeat request.
join-rate
The number of group joins per second. Group joining is the first phase of the rebalance protocol. A large value indicates that the consumer group is unstable and will likely be coupled with increased lag.
join-time-avg
The average time taken for a group rejoin. This value can get as high as the configured session timeout for the consumer, but should usually be lower.
join-time-max
The max time taken for a group rejoin. This value should not get much higher than the configured session timeout for the consumer.
last-heartbeat-seconds-ago
The number of seconds since the last controller heartbeat.
sync-rate
The number of group syncs per second. Group synchronization is the second and last phase of the rebalance protocol. Similar to join-rate, a large value indicates group instability.
sync-time-avg
The average time taken for a group sync.
sync-time-max
The max time taken for a group sync.

Global Request Metrics

MBean: kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)

incoming-byte-rate
The average number of incoming bytes received per second from all servers.
outgoing-byte-rate
The average number of outgoing bytes sent per second to all servers.
request-latency-avg
The average request latency in ms.
request-latency-max
The maximum request latency in ms.
request-rate
The average number of requests sent per second.
response-rate
The average number of responses received per second.

Global Connection Metrics

MBean: kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)

connection-count
The current number of active connections.
connection-creation-rate
New connections established per second in the window.
connection-close-rate
Connections closed per second in the window.
io-ratio
The fraction of time the I/O thread spent doing I/O.
io-time-ns-avg
The average length of time for I/O per select call in nanoseconds.
io-wait-ratio
The fraction of time the I/O thread spent waiting.
io-wait-time-ns-avg
The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.
select-rate
Number of times the I/O layer checked for new I/O to perform per second.

Per-Broker Metrics

MBean: kafka.consumer:type=consumer-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)

Besides the Global Request Metrics, the following metrics are also available per broker:

incoming-byte-rate
The average number of bytes received per second from the broker.
outgoing-byte-rate
The average number of bytes sent per second to the broker.
request-size-avg
The average size of all requests in the window for a broker.
request-size-max
The maximum size of any request sent in the window for a broker.
request-rate
The average number of requests sent per second to the broker.
response-rate
The average number of responses received per second from the broker.

Old Consumer Metrics

kafka.consumer:type=ConsumerFetcherManager,name=MaxLag,clientId=([-.\w]+)
Number of messages the consumer lags behind the producer by.
kafka.consumer:type=ConsumerFetcherManager,name=MinFetchRate,clientId=([-.\w]+)
The minimum rate at which the consumer sends fetch requests to the broker. If a consumer is dead, this value drops to roughly 0.
kafka.consumer:type=ConsumerTopicMetrics,name=MessagesPerSec,clientId=([-.\w]+)
The throughput in messages consumed per second.
kafka.consumer:type=ConsumerTopicMetrics,name=MessagesPerSec,clientId=([-.\w]+)
The throughput in bytes consumed per second.

The following metrics are available only on the high-level consumer:

kafka.consumer:type=ZookeeperConsumerConnector,name=KafkaCommitsPerSec,clientId=([-.\w]+)
The rate at which this consumer commits offsets to Kafka. This is only relevant if offsets.storage=kafka.
kafka.consumer:type=ZookeeperConsumerConnector,name=ZooKeeperCommitsPerSec,clientId=([-.\w]+)
The rate at which this consumer commits offsets to ZooKeeper. This is only relevant if offsets.storage=zookeeper. Monitor this value if your ZooKeeper cluster is under performing due to high write load.
kafka.consumer:type=ZookeeperConsumerConnector,name=RebalanceRateAndTime,clientId=([-.\w]+)
The rate and latency of the rebalance operation on this consumer.
kafka.consumer:type=ZookeeperConsumerConnector,name=OwnedPartitionsCount,clientId=([-.\w]+),groupId=([-.\w]+)
The number of partitions owned by this consumer.