jsancio commented on code in PR #18304:
URL: https://github.com/apache/kafka/pull/18304#discussion_r1928962363


##########
raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java:
##########
@@ -70,6 +71,9 @@ public final class KRaftControlRecordStateMachine {
     // 2. The read operations lastVoterSet, voterSetAtOffset and 
kraftVersionAtOffset read
     // the nextOffset first before reading voterSetHistory or 
kraftVersionHistory
     private volatile long nextOffset = STARTING_NEXT_OFFSET;
+    private final KafkaRaftMetrics kafkaRaftMetrics;
+    private final ExternalKRaftMetrics externalKRaftMetrics;
+    private final Optional<VoterSet> staticVoterSet;

Review Comment:
   You can just model this with `VoterSet staticVoterSet`. `Optional.empty()` 
has the same meaning as `VoterSet.empty()` or `VoterSet#isEmpty()`.



##########
core/src/test/scala/kafka/raft/DefaultExternalKRaftMetricsTest.scala:
##########
@@ -38,17 +38,17 @@ final class DefaultExternalKRaftMetricsTest {
     assertFalse(brokerServerMetrics.ignoredStaticVoters())
     assertFalse(controllerMetadataMetrics.ignoredStaticVoters())
 
-    metrics.setIgnoredStaticVoters()
+    metrics.setIgnoredStaticVoters(true)
 
     assertTrue(brokerServerMetrics.ignoredStaticVoters())
     assertTrue(controllerMetadataMetrics.ignoredStaticVoters())
 
-    metrics.setIgnoredStaticVoters()
+    metrics.setIgnoredStaticVoters(false)
 
-    assertTrue(brokerServerMetrics.ignoredStaticVoters())
-    assertTrue(controllerMetadataMetrics.ignoredStaticVoters())
+    assertFalse(brokerServerMetrics.ignoredStaticVoters())
+    assertFalse(controllerMetadataMetrics.ignoredStaticVoters())
 
     metrics = new DefaultExternalKRaftMetrics(None, None)
-    metrics.setIgnoredStaticVoters()
+    metrics.setIgnoredStaticVoters(true)

Review Comment:
   Make this a different test.



##########
core/src/main/scala/kafka/raft/DefaultExternalKRaftMetrics.scala:
##########
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.raft
+
+import org.apache.kafka.controller.metrics.ControllerMetadataMetrics
+import org.apache.kafka.raft.ExternalKRaftMetrics
+import org.apache.kafka.server.metrics.BrokerServerMetrics
+
+class DefaultExternalKRaftMetrics(
+  val brokerServerMetricsOpt: Option[BrokerServerMetrics],
+  val controllerMetadataMetricsOpt: Option[ControllerMetadataMetrics]

Review Comment:
   Okay but in general Kafka doesn't tend to encode the type (Option) of the 
variable in the variable name. E.g. you can call it `controllerMetadataMetrics` 
instead of `controllerMetadataMetricsOpt`.



##########
raft/src/main/java/org/apache/kafka/raft/QuorumState.java:
##########
@@ -722,8 +727,11 @@ public <T> LeaderState<T> transitionToLeader(long 
epochStartOffset, BatchAccumul
             accumulator,
             localListeners,
             fetchTimeoutMs,
-            logContext
+            logContext,
+            kafkaRaftMetrics
         );
+        kafkaRaftMetrics.addLeaderMetrics();

Review Comment:
   Since `LeaderState#close` removes these metrics you should keep this 
coupling in the `LeaderState` constructor. In other words move 
`kafkaRaftMetrics.addLeaderMetrics()` to the constructor for `LeaderState`. 
This is also called resource acquisition is initialization (RAII).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to