6.8 Monitoring

Kafka uses Yammer Metrics for metrics reporting in the server. The Java clients use Kafka Metrics, a built-in metrics registry that minimizes transitive dependencies pulled into client applications. Both expose metrics via JMX and can be configured to report stats using pluggable stats reporters to hook up to your monitoring system.

All Kafka rate metrics have a corresponding cumulative count metric with suffix -total. For example, records-consumed-rate has a corresponding metric named records-consumed-total.

The easiest way to see the available metrics is to fire up jconsole and point it at a running kafka client or server; this will allow browsing all metrics with JMX.

Security Considerations for Remote Monitoring using JMX

Apache Kafka disables remote JMX by default. You can enable remote monitoring using JMX by setting the environment variable JMX_PORT for processes started using the CLI or standard Java system properties to enable remote JMX programmatically. You must enable security when enabling remote JMX in production scenarios to ensure that unauthorized users cannot monitor or control your broker or application as well as the platform on which these are running. Note that authentication is disabled for JMX by default in Kafka and security configs must be overridden for production deployments by setting the environment variable KAFKA_JMX_OPTS for processes started using the CLI or by setting appropriate Java system properties. See Monitoring and Management Using JMX Technology for details on securing JMX.

We do graphing and alerting on the following metrics:

DescriptionMbean nameNormal value
Message in ratekafka.server:type=BrokerTopicMetrics,name=MessagesInPerSec
Byte in rate from clientskafka.server:type=BrokerTopicMetrics,name=BytesInPerSec
Byte in rate from other brokerskafka.server:type=BrokerTopicMetrics,name=ReplicationBytesInPerSec
Request ratekafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower}
Error ratekafka.network:type=RequestMetrics,name=ErrorsPerSec,request=([-.\w]+),error=([-.\w]+)Number of errors in responses counted per-request-type, per-error-code. If a response contains multiple errors, all are counted. error=NONE indicates successful responses.
Request size in byteskafka.network:type=RequestMetrics,name=RequestBytes,request=([-.\w]+)Size of requests for each request type.
Temporary memory size in byteskafka.network:type=RequestMetrics,name=TemporaryMemoryBytes,request={Produce|Fetch}Temporary memory used for message format conversions and decompression.
Message conversion timekafka.network:type=RequestMetrics,name=MessageConversionsTimeMs,request={Produce|Fetch}Time in milliseconds spent on message format conversions.
Message conversion ratekafka.server:type=BrokerTopicMetrics,name={Produce|Fetch}MessageConversionsPerSec,topic=([-.\w]+)Number of records which required message format conversion.
Request Queue Sizekafka.network:type=RequestChannel,name=RequestQueueSizeSize of the request queue.
Byte out rate to clientskafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec
Byte out rate to other brokerskafka.server:type=BrokerTopicMetrics,name=ReplicationBytesOutPerSec
Message validation failure rate due to no key specified for compacted topickafka.server:type=BrokerTopicMetrics,name=NoKeyCompactedTopicRecordsPerSec
Message validation failure rate due to invalid magic numberkafka.server:type=BrokerTopicMetrics,name=InvalidMagicNumberRecordsPerSec
Message validation failure rate due to incorrect crc checksumkafka.server:type=BrokerTopicMetrics,name=InvalidMessageCrcRecordsPerSec
Message validation failure rate due to non-continuous offset or sequence number in batchkafka.server:type=BrokerTopicMetrics,name=InvalidOffsetOrSequenceRecordsPerSec
Log flush rate and timekafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs
# of under replicated partitions (the number of non-reassigning replicas - the number of ISR > 0)kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions0
# of under minIsr partitions (|ISR| < min.insync.replicas)kafka.server:type=ReplicaManager,name=UnderMinIsrPartitionCount0
# of at minIsr partitions (|ISR| = min.insync.replicas)kafka.server:type=ReplicaManager,name=AtMinIsrPartitionCount0
# of offline log directorieskafka.log:type=LogManager,name=OfflineLogDirectoryCount0
Is controller active on brokerkafka.controller:type=KafkaController,name=ActiveControllerCountonly one broker in the cluster should have 1
Leader election ratekafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMsnon-zero when there are broker failures
Unclean leader election ratekafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec0
Pending topic deleteskafka.controller:type=KafkaController,name=TopicsToDeleteCount
Pending replica deleteskafka.controller:type=KafkaController,name=ReplicasToDeleteCount
Ineligible pending topic deleteskafka.controller:type=KafkaController,name=TopicsIneligibleToDeleteCount
Ineligible pending replica deleteskafka.controller:type=KafkaController,name=ReplicasIneligibleToDeleteCount
Partition countskafka.server:type=ReplicaManager,name=PartitionCountmostly even across brokers
Leader replica countskafka.server:type=ReplicaManager,name=LeaderCountmostly even across brokers
ISR shrink ratekafka.server:type=ReplicaManager,name=IsrShrinksPerSecIf 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.
ISR expansion ratekafka.server:type=ReplicaManager,name=IsrExpandsPerSecSee above
Max lag in messages btw follower and leader replicaskafka.server:type=ReplicaFetcherManager,name=MaxLag,clientId=Replicalag should be proportional to the maximum batch size of a produce request.
Lag in messages per follower replicakafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)lag should be proportional to the maximum batch size of a produce request.
Requests waiting in the producer purgatorykafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Producenon-zero if ack=-1 is used
Requests waiting in the fetch purgatorykafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Fetchsize depends on fetch.wait.max.ms in the consumer
Request total timekafka.network:type=RequestMetrics,name=TotalTimeMs,request={Produce|FetchConsumer|FetchFollower}broken into queue, local, remote and response send time
Time the request waits in the request queuekafka.network:type=RequestMetrics,name=RequestQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request is processed at the leaderkafka.network:type=RequestMetrics,name=LocalTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request waits for the followerkafka.network:type=RequestMetrics,name=RemoteTimeMs,request={Produce|FetchConsumer|FetchFollower}non-zero for produce requests when ack=-1
Time the request waits in the response queuekafka.network:type=RequestMetrics,name=ResponseQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time to send the responsekafka.network:type=RequestMetrics,name=ResponseSendTimeMs,request={Produce|FetchConsumer|FetchFollower}
Number of messages the consumer lags behind the producer by. Published by the consumer, not broker.kafka.consumer:type=consumer-fetch-manager-metrics,client-id={client-id} Attribute: records-lag-max
The average fraction of time the network processors are idlekafka.network:type=SocketServer,name=NetworkProcessorAvgIdlePercentbetween 0 and 1, ideally > 0.3
The number of connections disconnected on a processor due to a client not re-authenticating and then using the connection beyond its expiration time for anything other than re-authenticationkafka.server:type=socket-server-metrics,listener=[SASL_PLAINTEXT|SASL_SSL],networkProcessor=<#>,name=expired-connections-killed-countideally 0 when re-authentication is enabled, implying there are no longer any older, pre-2.2.0 clients connecting to this (listener, processor) combination
The total number of connections disconnected, across all processors, due to a client not re-authenticating and then using the connection beyond its expiration time for anything other than re-authenticationkafka.network:type=SocketServer,name=ExpiredConnectionsKilledCountideally 0 when re-authentication is enabled, implying there are no longer any older, pre-2.2.0 clients connecting to this broker
The average fraction of time the request handler threads are idlekafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercentbetween 0 and 1, ideally > 0.3
Bandwidth quota metrics per (user, client-id), user or client-idkafka.server:type={Produce|Fetch},user=([-.\w]+),client-id=([-.\w]+)Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0. byte-rate indicates the data produce/consume rate of the client in bytes/sec. For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified.
Request quota metrics per (user, client-id), user or client-idkafka.server:type=Request,user=([-.\w]+),client-id=([-.\w]+)Two attributes. throttle-time indicates the amount of time in ms the client was throttled. Ideally = 0. request-time indicates the percentage of time spent in broker network and I/O threads to process requests from client group. For (user, client-id) quotas, both user and client-id are specified. If per-client-id quota is applied to the client, user is not specified. If per-user quota is applied, client-id is not specified.
Requests exempt from throttlingkafka.server:type=Requestexempt-throttle-time indicates the percentage of time spent in broker network and I/O threads to process requests that are exempt from throttling.
ZooKeeper client request latencykafka.server:type=ZooKeeperClientMetrics,name=ZooKeeperRequestLatencyMsLatency in millseconds for ZooKeeper requests from broker.
ZooKeeper connection statuskafka.server:type=SessionExpireListener,name=SessionStateConnection status of broker’s ZooKeeper session which may be one of Disconnected|SyncConnected|AuthFailed|ConnectedReadOnly|SaslAuthenticated|Expired.
Max time to load group metadatakafka.server:type=group-coordinator-metrics,name=partition-load-time-maxmaximum time, in milliseconds, it took to load offsets and group metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)
Avg time to load group metadatakafka.server:type=group-coordinator-metrics,name=partition-load-time-avgaverage time, in milliseconds, it took to load offsets and group metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)
Max time to load transaction metadatakafka.server:type=transaction-coordinator-metrics,name=partition-load-time-maxmaximum time, in milliseconds, it took to load transaction metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)
Avg time to load transaction metadatakafka.server:type=transaction-coordinator-metrics,name=partition-load-time-avgaverage time, in milliseconds, it took to load transaction metadata from the consumer offset partitions loaded in the last 30 seconds (including time spent waiting for the loading task to be scheduled)
Consumer Group Offset Countkafka.server:type=GroupMetadataManager,name=NumOffsetsTotal number of committed offsets for Consumer Groups
Consumer Group Countkafka.server:type=GroupMetadataManager,name=NumGroupsTotal number of Consumer Groups
Consumer Group Count, per Statekafka.server:type=GroupMetadataManager,name=NumGroups[PreparingRebalance,CompletingRebalance,Empty,Stable,Dead]The number of Consumer Groups in each state: PreparingRebalance, CompletingRebalance, Empty, Stable, Dead
Number of reassigning partitionskafka.server:type=ReplicaManager,name=ReassigningPartitionsThe number of reassigning leader partitions on a broker.
Outgoing byte rate of reassignment traffickafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesOutPerSec
Incoming byte rate of reassignment traffickafka.server:type=BrokerTopicMetrics,name=ReassignmentBytesInPerSec

Common monitoring metrics for producer/consumer/connect/streams

The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.

Metric/Attribute nameDescriptionMbean name
connection-close-rateConnections closed per second in the window.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
connection-close-totalTotal connections closed in the window.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
connection-creation-rateNew connections established per second in the window.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
connection-creation-totalTotal new connections established in the window.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
network-io-rateThe average number of network operations (reads or writes) on all connections per second.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
network-io-totalThe total number of network operations (reads or writes) on all connections.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
outgoing-byte-rateThe average number of outgoing bytes sent per second to all servers.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
outgoing-byte-totalThe total number of outgoing bytes sent to all servers.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
request-rateThe average number of requests sent per second.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
request-totalThe total number of requests sent.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
request-size-avgThe average size of all requests in the window.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
request-size-maxThe maximum size of any request sent in the window.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
incoming-byte-rateBytes/second read off all sockets.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
incoming-byte-totalTotal bytes read off all sockets.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
response-rateResponses received per second.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
response-totalTotal responses received.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
select-rateNumber of times the I/O layer checked for new I/O to perform per second.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
select-totalTotal number of times the I/O layer checked for new I/O to perform.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
io-wait-time-ns-avgThe average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
io-wait-ratioThe fraction of time the I/O thread spent waiting.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
io-time-ns-avgThe average length of time for I/O per select call in nanoseconds.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
io-ratioThe fraction of time the I/O thread spent doing I/O.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
connection-countThe current number of active connections.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
successful-authentication-rateConnections per second that were successfully authenticated using SASL or SSL.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
successful-authentication-totalTotal connections that were successfully authenticated using SASL or SSL.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
failed-authentication-rateConnections per second that failed authentication.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
failed-authentication-totalTotal connections that failed authentication.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
successful-reauthentication-rateConnections per second that were successfully re-authenticated using SASL.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
successful-reauthentication-totalTotal connections that were successfully re-authenticated using SASL.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
reauthentication-latency-maxThe maximum latency in ms observed due to re-authentication.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
reauthentication-latency-avgThe average latency in ms observed due to re-authentication.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
failed-reauthentication-rateConnections per second that failed re-authentication.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
failed-reauthentication-totalTotal connections that failed re-authentication.kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
successful-authentication-no-reauth-totalTotal connections that were successfully authenticated by older, pre-2.2.0 SASL clients that do not support re-authentication. May only be non-zerokafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)

Common Per-broker metrics for producer/consumer/connect/streams

The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.

Metric/Attribute nameDescriptionMbean name
outgoing-byte-rateThe average number of outgoing bytes sent per second for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
outgoing-byte-totalThe total number of outgoing bytes sent for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
request-rateThe average number of requests sent per second for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
request-totalThe total number of requests sent for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
request-size-avgThe average size of all requests in the window for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
request-size-maxThe maximum size of any request sent in the window for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
incoming-byte-rateThe average number of bytes received per second for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
incoming-byte-totalThe total number of bytes received for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
request-latency-avgThe average request latency in ms for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
request-latency-maxThe maximum request latency in ms for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
response-rateResponses received per second for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)
response-totalTotal responses received for a node.kafka.[producer|consumer|connect]:type=[consumer|producer|connect]-node-metrics,client-id=([-.\w]+),node-id=([0-9]+)

Producer monitoring

The following metrics are available on producer instances.

Metric/Attribute nameDescriptionMbean name
waiting-threadsThe number of user threads blocked waiting for buffer memory to enqueue their records.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
buffer-total-bytesThe maximum amount of buffer memory the client can use (whether or not it is currently used).kafka.producer:type=producer-metrics,client-id=([-.\w]+)
buffer-available-bytesThe total amount of buffer memory that is not being used (either unallocated or in the free list).kafka.producer:type=producer-metrics,client-id=([-.\w]+)
bufferpool-wait-timeThe fraction of time an appender waits for space allocation.kafka.producer:type=producer-metrics,client-id=([-.\w]+)
Producer Sender Metrics
kafka.producer:type=producer-metrics,client-id=”{client-id}”
Attribute nameDescription
batch-size-avgThe average number of bytes sent per partition per-request.
batch-size-maxThe max number of bytes sent per partition per-request.
batch-split-rateThe average number of batch splits per second
batch-split-totalThe total number of batch splits
compression-rate-avgThe average compression rate of record batches, defined as the average ratio of the compressed batch size over the uncompressed size.
metadata-ageThe age in seconds of the current producer metadata being used.
produce-throttle-time-avgThe average time in ms a request was throttled by a broker
produce-throttle-time-maxThe maximum time in ms a request was throttled by a broker
record-error-rateThe average per-second number of record sends that resulted in errors
record-error-totalThe total number of record sends that resulted in errors
record-queue-time-avgThe average time in ms record batches spent in the send buffer.
record-queue-time-maxThe maximum time in ms record batches spent in the send buffer.
record-retry-rateThe average per-second number of retried record sends
record-retry-totalThe total number of retried record sends
record-send-rateThe average number of records sent per second.
record-send-totalThe total number of records sent.
record-size-avgThe average record size
record-size-maxThe maximum record size
records-per-request-avgThe average number of records per request.
request-latency-avgThe average request latency in ms
request-latency-maxThe maximum request latency in ms
requests-in-flightThe current number of in-flight requests awaiting a response.
kafka.producer:type=producer-topic-metrics,client-id=”{client-id}”,topic=”{topic}”
Attribute nameDescription
byte-rateThe average number of bytes sent per second for a topic.
byte-totalThe total number of bytes sent for a topic.
compression-rateThe average compression rate of record batches for a topic, defined as the average ratio of the compressed batch size over the uncompressed size.
record-error-rateThe average per-second number of record sends that resulted in errors for a topic
record-error-totalThe total number of record sends that resulted in errors for a topic
record-retry-rateThe average per-second number of retried record sends for a topic
record-retry-totalThe total number of retried record sends for a topic
record-send-rateThe average number of records sent per second for a topic.
record-send-totalThe total number of records sent for a topic.

consumer monitoring

The following metrics are available on consumer instances.

Metric/Attribute nameDescriptionMbean name
time-between-poll-avgThe average delay between invocations of poll().kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)
time-between-poll-maxThe max delay between invocations of poll().kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)
last-poll-seconds-agoThe number of seconds since the last poll() invocation.kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)
poll-idle-ratio-avgThe average fraction of time the consumer’s poll() is idle as opposed to waiting for the user code to process records.kafka.consumer:type=consumer-metrics,client-id=([-.\w]+)
Consumer Group Metrics
Metric/Attribute nameDescriptionMbean name
commit-latency-avgThe average time taken for a commit requestkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
commit-latency-maxThe max time taken for a commit requestkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
commit-rateThe number of commit calls per secondkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
commit-totalThe total number of commit callskafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
assigned-partitionsThe number of partitions currently assigned to this consumerkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
heartbeat-response-time-maxThe max time taken to receive a response to a heartbeat requestkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
heartbeat-rateThe average number of heartbeats per secondkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
heartbeat-totalThe total number of heartbeatskafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
join-time-avgThe average time taken for a group rejoinkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
join-time-maxThe max time taken for a group rejoinkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
join-rateThe number of group joins per secondkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
join-totalThe total number of group joinskafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
sync-time-avgThe average time taken for a group synckafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
sync-time-maxThe max time taken for a group synckafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
sync-rateThe number of group syncs per secondkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
sync-totalThe total number of group syncskafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
rebalance-latency-avgThe average time taken for a group rebalancekafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
rebalance-latency-maxThe max time taken for a group rebalancekafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
rebalance-latency-totalThe total time taken for group rebalances so farkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
rebalance-totalThe total number of group rebalances participatedkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
rebalance-rate-per-hourThe number of group rebalance participated per hourkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
failed-rebalance-totalThe total number of failed group rebalanceskafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
failed-rebalance-rate-per-hourThe number of failed group rebalance event per hourkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
last-rebalance-seconds-agoThe number of seconds since the last rebalance eventkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
last-heartbeat-seconds-agoThe number of seconds since the last controller heartbeatkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
partitions-revoked-latency-avgThe average time taken by the on-partitions-revoked rebalance listener callbackkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
partitions-revoked-latency-maxThe max time taken by the on-partitions-revoked rebalance listener callbackkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
partitions-assigned-latency-avgThe average time taken by the on-partitions-assigned rebalance listener callbackkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
partitions-assigned-latency-maxThe max time taken by the on-partitions-assigned rebalance listener callbackkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
partitions-lost-latency-avgThe average time taken by the on-partitions-lost rebalance listener callbackkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
partitions-lost-latency-maxThe max time taken by the on-partitions-lost rebalance listener callbackkafka.consumer:type=consumer-coordinator-metrics,client-id=([-.\w]+)
Consumer Fetch Metrics
kafka.consumer:type=consumer-fetch-manager-metrics,client-id=”{client-id}”
Attribute nameDescription
bytes-consumed-rateThe average number of bytes consumed per second
bytes-consumed-totalThe total number of bytes consumed
fetch-latency-avgThe average time taken for a fetch request.
fetch-latency-maxThe max time taken for any fetch request.
fetch-rateThe number of fetch requests per second.
fetch-size-avgThe average number of bytes fetched per request
fetch-size-maxThe maximum number of bytes fetched per request
fetch-throttle-time-avgThe average throttle time in ms
fetch-throttle-time-maxThe maximum throttle time in ms
fetch-totalThe total number of fetch requests.
records-consumed-rateThe average number of records consumed per second
records-consumed-totalThe total number of records consumed
records-lag-maxThe maximum lag in terms of number of records for any partition in this window
records-lead-minThe minimum lead in terms of number of records for any partition in this window
records-per-request-avgThe average number of records in each request
kafka.consumer:type=consumer-fetch-manager-metrics,client-id=”{client-id}”,topic=”{topic}”
Attribute nameDescription
bytes-consumed-rateThe average number of bytes consumed per second for a topic
bytes-consumed-totalThe total number of bytes consumed for a topic
fetch-size-avgThe average number of bytes fetched per request for a topic
fetch-size-maxThe maximum number of bytes fetched per request for a topic
records-consumed-rateThe average number of records consumed per second for a topic
records-consumed-totalThe total number of records consumed for a topic
records-per-request-avgThe average number of records in each request for a topic
kafka.consumer:type=consumer-fetch-manager-metrics,partition=”{partition}”,topic=”{topic}”,client-id=”{client-id}”
Attribute nameDescription
preferred-read-replicaThe current read replica for the partition, or -1 if reading from leader
records-lagThe latest lag of the partition
records-lag-avgThe average lag of the partition
records-lag-maxThe max lag of the partition
records-leadThe latest lead of the partition
records-lead-avgThe average lead of the partition
records-lead-minThe min lead of the partition

Connect Monitoring

A Connect worker process contains all the producer and consumer metrics as well as metrics specific to Connect. The worker process itself has a number of metrics, while each connector and task have additional metrics. [2020-12-04 10:03:05,630] INFO Metrics scheduler closed (org.apache.kafka.common.metrics.Metrics:668) [2020-12-04 10:03:05,632] INFO Metrics reporters closed (org.apache.kafka.common.metrics.Metrics:678)

kafka.connect:type=connect-worker-metrics
Attribute nameDescription
connector-countThe number of connectors run in this worker.
connector-startup-attempts-totalThe total number of connector startups that this worker has attempted.
connector-startup-failure-percentageThe average percentage of this worker’s connectors starts that failed.
connector-startup-failure-totalThe total number of connector starts that failed.
connector-startup-success-percentageThe average percentage of this worker’s connectors starts that succeeded.
connector-startup-success-totalThe total number of connector starts that succeeded.
task-countThe number of tasks run in this worker.
task-startup-attempts-totalThe total number of task startups that this worker has attempted.
task-startup-failure-percentageThe average percentage of this worker’s tasks starts that failed.
task-startup-failure-totalThe total number of task starts that failed.
task-startup-success-percentageThe average percentage of this worker’s tasks starts that succeeded.
task-startup-success-totalThe total number of task starts that succeeded.
kafka.connect:type=connect-worker-metrics,connector=”{connector}”
Attribute nameDescription
connector-destroyed-task-countThe number of destroyed tasks of the connector on the worker.
connector-failed-task-countThe number of failed tasks of the connector on the worker.
connector-paused-task-countThe number of paused tasks of the connector on the worker.
connector-running-task-countThe number of running tasks of the connector on the worker.
connector-total-task-countThe number of tasks of the connector on the worker.
connector-unassigned-task-countThe number of unassigned tasks of the connector on the worker.
kafka.connect:type=connect-worker-rebalance-metrics
Attribute nameDescription
completed-rebalances-totalThe total number of rebalances completed by this worker.
connect-protocolThe Connect protocol used by this cluster
epochThe epoch or generation number of this worker.
leader-nameThe name of the group leader.
rebalance-avg-time-msThe average time in milliseconds spent by this worker to rebalance.
rebalance-max-time-msThe maximum time in milliseconds spent by this worker to rebalance.
rebalancingWhether this worker is currently rebalancing.
time-since-last-rebalance-msThe time in milliseconds since this worker completed the most recent rebalance.
kafka.connect:type=connector-metrics,connector=”{connector}”
Attribute nameDescription
connector-classThe name of the connector class.
connector-typeThe type of the connector. One of ‘source’ or ‘sink’.
connector-versionThe version of the connector class, as reported by the connector.
statusThe status of the connector. One of ‘unassigned’, ‘running’, ‘paused’, ‘failed’, or ‘destroyed’.
kafka.connect:type=connector-task-metrics,connector=”{connector}”,task=”{task}”
Attribute nameDescription
batch-size-avgThe average size of the batches processed by the connector.
batch-size-maxThe maximum size of the batches processed by the connector.
offset-commit-avg-time-msThe average time in milliseconds taken by this task to commit offsets.
offset-commit-failure-percentageThe average percentage of this task’s offset commit attempts that failed.
offset-commit-max-time-msThe maximum time in milliseconds taken by this task to commit offsets.
offset-commit-success-percentageThe average percentage of this task’s offset commit attempts that succeeded.
pause-ratioThe fraction of time this task has spent in the pause state.
running-ratioThe fraction of time this task has spent in the running state.
statusThe status of the connector task. One of ‘unassigned’, ‘running’, ‘paused’, ‘failed’, or ‘destroyed’.
kafka.connect:type=sink-task-metrics,connector=”{connector}”,task=”{task}”
Attribute nameDescription
offset-commit-completion-rateThe average per-second number of offset commit completions that were completed successfully.
offset-commit-completion-totalThe total number of offset commit completions that were completed successfully.
offset-commit-seq-noThe current sequence number for offset commits.
offset-commit-skip-rateThe average per-second number of offset commit completions that were received too late and skipped/ignored.
offset-commit-skip-totalThe total number of offset commit completions that were received too late and skipped/ignored.
partition-countThe number of topic partitions assigned to this task belonging to the named sink connector in this worker.
put-batch-avg-time-msThe average time taken by this task to put a batch of sinks records.
put-batch-max-time-msThe maximum time taken by this task to put a batch of sinks records.
sink-record-active-countThe number of records that have been read from Kafka but not yet completely committed/flushed/acknowledged by the sink task.
sink-record-active-count-avgThe average number of records that have been read from Kafka but not yet completely committed/flushed/acknowledged by the sink task.
sink-record-active-count-maxThe maximum number of records that have been read from Kafka but not yet completely committed/flushed/acknowledged by the sink task.
sink-record-lag-maxThe maximum lag in terms of number of records that the sink task is behind the consumer’s position for any topic partitions.
sink-record-read-rateThe average per-second number of records read from Kafka for this task belonging to the named sink connector in this worker. This is before transformations are applied.
sink-record-read-totalThe total number of records read from Kafka by this task belonging to the named sink connector in this worker, since the task was last restarted.
sink-record-send-rateThe average per-second number of records output from the transformations and sent/put to this task belonging to the named sink connector in this worker. This is after transformations are applied and excludes any records filtered out by the transformations.
sink-record-send-totalThe total number of records output from the transformations and sent/put to this task belonging to the named sink connector in this worker, since the task was last restarted.
kafka.connect:type=source-task-metrics,connector=”{connector}”,task=”{task}”
Attribute nameDescription
poll-batch-avg-time-msThe average time in milliseconds taken by this task to poll for a batch of source records.
poll-batch-max-time-msThe maximum time in milliseconds taken by this task to poll for a batch of source records.
source-record-active-countThe number of records that have been produced by this task but not yet completely written to Kafka.
source-record-active-count-avgThe average number of records that have been produced by this task but not yet completely written to Kafka.
source-record-active-count-maxThe maximum number of records that have been produced by this task but not yet completely written to Kafka.
source-record-poll-rateThe average per-second number of records produced/polled (before transformation) by this task belonging to the named source connector in this worker.
source-record-poll-totalThe total number of records produced/polled (before transformation) by this task belonging to the named source connector in this worker.
source-record-write-rateThe average per-second number of records output from the transformations and written to Kafka for this task belonging to the named source connector in this worker. This is after transformations are applied and excludes any records filtered out by the transformations.
source-record-write-totalThe number of records output from the transformations and written to Kafka for this task belonging to the named source connector in this worker, since the task was last restarted.
kafka.connect:type=task-error-metrics,connector=”{connector}”,task=”{task}”
Attribute nameDescription
deadletterqueue-produce-failuresThe number of failed writes to the dead letter queue.
deadletterqueue-produce-requestsThe number of attempted writes to the dead letter queue.
last-error-timestampThe epoch timestamp when this task last encountered an error.
total-errors-loggedThe number of errors that were logged.
total-record-errorsThe number of record processing errors in this task.
total-record-failuresThe number of record processing failures in this task.
total-records-skippedThe number of records skipped due to errors.
total-retriesThe number of operations retried.

Streams Monitoring

A Kafka Streams instance contains all the producer and consumer metrics as well as additional metrics specific to Streams. By default Kafka Streams has metrics with two recording levels: debug and info.

Note that the metrics have a 4-layer hierarchy. At the top level there are client-level metrics for each started Kafka Streams client. Each client has stream threads, with their own metrics. Each stream thread has tasks, with their own metrics. Each task has a number of processor nodes, with their own metrics. Each task also has a number of state stores and record caches, all with their own metrics.

Use the following configuration option to specify which metrics you want collected:

  1. metrics.recording.level="info"
Client Metrics

All of the following metrics have a recording level of info:

Metric/Attribute nameDescriptionMbean name
versionThe version of the Kafka Streams client.kafka.streams:type=stream-metrics,client-id=([-.\w]+)
commit-idThe version control commit ID of the Kafka Streams client.kafka.streams:type=stream-metrics,client-id=([-.\w]+)
application-idThe application ID of the Kafka Streams client.kafka.streams:type=stream-metrics,client-id=([-.\w]+)
topology-descriptionThe description of the topology executed in the Kafka Streams client.kafka.streams:type=stream-metrics,client-id=([-.\w]+)
stateThe state of the Kafka Streams client.kafka.streams:type=stream-metrics,client-id=([-.\w]+)
Thread Metrics

All of the following metrics have a recording level of info:

Metric/Attribute nameDescriptionMbean name
commit-latency-avgThe average execution time in ms, for committing, across all running tasks of this thread.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
commit-latency-maxThe maximum execution time in ms, for committing, across all running tasks of this thread.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
poll-latency-avgThe average execution time in ms, for consumer polling.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
poll-latency-maxThe maximum execution time in ms, for consumer polling.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
process-latency-avgThe average execution time in ms, for processing.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
process-latency-maxThe maximum execution time in ms, for processing.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
punctuate-latency-avgThe average execution time in ms, for punctuating.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
punctuate-latency-maxThe maximum execution time in ms, for punctuating.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
commit-rateThe average number of commits per second.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
commit-totalThe total number of commit calls.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
poll-rateThe average number of consumer poll calls per second.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
poll-totalThe total number of consumer poll calls.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
process-rateThe average number of processed records per second.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
process-totalThe total number of processed records.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
punctuate-rateThe average number of punctuate calls per second.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
punctuate-totalThe total number of punctuate calls.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
task-created-rateThe average number of tasks created per second.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
task-created-totalThe total number of tasks created.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
task-closed-rateThe average number of tasks closed per second.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
task-closed-totalThe total number of tasks closed.kafka.streams:type=stream-thread-metrics,thread-id=([-.\w]+)
Task Metrics

All of the following metrics have a recording level of debug, except for metrics dropped-records-rate and dropped-records-total which have a recording level of info:

Metric/Attribute nameDescriptionMbean name
process-latency-avgThe average execution time in ns, for processing.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
process-latency-maxThe maximum execution time in ns, for processing.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
process-rateThe average number of processed records per second across all source processor nodes of this task.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
process-totalThe total number of processed records across all source processor nodes of this task.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
commit-latency-avgThe average execution time in ns, for committing.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
commit-latency-maxThe maximum execution time in ns, for committing.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
commit-rateThe average number of commit calls per second.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
commit-totalThe total number of commit calls.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
record-lateness-avgThe average observed lateness of records (stream time - record timestamp).kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
record-lateness-maxThe max observed lateness of records (stream time - record timestamp).kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
enforced-processing-rateThe average number of enforced processings per second.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
enforced-processing-totalThe total number enforced processings.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
dropped-records-rateThe average number of records dropped within this task.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
dropped-records-totalThe total number of records dropped within this task.kafka.streams:type=stream-task-metrics,thread-id=([-.\w]+),task-id=([-.\w]+)
Processor Node Metrics

The following metrics are only available on certain types of nodes, i.e., process-rate and process-total are only available for source processor nodes and suppression-emit-rate and suppression-emit-total are only available for suppression operation nodes. All of the metrics have a recording level of debug:

Metric/Attribute nameDescriptionMbean name
process-rateThe average number of records processed by a source processor node per second.kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
process-totalThe total number of records processed by a source processor node per second.kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
suppression-emit-rateThe rate at which records that have been emitted downstream from suppression operation nodes.kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
suppression-emit-totalThe total number of records that have been emitted downstream from suppression operation nodes.kafka.streams:type=stream-processor-node-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),processor-node-id=([-.\w]+)
State Store Metrics

All of the following metrics have a recording level of debug. Note that the store-scope value is specified in StoreSupplier#metricsScope() for user’s customized state stores; for built-in state stores, currently we have:

  • in-memory-state
  • in-memory-lru-state
  • in-memory-window-state
  • in-memory-suppression (for suppression buffers)
  • rocksdb-state (for RocksDB backed key-value store)
  • rocksdb-window-state (for RocksDB backed window store)
  • rocksdb-session-state (for RocksDB backed session store)

Metrics suppression-buffer-size-avg, suppression-buffer-size-max, suppression-buffer-count-avg, and suppression-buffer-count-max are only available for suppression buffers. All other metrics are not available for suppression buffers.

Metric/Attribute nameDescriptionMbean name
put-latency-avgThe average put execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-latency-maxThe maximum put execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-if-absent-latency-avgThe average put-if-absent execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-if-absent-latency-maxThe maximum put-if-absent execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
get-latency-avgThe average get execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
get-latency-maxThe maximum get execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
delete-latency-avgThe average delete execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
delete-latency-maxThe maximum delete execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-all-latency-avgThe average put-all execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-all-latency-maxThe maximum put-all execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
all-latency-avgThe average all operation execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
all-latency-maxThe maximum all operation execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
range-latency-avgThe average range execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
range-latency-maxThe maximum range execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
flush-latency-avgThe average flush execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
flush-latency-maxThe maximum flush execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
restore-latency-avgThe average restore execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
restore-latency-maxThe maximum restore execution time in ns.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-rateThe average put rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-if-absent-rateThe average put-if-absent rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
get-rateThe average get rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
delete-rateThe average delete rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
put-all-rateThe average put-all rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
all-rateThe average all operation rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
range-rateThe average range rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
flush-rateThe average flush rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
restore-rateThe average restore rate for this store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
suppression-buffer-size-avgThe average total size, in bytes, of the buffered data over the sampling window.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)
suppression-buffer-size-maxThe maximum total size, in bytes, of the buffered data over the sampling window.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)
suppression-buffer-count-avgThe average number of records buffered over the sampling window.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)
suppression-buffer-count-maxThe maximum number of records buffered over the sampling window.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),in-memory-suppression-id=([-.\w]+)
RocksDB Metrics

RocksDB metrics are grouped into statistics-based metrics and properties-based metrics. The former are recorded from statistics that a RocksDB state store collects whereas the latter are recorded from properties that RocksDB exposes. Statistics collected by RocksDB provide cumulative measurements over time, e.g. bytes written to the state store. Properties exposed by RocksDB provide current measurements, e.g., the amount of memory currently used. Note that the store-scope for built-in RocksDB state stores are currently the following:

  • rocksdb-state (for RocksDB backed key-value store)
  • rocksdb-window-state (for RocksDB backed window store)
  • rocksdb-session-state (for RocksDB backed session store)

RocksDB Statistics-based Metrics: All of the following statistics-based metrics have a recording level of debug because collecting statistics in RocksDB may have an impact on performance. Statistics-based metrics are collected every minute from the RocksDB state stores. If a state store consists of multiple RocksDB instances, as is the case for aggregations over time and session windows, each metric reports an aggregation over the RocksDB instances of the state store.

Metric/Attribute nameDescriptionMbean name
bytes-written-rateThe average number of bytes written per second to the RocksDB state store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
bytes-written-totalThe total number of bytes written to the RocksDB state store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
bytes-read-rateThe average number of bytes read per second from the RocksDB state store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
bytes-read-totalThe total number of bytes read from the RocksDB state store.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
memtable-bytes-flushed-rateThe average number of bytes flushed per second from the memtable to disk.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
memtable-bytes-flushed-totalThe total number of bytes flushed from the memtable to disk.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
memtable-hit-ratioThe ratio of memtable hits relative to all lookups to the memtable.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
block-cache-data-hit-ratioThe ratio of block cache hits for data blocks relative to all lookups for data blocks to the block cache.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
block-cache-index-hit-ratioThe ratio of block cache hits for index blocks relative to all lookups for index blocks to the block cache.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
block-cache-filter-hit-ratioThe ratio of block cache hits for filter blocks relative to all lookups for filter blocks to the block cache.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
write-stall-duration-avgThe average duration of write stalls in ms.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
write-stall-duration-totalThe total duration of write stalls in ms.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
bytes-read-compaction-rateThe average number of bytes read per second during compaction.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
bytes-written-compaction-rateThe average number of bytes written per second during compaction.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
number-open-filesThe number of current open files.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
number-file-errors-totalThe total number of file errors occurred.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)

RocksDB Properties-based Metrics: All of the following properties-based metrics have a recording level of info and are recorded when the metrics are accessed. If a state store consists of multiple RocksDB instances, as is the case for aggregations over time and session windows, each metric reports the sum over all the RocksDB instances of the state store, except for the block cache metrics block-cache-*. The block cache metrics report the sum over all RocksDB instances if each instance uses its own block cache, and they report the recorded value from only one instance if a single block cache is shared among all instances.

Metric/Attribute nameDescriptionMbean name
num-immutable-mem-tableThe number of immutable memtables that have not yet been flushed.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
cur-size-active-mem-tableThe approximate size of the active memtable in bytes.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
cur-size-all-mem-tablesThe approximate size of active and unflushed immutable memtables in bytes.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
size-all-mem-tablesThe approximate size of active, unflushed immutable, and pinned immutable memtables in bytes.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
num-entries-active-mem-tableThe number of entries in the active memtable.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
num-entries-imm-mem-tablesThe number of entries in the unflushed immutable memtables.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
num-deletes-active-mem-tableThe number of delete entries in the active memtable.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
num-deletes-imm-mem-tablesThe number of delete entries in the unflushed immutable memtables.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
mem-table-flush-pendingThis metric reports 1 if a memtable flush is pending, otherwise it reports 0.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
num-running-flushesThe number of currently running flushes.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
compaction-pendingThis metric reports 1 if at least one compaction is pending, otherwise it reports 0.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
num-running-compactionsThe number of currently running compactions.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
estimate-pending-compaction-bytesThe estimated total number of bytes a compaction needs to rewrite on disk to get all levels down to under target size (only valid for level compaction).kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
total-sst-files-sizeThe total size in bytes of all SST files.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
live-sst-files-sizeThe total size in bytes of all SST files that belong to the latest LSM tree.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
num-live-versionsNumber of live versions of the LSM tree.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
block-cache-capacityThe capacity of the block cache in bytes.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
block-cache-usageThe memory size of the entries residing in block cache in bytes.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
block-cache-pinned-usageThe memory size for the entries being pinned in the block cache in bytes.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
estimate-num-keysThe estimated number of keys in the active and unflushed immutable memtables and storage.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
estimate-table-readers-memThe estimated memory in bytes used for reading SST tables, excluding memory used in block cache.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
background-errorsThe total number of background errors.kafka.streams:type=stream-state-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),[store-scope]-id=([-.\w]+)
Record Cache Metrics

All of the following metrics have a recording level of debug:

Metric/Attribute nameDescriptionMbean name
hit-ratio-avgThe average cache hit ratio defined as the ratio of cache read hits over the total cache read requests.kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+)
hit-ratio-minThe mininum cache hit ratio.kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+)
hit-ratio-maxThe maximum cache hit ratio.kafka.streams:type=stream-record-cache-metrics,thread-id=([-.\w]+),task-id=([-.\w]+),record-cache-id=([-.\w]+)

Others

We recommend monitoring GC time and other stats and various server stats such as CPU utilization, I/O service time, etc. On the client side, we recommend monitoring the message/byte rate (global and per topic), request rate/size/time, and on the consumer side, max lag in messages among all partitions and min fetch request rate. For a consumer to keep up, max lag needs to be less than a threshold and min fetch rate needs to be larger than 0.