diff --git a/docs/ops.html b/docs/ops.html index b1dd7d615a9f3..5be6a69b75499 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3980,6 +3980,348 @@
Share Group Monitoring
+ The following set of metrics are available for monitoring the share group: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Metric/Attribute nameMbean nameDescription
TotalShareFetchRequestsPerSeckafka.server:type=BrokerTopicMetrics,name=TotalShareFetchRequestsPerSec,topic=([-.\w]+)The fetch request rate per second.
FailedShareFetchRequestsPerSeckafka.server:type=BrokerTopicMetrics,name=FailedShareFetchRequestsPerSec,topic=([-.\w]+)The share fetch request rate for requests that failed.
TotalShareAcknowledgementRequestsPerSeckafka.server:type=BrokerTopicMetrics,name=TotalShareAcknowledgementRequestsPerSec,topic=([-.\w]+)The acknowledgement request rate per second.
FailedShareAcknowledgementRequestsPerSeckafka.server:type=BrokerTopicMetrics,name=FailedShareAcknowledgementRequestsPerSec,topic=([-.\w]+)The share acknowledgement request rate for requests that failed.
RecordAcknowledgementsPerSeckafka.server:type=ShareGroupMetrics,name=RecordAcknowledgementsPerSec,ackType={Accept|Release|Reject|Renew}The rate per second of records acknowledged per acknowledgement type.
PartitionLoadTimeMskafka.server:type=ShareGroupMetrics,name=PartitionLoadTimeMsThe time taken to load the share partitions.
RequestTopicPartitionsFetchRatiokafka.server:type=ShareGroupMetrics,name=RequestTopicPartitionsFetchRatio,group=([-.\w]+)The ratio of topic-partitions acquired to the total number of topic-partitions in share fetch request.
TopicPartitionsAcquireTimeMskafka.server:type=ShareGroupMetrics,name=TopicPartitionsAcquireTimeMs,group=([-.\w]+)The time elapsed (in millisecond) to acquire any topic partition for fetch.
AcquisitionLockTimeoutPerSeckafka.server:type=SharePartitionMetrics,name=AcquisitionLockTimeoutPerSec,group=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)The rate of acquisition locks for records which are not acknowledged within the timeout.
InFlightMessageCountkafka.server:type=SharePartitionMetrics,name=InFlightMessageCount,group=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)The number of in-flight messages for the share partition.
InFlightBatchCountkafka.server:type=SharePartitionMetrics,name=InFlightBatchCount,group=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)The number of in-flight batches for the share partition.
InFlightBatchMessageCountkafka.server:type=SharePartitionMetrics,name=InFlightBatchMessageCount,group=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)The number of messages in the in-flight batch.
FetchLockTimeMskafka.server:type=SharePartitionMetrics,name=FetchLockTimeMs,group=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)The time elapsed (in milliseconds) while a share partition is held under lock for fetching messages.
FetchLockRatiokafka.server:type=SharePartitionMetrics,name=FetchLockRatio,group=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)The fraction of time that share partition is held under lock.
ShareSessionEvictionsPerSeckafka.server:type=ShareSessionCache,name=ShareSessionEvictionsPerSecThe share session eviction rate per second.
SharePartitionsCountkafka.server:type=ShareSessionCache,name=SharePartitionsCountThe number of cached share partitions.
ShareSessionsCountkafka.server:type=ShareSessionCache,name=ShareSessionsCountThe number of cached share sessions.
NumDelayedOperations (ShareFetch)kafka.server:type=DelayedOperationPurgatory,name=NumDelayedOperations,delayedOperation=ShareFetchThe number of delayed operations for share fetch purgatory.
PurgatorySize (ShareFetch)kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=ShareFetchThe number of requests waiting in the share fetch purgatory. This is high if share consumers use a large value for fetch.wait.max.ms.
ExpiresPerSeckafka.server:type=DelayedShareFetchMetrics,name=ExpiresPerSecThe expired delayed share fetch operation rate per second.
+ +
Coordinator Metrics
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Metric/Attribute nameMbean nameDescription
group-countkafka.server:type=group-coordinator-metrics,name=group-count,protocol=shareThe total number of share groups managed by group coordinator.
share-group-rebalance-ratekafka.server:type=group-coordinator-metrics,name=share-group-rebalance-rateThe total number of share group rebalances.
share-group-rebalance-countkafka.server:type=group-coordinator-metrics,name=share-group-rebalance-countThe total number of share group rebalances.
group-countkafka.server:type=group-coordinator-metrics,name=group-count,protocol=shareThe total number of share groups managed by group coordinator.
partition-load-time-maxkafka.server:type=share-coordinator-metrics,name=partition-load-time-maxThe maximum time taken in milliseconds to load the share-group state from the share-group state partitions.
partition-load-time-avgkafka.server:type=share-coordinator-metrics,name=partition-load-time-avgThe average time taken in milliseconds to load the share-group state from the share-group state partitions.
thread-idle-ratio-minkafka.server:type=share-coordinator-metrics,name=thread-idle-ratio-minThe minimum fraction of time the share coordinator thread is idle.
thread-idle-ratio-avgkafka.server:type=share-coordinator-metrics,name=thread-idle-ratio-avgThe average fraction of time the share coordinator thread is idle.
write-ratekafka.server:type=share-coordinator-metrics,name=write-rateThe number of share-group state write calls per second.
write-totalkafka.server:type=share-coordinator-metrics,name=write-totalThe total number of share-group state write calls.
write-latency-avgkafka.server:type=share-coordinator-metrics,name=write-latency-avgThe average time taken for a share-group state write call, including the time to write to the share-group state topic.
write-latency-maxkafka.server:type=share-coordinator-metrics,name=write-latency-maxThe maximum time taken for a share-group state write call, including the time to write to the share-group state topic.
num-partitionskafka.server:type=share-coordinator-metrics,name=num-partitions,state={loading|active|failed}The number of partitions in the share-state topic in each state.
last-pruned-offsetkafka.server:type=share-coordinator-metrics,name=last-pruned-offset,topic=([-.\w]+),partition=([0-9]+)The offset at which the share-group state topic was last pruned.
+ +
Client Metrics
+ The following metrics are available on share consumer instances: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Metric/Attribute nameMbean nameDescription
last-poll-seconds-agokafka.consumer:type=consumer-share-metrics,name=last-poll-seconds-ago,client-id=([-.\w]+)The number of seconds since the last poll() invocation.
time-between-poll-avgkafka.consumer:type=consumer-share-metrics,name=time-between-poll-avg,client-id=([-.\w]+)The average delay between invocations of poll() in milliseconds.
time-between-poll-maxkafka.consumer:type=consumer-share-metrics,name=time-between-poll-max,client-id=([-.\w]+)The maximum delay between invocations of poll() in milliseconds.
poll-idle-ratio-avgkafka.consumer:type=consumer-share-metrics,name=poll-idle-ratio-avg,client-id=([-.\w]+)The average fraction of time the consumer's poll() is idle as opposed to waiting for the user code to process records.
heartbeat-response-time-maxkafka.consumer:type=consumer-share-coordinator-metrics,name=heartbeat-response-time-max,client-id=([-.\w]+)The maximum time taken to receive a response to a heartbeat request in milliseconds.
heartbeat-ratekafka.consumer:type=consumer-share-coordinator-metrics,name=heartbeat-rate,client-id=([-.\w]+)The number of heartbeats per second.
heartbeat-totalkafka.consumer:type=consumer-share-coordinator-metrics,name=heartbeat-total,client-id=([-.\w]+)The total number of heartbeats.
last-heartbeat-seconds-agokafka.consumer:type=consumer-share-coordinator-metrics,name=last-heartbeat-seconds-ago,client-id=([-.\w]+)The number of seconds since the last coordinator heartbeat was sent.
rebalance-totalkafka.consumer:type=consumer-share-coordinator-metrics,name=rebalance-total,client-id=([-.\w]+)The total number of share group rebalances count.
rebalance-rate-per-hourkafka.consumer:type=consumer-share-coordinator-metrics,name=rebalance-rate-per-hour,client-id=([-.\w]+)The number of share group rebalances event per hour.
fetch-size-avgkafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-size-avg,client-id=([-.\w]+)The average number of bytes fetched per request.
fetch-size-maxkafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-size-max,client-id=([-.\w]+)The maximum number of bytes fetched per request.
records-per-request-avgkafka.consumer:type=consumer-share-fetch-manager-metrics,name=records-per-request-avg,client-id=([-.\w]+)The average number of records in each request.
records-per-request-maxkafka.consumer:type=consumer-share-fetch-manager-metrics,name=records-per-request-max,client-id=([-.\w]+)The maximum number of records in a request.
bytes-consumed-ratekafka.consumer:type=consumer-share-fetch-manager-metrics,name=bytes-consumed-rate,client-id=([-.\w]+)The average number of bytes consumed per second.
bytes-consumed-totalkafka.consumer:type=consumer-share-fetch-manager-metrics,name=bytes-consumed-total,client-id=([-.\w]+)The total number of bytes consumed.
records-consumed-ratekafka.consumer:type=consumer-share-fetch-manager-metrics,name=records-consumed-rate,client-id=([-.\w]+)The average number of records fetched per second.
records-consumed-totalkafka.consumer:type=consumer-share-fetch-manager-metrics,name=records-consumed-total,client-id=([-.\w]+)The total number of records fetched.
acknowledgements-send-ratekafka.consumer:type=consumer-share-fetch-manager-metrics,name=acknowledgements-send-rate,client-id=([-.\w]+)The average number of record acknowledgements sent per second.
acknowledgements-send-totalkafka.consumer:type=consumer-share-fetch-manager-metrics,name=acknowledgements-send-total,client-id=([-.\w]+)The total number of record acknowledgements sent.
acknowledgements-error-ratekafka.consumer:type=consumer-share-fetch-manager-metrics,name=acknowledgements-error-rate,client-id=([-.\w]+)The average number of record acknowledgements that resulted in errors per second.
acknowledgements-error-totalkafka.consumer:type=consumer-share-fetch-manager-metrics,name=acknowledgements-error-total,client-id=([-.\w]+)The total number of record acknowledgements that resulted in errors.
fetch-latency-avgkafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-latency-avg,client-id=([-.\w]+)The average time taken for a fetch request.
fetch-latency-maxkafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-latency-max,client-id=([-.\w]+)The maximum time taken for any fetch request.
fetch-ratekafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-rate,client-id=([-.\w]+)The number of fetch requests per second.
fetch-totalkafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-total,client-id=([-.\w]+)The total number of fetch requests.
fetch-throttle-time-avgkafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-throttle-time-avg,client-id=([-.\w]+)The average throttle time in milliseconds.
fetch-throttle-time-maxkafka.consumer:type=consumer-share-fetch-manager-metrics,name=fetch-throttle-time-max,client-id=([-.\w]+)The maximum throttle time in milliseconds.
+

Others

We recommend monitoring GC time and other stats and various server stats such as CPU utilization, I/O service time, etc. diff --git a/docs/toc.html b/docs/toc.html index dd1ecaa4c4a70..d7bc26d0f4e8f 100644 --- a/docs/toc.html +++ b/docs/toc.html @@ -162,6 +162,7 @@
  • Consumer Monitoring
  • Connect Monitoring
  • Streams Monitoring +
  • Share Group Monitoring
  • Others