jsancio commented on code in PR #18304: URL: https://github.com/apache/kafka/pull/18304#discussion_r1929185438
########## raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java: ########## @@ -130,7 +132,7 @@ public void truncateNewEntries(long endOffset) { } kafkaRaftMetrics.updateNumVoters(voterSetHistory.lastValue().size()); - if (staticVoterSet.isPresent() && voterSetHistory.lastEntry().isEmpty()) { + if (staticVoterSet.size() > 0 && voterSetHistory.lastEntry().isEmpty()) { Review Comment: Use `!staticVoterSet.isEmpty()`. Sometimes `size()` is not constant time. ########## raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java: ########## @@ -298,7 +300,7 @@ private void handleBatch(Batch<?> batch, OptionalLong overrideOffset) { case KRAFT_VOTERS: VoterSet voters = VoterSet.fromVotersRecord((VotersRecord) record.message()); kafkaRaftMetrics.updateNumVoters(voters.size()); - if (staticVoterSet.isPresent()) { + if (staticVoterSet.size() > 0) { Review Comment: Use `!staticVoterSet.isEmpty()`. Sometimes `size()` is not constant time. ########## raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java: ########## @@ -103,9 +103,11 @@ public KRaftControlRecordStateMachine( this.logger = logContext.logger(this.getClass()); this.kafkaRaftMetrics = kafkaRaftMetrics; this.externalKRaftMetrics = externalKRaftMetrics; - this.staticVoterSet = staticVoterSet.size() > 0 ? Optional.of(staticVoterSet) : Optional.empty(); + this.staticVoterSet = staticVoterSet; - this.staticVoterSet.ifPresent(voters -> kafkaRaftMetrics.updateNumVoters(voters.size())); + if (staticVoterSet.size() > 0) { + kafkaRaftMetrics.updateNumVoters(staticVoterSet.size()); + } Review Comment: ```java kafkaRaftMetrics.updateNumVoters(staticVoterSet.size()); ``` Do you need the conditional? -- 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