-
Notifications
You must be signed in to change notification settings - Fork 14.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-16524: Metrics for KIP-853 #18304
Conversation
A label of 'needs-attention' was automatically added to this PR in order to raise the |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the changes @kevin-wu24 . Quick review of src/main
.
raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
Outdated
Show resolved
Hide resolved
…ers and number-of-voters metrics be updated in the internal partition listener
…in a thread-safe manner
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your improvements @kevin-wu24
metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/ExternalKRaftMetrics.java
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/ExternalKRaftMetricIgnoredStaticVoters.java
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/ExternalKRaftMetricIgnoredStaticVoters.java
Outdated
Show resolved
Hide resolved
raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java
Outdated
Show resolved
Hide resolved
raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java
Show resolved
Hide resolved
raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java
Show resolved
Hide resolved
server/src/main/java/org/apache/kafka/server/metrics/BrokerServerMetrics.java
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/ExternalKRaftMetrics.java
Outdated
Show resolved
Hide resolved
Pushed some commits. Changes of note:
|
…ower, updated QuorumState flowchart
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the changes. I reviewed src/main
.
core/src/main/scala/kafka/raft/DefaultExternalKRaftMetrics.scala
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java
Outdated
Show resolved
Hide resolved
server/src/main/java/org/apache/kafka/server/metrics/BrokerServerMetrics.java
Outdated
Show resolved
Hide resolved
server/src/main/java/org/apache/kafka/server/metrics/BrokerServerMetrics.java
Outdated
Show resolved
Hide resolved
server/src/main/java/org/apache/kafka/server/metrics/BrokerServerMetrics.java
Outdated
Show resolved
Hide resolved
… we fallback to static voter config
…s to determine if truncation results in using static voters config
Thanks for the review @jsancio. Pushed some commits:
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. Submitted a quick review.
core/src/test/scala/kafka/raft/DefaultExternalKRaftMetricsTest.scala
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java
Outdated
Show resolved
Hide resolved
core/src/main/scala/kafka/raft/DefaultExternalKRaftMetrics.scala
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the update @kevin-wu24 . Let's make sure that the metrics that represent a boolean expose the value as 1 or 0 and not as true or false.
metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java
Outdated
Show resolved
Hide resolved
metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java
Show resolved
Hide resolved
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java
Outdated
Show resolved
Hide resolved
raft/src/test/java/org/apache/kafka/raft/MockExternalKRaftMetrics.java
Outdated
Show resolved
Hide resolved
raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java
Outdated
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
Show resolved
Hide resolved
raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
Outdated
Show resolved
Hide resolved
raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java
Outdated
Show resolved
Hide resolved
server/src/main/java/org/apache/kafka/server/metrics/BrokerServerMetrics.java
Outdated
Show resolved
Hide resolved
server/src/test/java/org/apache/kafka/server/metrics/BrokerServerMetricsTest.java
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
This change implement some of the metrics enumerated in KIP-853. The KafkaRaftMetrics object now exposes number-of-voters, number-of-observers and uncommitted-voter-change. The number-of-observers and uncommitted-voter-change metrics are only present on the active controller or leader, since it does not make sense for other replicas to report these metrics. In order to make these two metrics thread-safe, KafkaRaftMetrics needs to be passed into LeaderState, and therefore QuorumState. This introduces a circularity since the KafkaRaftMetrics constructor takes in QuorumState. To break the circularity for now, the logic using QuorumState will be moved to the KafkaRaftMetrics#initialize method. The BrokerServerMetrics object now exposes ignored-static-voters. The ControllerServerMetrics object now exposes IgnoredStaticVoters. To implement both metrics for "ignored static voters", this PR introduces the ExternalKRaftMetrics interface, which allows for higher layer metrics objects to be accessible within the raft module. Reviewers: José Armando García Sancio <[email protected]>
This PR implements the following gauges in KafkaRaftMetrics, BrokerServerMetrics, and ControllerServerMetrics:
KafkaRaftMetrics:
number-of-voters
: number of voters for a KRaft topic partition. All of the replicas will report this metric.number-of-observers
: number of observers being tracked by the KRaft topic partition leader.uncommitted-voter-change
: 1 if there is a voter change that has not been committed, 0 otherwise.number-of-observers
anduncommitted-voter-change
will only be present on the active controller, since it does not make sense for other replicas to report these metrics. Ifnumber-of-offline-voters
is implemented, it will also only be reported by the leader.In order to make these two metrics thread-safe,
KafkaRaftMetrics
needs to be passed intoLeaderState
, and thereforeQuorumState
. This introduces a circularity since theKafkaRaftMetrics
constructor takes inQuorumState
. To break the circularity for now, the logic usingQuorumState
will be moved to aKafkaRaftMetrics#initialize
method.BrokerServerMetrics:
ignored-static-voters
: 1 if reading the voter set from the log, 0 if the voter set comes from static configuration.ControllerServerMetrics:
IgnoredStaticVoters
: 1 if reading the voter set from the log, 0 if the voter set comes from static configuration. (same as the metric above but for controllers)This metric will not be implemented in this PR. Instead, theIsObserver
: 1 if the controller node is not a voter for the KRaft cluster metadata partition, 0 otherwise.current-state
metric in conjunction with the presence ofkafka.server.controller
metrics can be used to provide the same information.To implement the
ignored-static-voters
metrics, this PR introduces theExternalKRaftMetrics
class, which allows for higher layer metrics objects to be accessible within theraft
module.number-of-voters
andignored-static-voters
will be updated inKRaftControlRecordStateMachine#handleBatch
when a new VotersRecord is read from log.Testing:
KRaftControlRecordStateMachineTest
andKafkaRaftMetricsTest
.KafkaRaftClientReconfigTest
to verify new metrics behavior.Committer Checklist (excluded from commit message)