jsancio commented on code in PR #18304: URL: https://github.com/apache/kafka/pull/18304#discussion_r1934299488
########## metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java: ########## @@ -117,6 +120,13 @@ public Integer value() { })); registry.ifPresent(r -> uncleanLeaderElectionMeter = Optional.of(registry.get().newMeter(UNCLEAN_LEADER_ELECTIONS_PER_SEC, "elections", TimeUnit.SECONDS))); + + registry.ifPresent(r -> r.newGauge(IGNORED_STATIC_VOTERS, new Gauge<Boolean>() { Review Comment: I took at look at this using the MBeans browser. Using a boolean doesn't expose the value as 1 for true and 0 for false. You need to expose the gauge as an integer: `return ignoredStaticVoter() ? 1 : 0` ########## raft/src/test/java/org/apache/kafka/raft/MockExternalKRaftMetrics.java: ########## @@ -0,0 +1,35 @@ +/* + * 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 org.apache.kafka.raft; + +/** + * This class is used to replicate the behavior of {@link kafka.raft.DefaultExternalKRaftMetrics} + * for testing within the raft layer. + */ +public class MockExternalKRaftMetrics implements ExternalKRaftMetrics { Review Comment: This seems to be a very simple mock. Did you try using Mockito's mock? E.g. `Mockito.mock`. If you search for that method, you should find many examples of using that method. ########## raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java: ########## @@ -51,10 +58,28 @@ private static KRaftControlRecordStateMachine buildPartitionListener(MockLog log STRING_SERDE, BufferSupplier.NO_CACHING, 1024, - new LogContext() + new LogContext(), + kafkaRaftMetrics, + externalMetrics ); } + private static void checkMetricValues(int expectedNumberOfVoters, boolean expectedIgnoredStaticVoters) { + assertEquals(expectedNumberOfVoters, getNumberOfVoters(metrics).metricValue()); + assertEquals(expectedIgnoredStaticVoters, externalMetrics.getIgnoredStaticVoters()); + } + + private static KafkaMetric getNumberOfVoters(final Metrics metrics) { + return metrics.metrics().get(metrics.metricName("number-of-voters", "raft-metrics")); + } + + @BeforeEach + public void resetMetrics() { + metrics = new Metrics(); + kafkaRaftMetrics = new KafkaRaftMetrics(metrics, "raft"); + externalMetrics = new MockExternalKRaftMetrics(); + } Review Comment: Please void using non-final static or instance variables, and resetting them using `BeforeEach` and `AfterEach` methods. They make tests very difficult to read and maintain. Please create these objects explicitly in every test that needs them. ########## raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java: ########## @@ -137,6 +99,14 @@ public KafkaRaftMetrics(Metrics metrics, String metricGrpPrefix, QuorumState sta "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.numVotersMetricName = metrics.metricName("number-of-voters", metricGroupName, "Number of voters for a KRaft topic partition."); + metrics.addMetric(this.numVotersMetricName, (Gauge<Integer>) (mConfig, currentTimestamp) -> numVoters); Review Comment: I still don't understand why you need this cast to `Gauge<Integer>`. The metric above `numUnknownVoterConnectionsMetricName` is also an integer and it doesn't need a cast. Why is this behaving differently? ########## metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java: ########## @@ -48,6 +48,7 @@ public void testMetricNames() { "kafka.controller:type=KafkaController,name=MetadataErrorCount", "kafka.controller:type=KafkaController,name=OfflinePartitionsCount", "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount", + "kafka.controller:type=KafkaController,name=IgnoredStaticVoters", Review Comment: Let's add a test specific for this metric that checks that the value is 1 or 0 when expected. ########## server/src/main/java/org/apache/kafka/server/metrics/BrokerServerMetrics.java: ########## @@ -89,12 +93,18 @@ public BrokerServerMetrics(Metrics metrics) { METRIC_GROUP_NAME, "The number of errors encountered by the BrokerMetadataPublisher while applying a new MetadataImage based on the latest MetadataDelta." ); + ignoredStaticVotersName = metrics.metricName( + "ignored-static-voters", + METRIC_GROUP_NAME, + "1 if controller.quorum.voters is set but was not used by the broker, 0 otherwise." + ); metrics.addMetric(lastAppliedRecordOffsetName, (config, now) -> lastAppliedImageProvenance.get().lastContainedOffset()); metrics.addMetric(lastAppliedRecordTimestampName, (config, now) -> lastAppliedImageProvenance.get().lastContainedLogTimeMs()); metrics.addMetric(lastAppliedRecordLagMsName, (config, now) -> now - lastAppliedImageProvenance.get().lastContainedLogTimeMs()); metrics.addMetric(metadataLoadErrorCountName, (config, now) -> metadataLoadErrorCount.get()); metrics.addMetric(metadataApplyErrorCountName, (config, now) -> metadataApplyErrorCount.get()); + metrics.addMetric(ignoredStaticVotersName, (Gauge<Boolean>) (config, now) -> ignoredStaticVoters.get()); Review Comment: According to the KIP is metrics is expected to have the values 1 and 0, not true and false. ########## raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java: ########## @@ -2351,4 +2459,21 @@ private static ApiVersionsResponseData apiVersionsResponse(Errors error, Support .setErrorCode(error.code()) .setSupportedFeatures(supportedFeatures); } + + private static KafkaMetric getMetric(final Metrics metrics, final String name) { + return metrics.metrics().get(metrics.metricName(name, "raft-metrics")); + } + + private static void checkLeaderMetricValues( + final int expectedVoters, + final int expectedObservers, + final boolean expectedUncommittedVoterChange, + final boolean expectedIgnoredStaticVoters, Review Comment: I couldn't find a single test in this suite that check that this value (`expectedIgnoredStaticVoters`) is true. ########## raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java: ########## @@ -207,13 +229,37 @@ public void updateElectionStartMs(long currentTimeMs) { electionStartMs = OptionalLong.of(currentTimeMs); } + public void updateNumVoters(int numVoters) { + this.numVoters = numVoters; + } + + public void updateNumObservers(int numObservers) { + this.numObservers = numObservers; + } + + public void updateUncommittedVoterChange(boolean uncommittedVoterChange) { + this.uncommittedVoterChange = uncommittedVoterChange; + } + public void maybeUpdateElectionLatency(long currentTimeMs) { if (electionStartMs.isPresent()) { electionTimeSensor.record(currentTimeMs - electionStartMs.getAsLong(), currentTimeMs); electionStartMs = OptionalLong.empty(); } } + public void addLeaderMetrics() { + metrics.addMetric(numObserversMetricName, (Gauge<Integer>) (config, now) -> numObservers); + metrics.addMetric(uncommittedVoterChangeMetricName, (Gauge<Boolean>) (config, now) -> uncommittedVoterChange); Review Comment: This exposes the metrics as false/true. The KIP mentions that the metric is meant to expose 0/1. ########## server/src/test/java/org/apache/kafka/server/metrics/BrokerServerMetricsTest.java: ########## @@ -47,7 +47,8 @@ public void testMetricsExported() throws Exception { new MetricName("last-applied-record-timestamp", expectedGroup, "", Collections.emptyMap()), new MetricName("last-applied-record-lag-ms", expectedGroup, "", Collections.emptyMap()), new MetricName("metadata-load-error-count", expectedGroup, "", Collections.emptyMap()), - new MetricName("metadata-apply-error-count", expectedGroup, "", Collections.emptyMap()) + new MetricName("metadata-apply-error-count", expectedGroup, "", Collections.emptyMap()), + new MetricName("ignored-static-voters", expectedGroup, "", Collections.emptyMap()) Review Comment: Can we add a test specific to the new metrics "ignored-static-voters"? ########## raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java: ########## @@ -469,6 +478,54 @@ public void shouldRecordRate(KRaftVersion kraftVersion) { assertEquals(2.4, getMetric(metrics, "fetch-records-rate").metricValue()); } + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testNumberOfVoters(KRaftVersion kraftVersion) { + raftMetrics = new KafkaRaftMetrics(metrics, "raft"); + QuorumState state = buildQuorumState(localStandaloneVoterSet(kraftVersion), kraftVersion); + raftMetrics.initialize(state); + state.initialize(new OffsetAndEpoch(0L, 0)); + + assertEquals(0, getMetric(metrics, "number-of-voters").metricValue()); + + raftMetrics.updateNumVoters(3); + assertEquals(3, getMetric(metrics, "number-of-voters").metricValue()); + } + + @ParameterizedTest + @EnumSource(value = KRaftVersion.class) + public void testLeaderMetrics(KRaftVersion kraftVersion) { + raftMetrics = new KafkaRaftMetrics(metrics, "raft"); + QuorumState state = buildQuorumState(localStandaloneVoterSet(kraftVersion), kraftVersion); + raftMetrics.initialize(state); + state.initialize(new OffsetAndEpoch(0L, 0)); + + assertNull(getMetric(metrics, "number-of-observers")); + assertNull(getMetric(metrics, "uncommitted-voter-change")); + + raftMetrics.addLeaderMetrics(); + + assertEquals(0, getMetric(metrics, "number-of-observers").metricValue()); + assertEquals(false, getMetric(metrics, "uncommitted-voter-change").metricValue()); Review Comment: I made a similar comment in the `src/main` implementation but this metric should be an integer with the values 1 or 0. -- 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