mirror of https://github.com/apache/kafka.git
MINOR: Adding KRaft Monitoring Related Metrics to docs/ops.html (#12679)
This commit adds KRaft monitoring related metrics to the Kafka docs (docs/ops.html). Reviewers: Jason Gustafson <jason@confluent.io>, Luke Chen <showuon@gmail.com>
This commit is contained in:
parent
b0ace18035
commit
eb8f0bd5e4
200
docs/ops.html
200
docs/ops.html
|
@ -1815,6 +1815,206 @@ $ bin/kafka-acls.sh \
|
|||
</tr>
|
||||
</tbody></table>
|
||||
|
||||
<h4 class="anchor-heading"><a id="kraft_monitoring" class="anchor-link"></a><a href="#kraft_monitoring">KRaft Monitoring Metrics</a></h4>
|
||||
The set of metrics that allow monitoring of the KRaft quorum and the metadata log.<br>
|
||||
Note that some exposed metrics depend on the role of the node as defined by <code>process.roles</code>
|
||||
<h5 class="anchor-heading"><a id="kraft_quorum_monitoring" class="anchor-link"></a><a href="#kraft_quorum_monitoring">KRaft Quorum Monitoring Metrics</a></h5>
|
||||
These metrics are reported on both Controllers and Brokers in a KRaft Cluster
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
<tr>
|
||||
<th>Metric/Attribute name</th>
|
||||
<th>Description</th>
|
||||
<th>Mbean name</th>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Current State</td>
|
||||
<td>The current state of this member; possible values are leader, candidate, voted, follower, unattached.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=current-state</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Current Leader</td>
|
||||
<td>The current quorum leader's id; -1 indicates unknown.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=current-leader</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Current Voted</td>
|
||||
<td>The current voted leader's id; -1 indicates not voted for anyone.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=current-vote</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Current Epoch</td>
|
||||
<td>The current quorum epoch.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=current-epoch</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>High Watermark</td>
|
||||
<td>The high watermark maintained on this member; -1 if it is unknown.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=high-watermark</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Log End Offset</td>
|
||||
<td>The current raft log end offset.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=log-end-offset</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Number of Unknown Voter Connections</td>
|
||||
<td>Number of unknown voters whose connection information is not cached. This value of this metric is always 0.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=number-unknown-voter-connections</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Average Commit Latency</td>
|
||||
<td>The average time in milliseconds to commit an entry in the raft log.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=commit-latency-avg</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Maximum Commit Latency</td>
|
||||
<td>The maximum time in milliseconds to commit an entry in the raft log.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=commit-latency-max</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Average Election Latency</td>
|
||||
<td>The average time in milliseconds spent on electing a new leader.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=election-latency-avg</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Maximum Election Latency</td>
|
||||
<td>The maximum time in milliseconds spent on electing a new leader.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=election-latency-max</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Fetch Records Rate</td>
|
||||
<td>The average number of records fetched from the leader of the raft quorum.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=fetch-records-rate</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Append Records Rate</td>
|
||||
<td>The average number of records appended per sec by the leader of the raft quorum.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=append-records-raft</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Average Poll Idle Ratio</td>
|
||||
<td>The average fraction of time the client's poll() is idle as opposed to waiting for the user code to process records.</td>
|
||||
<td>kafka.server:type=raft-metrics,name=poll-idle-ratio-avg</td>
|
||||
</tr>
|
||||
</tbody>
|
||||
</table>
|
||||
<h5 class="anchor-heading"><a id="kraft_controller_monitoring" class="anchor-link"></a><a href="#kraft_controller_monitoring">KRaft Controller Monitoring Metrics</a></h5>
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
<tr>
|
||||
<th>Metric/Attribute name</th>
|
||||
<th>Description</th>
|
||||
<th>Mbean name</th>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Active Controller Count</td>
|
||||
<td>The number of Active Controllers on this node. Valid values are '0' or '1'.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=ActiveControllerCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Event Queue Time Ms</td>
|
||||
<td>A Histogram of the time in milliseconds that requests spent waiting in the Controller Event Queue.</td>
|
||||
<td>kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Event Queue Processing Time Ms</td>
|
||||
<td>A Histogram of the time in milliseconds that requests spent being processed in the Controller Event Queue.</td>
|
||||
<td>kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Fenced Broker Count</td>
|
||||
<td>The number of fenced brokers as observed by this Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=FencedBrokerCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Active Broker Count</td>
|
||||
<td>The number of fenced brokers as observed by this Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=ActiveBrokerCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Global Topic Count</td>
|
||||
<td>The number of global topics as observed by this Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=GlobalTopicCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Global Partition Count</td>
|
||||
<td>The number of global partitions as observed by this Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=GlobalPartitionCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Offline Partition Count</td>
|
||||
<td>The number of offline topic partitions (non-internal) as observed by this Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=OfflinePartitionCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Preferred Replica Imbalance Count</td>
|
||||
<td>The count of topic partitions for which the leader is not the preferred leader.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Metadata Error Count</td>
|
||||
<td>The number of times this controller node has encountered an error during metadata log processing.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=MetadataErrorCount</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Last Applied Record Offset</td>
|
||||
<td>The offset of the last record from the cluster metadata partition that was applied by the Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=LastAppliedRecordOffset</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Last Committed Record Offset</td>
|
||||
<td>The offset of the last record committed to this Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=LastCommittedRecordOffset</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Last Applied Record Timestamp</td>
|
||||
<td>The timestamp of the last record from the cluster metadata partition that was applied by the Controller.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=LastAppliedRecordTimestamp</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Last Applied Record Lag Ms</td>
|
||||
<td>The difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the controller.
|
||||
For active Controllers the value of this lag is always zero.</td>
|
||||
<td>kafka.controller:type=KafkaController,name=LastAppliedRecordLagMs</td>
|
||||
</tr>
|
||||
</tbody>
|
||||
</table>
|
||||
<h5 class="anchor-heading"><a id="kraft_broker_monitoring" class="anchor-link"></a><a href="#kraft_broker_monitoring">KRaft Broker Monitoring Metrics</a></h5>
|
||||
<table class="data-table">
|
||||
<tbody>
|
||||
<tr>
|
||||
<th>Metric/Attribute name</th>
|
||||
<th>Description</th>
|
||||
<th>Mbean name</th>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Last Applied Record Offset</td>
|
||||
<td>The offset of the last record from the cluster metadata partition that was applied by the broker</td>
|
||||
<td>kafka.server:type=broker-metadata-metrics,name=last-applied-record-offset</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Last Applied Record Timestamp</td>
|
||||
<td>The timestamp of the last record from the cluster metadata partition that was applied by the broker.</td>
|
||||
<td>kafka.server:type=broker-metadata-metrics,name=last-applied-record-timestamp</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Last Applied Record Lag Ms</td>
|
||||
<td>The difference between now and the timestamp of the last record from the cluster metadata partition that was applied by the broker</td>
|
||||
<td>kafka.server:type=broker-metadata-metrics,name=last-applied-record-lag-ms</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Metadata Load Error Count</td>
|
||||
<td>The number of errors encountered by the BrokerMetadataListener while loading the metadata log and generating a new MetadataDelta based on it.</td>
|
||||
<td>kafka.server:type=broker-metadata-metrics,name=metadata-load-error-count</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Metadata Apply Error Count</td>
|
||||
<td>The number of errors encountered by the BrokerMetadataPublisher while applying a new MetadataImage based on the latest MetadataDelta.</td>
|
||||
<td>kafka.server:type=broker-metadata-metrics,name=metadata-apply-error-count</td>
|
||||
</tr>
|
||||
</tbody>
|
||||
</table>
|
||||
<h4><a id="selector_monitoring" href="#selector_monitoring">Common monitoring metrics for producer/consumer/connect/streams</a></h4>
|
||||
|
||||
The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.
|
||||
|
|
|
@ -246,7 +246,9 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
|
|||
logContext,
|
||||
random);
|
||||
this.kafkaRaftMetrics = new KafkaRaftMetrics(metrics, "raft", quorum);
|
||||
kafkaRaftMetrics.updateNumUnknownVoterConnections(quorum.remoteVoters().size());
|
||||
// All Raft voters are statically configured and known at startup
|
||||
// so there are no unknown voter connections. Report this metric as 0.
|
||||
kafkaRaftMetrics.updateNumUnknownVoterConnections(0);
|
||||
|
||||
// Update the voter endpoints with what's in RaftConfig
|
||||
Map<Integer, RaftConfig.AddressSpec> voterAddresses = raftConfig.quorumVoterConnections();
|
||||
|
|
|
@ -107,7 +107,8 @@ public class KafkaRaftMetrics implements AutoCloseable {
|
|||
this.logEndEpochMetricName = metrics.metricName("log-end-epoch", metricGroupName, "The current raft log end epoch.");
|
||||
metrics.addMetric(this.logEndEpochMetricName, (mConfig, currentTimeMs) -> logEndOffset.epoch);
|
||||
|
||||
this.numUnknownVoterConnectionsMetricName = metrics.metricName("number-unknown-voter-connections", metricGroupName, "The number of voter connections recognized at this member.");
|
||||
this.numUnknownVoterConnectionsMetricName = metrics.metricName("number-unknown-voter-connections", metricGroupName,
|
||||
"Number of unknown voters whose connection information is not cached; would never be larger than quorum-size.");
|
||||
metrics.addMetric(this.numUnknownVoterConnectionsMetricName, (mConfig, currentTimeMs) -> numUnknownVoterConnections);
|
||||
|
||||
this.commitTimeSensor = metrics.sensor("commit-latency");
|
||||
|
@ -118,9 +119,9 @@ public class KafkaRaftMetrics implements AutoCloseable {
|
|||
|
||||
this.electionTimeSensor = metrics.sensor("election-latency");
|
||||
this.electionTimeSensor.add(metrics.metricName("election-latency-avg", metricGroupName,
|
||||
"The average time in milliseconds to elect a new leader."), new Avg());
|
||||
"The average time in milliseconds spent on electing a new leader."), new Avg());
|
||||
this.electionTimeSensor.add(metrics.metricName("election-latency-max", metricGroupName,
|
||||
"The maximum time in milliseconds to elect a new leader."), new Max());
|
||||
"The maximum time in milliseconds spent on electing a new leader."), new Max());
|
||||
|
||||
this.fetchRecordsSensor = metrics.sensor("fetch-records");
|
||||
this.fetchRecordsSensor.add(metrics.metricName("fetch-records-rate", metricGroupName,
|
||||
|
|
Loading…
Reference in New Issue