You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/06/26 18:22:53 UTC

[GitHub] [kafka] zhaohaidao opened a new pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

zhaohaidao opened a new pull request #10932:
URL: https://github.com/apache/kafka/pull/10932


   …o load snapshot
   
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r661037975



##########
File path: raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
##########
@@ -1014,6 +1015,26 @@ public void verify() {
         }
     }
 
+    private static class LeaderNeverLoadSnapshot implements Invariant {
+        final Cluster cluster;
+
+        private LeaderNeverLoadSnapshot(Cluster cluster) {
+            this.cluster = cluster;
+        }
+
+        @Override
+        public void verify() {
+            for (RaftNode raftNode : cluster.running()) {
+                if (raftNode.counter.isWritable()) {
+                    assertTrue(raftNode.counter.getHandleSnapshotCalls() == 0);

Review comment:
       done




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r662378094



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +160,18 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalls = 0;
+    }
+
+    public int handleSnapshotCalls() {

Review comment:
       fair enough




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r660772538



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -38,6 +38,9 @@
     private OptionalInt claimedEpoch = OptionalInt.empty();
     private long lastOffsetSnapshotted = -1;
 
+    private int handleSnapshotCalls = 0;
+    private boolean handleSnapshotCalled = false;

Review comment:
       fair enough




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] jsancio commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

Posted by GitBox <gi...@apache.org>.
jsancio commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r660142781



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +162,27 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalled = false;
+        handleSnapshotCalls = 0;
+    }
+
+    public boolean isLeader() {
+        return this.client.leaderAndEpoch().isLeader(nodeId);

Review comment:
       https://issues.apache.org/jira/browse/KAFKA-13006




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#issuecomment-870739108


   @jsancio Thanks for your advice. The comments have been addressed. Please continue to review  when you have time.


-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r662377755



##########
File path: raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
##########
@@ -1014,6 +1015,26 @@ public void verify() {
         }
     }
 
+    private static class LeaderNeverLoadSnapshot implements Invariant {
+        final Cluster cluster;
+
+        private LeaderNeverLoadSnapshot(Cluster cluster) {
+            this.cluster = cluster;
+        }
+
+        @Override
+        public void verify() {
+            for (RaftNode raftNode : cluster.running()) {
+                if (raftNode.counter.isWritable()) {
+                    assertEquals(0, raftNode.counter.handleSnapshotCalls());
+                } else {
+                    assertTrue(raftNode.counter.handleSnapshotCalls() >= 0);

Review comment:
       fair enough.  No assertion is needed if this is a follower node.

##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +160,18 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);

Review comment:
       done




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] abbccdda merged pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
abbccdda merged pull request #10932:
URL: https://github.com/apache/kafka/pull/10932


   


-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] jsancio commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

Posted by GitBox <gi...@apache.org>.
jsancio commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r660140438



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -38,6 +38,9 @@
     private OptionalInt claimedEpoch = OptionalInt.empty();
     private long lastOffsetSnapshotted = -1;
 
+    private int handleSnapshotCalls = 0;
+    private boolean handleSnapshotCalled = false;

Review comment:
       We should only store `handleSnapshotCalls` since `handleSnapshotCalled` is always equal to `handleSnapshotCalls > 0`.

##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +162,27 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalled = false;
+        handleSnapshotCalls = 0;
+    }
+
+    public boolean isLeader() {
+        return this.client.leaderAndEpoch().isLeader(nodeId);

Review comment:
       I think this is the issue you reported in the Jira. The `RaftClient.Listener` should not use `RaftClient.leaderAndEpoch` to determine if it is the leader. It should instead use `RaftClient.Listener.handleLeaderChange`.
   
   For this state machine `ReplicatedCounter` we should look at the `claimedEpoch` variable.
   
   I am going to create an issue to remove this method. cc @hachikuji 




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] jsancio commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

Posted by GitBox <gi...@apache.org>.
jsancio commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r660140438



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -38,6 +38,9 @@
     private OptionalInt claimedEpoch = OptionalInt.empty();
     private long lastOffsetSnapshotted = -1;
 
+    private int handleSnapshotCalls = 0;
+    private boolean handleSnapshotCalled = false;

Review comment:
       We should only store `handleSnapshotCalls` since `handleSnapshotCalled` is always equal to `handleSnapshotCalls > 0`.

##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +162,27 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalled = false;
+        handleSnapshotCalls = 0;
+    }
+
+    public boolean isLeader() {
+        return this.client.leaderAndEpoch().isLeader(nodeId);

Review comment:
       I think this is the issue you reported in the Jira. The `RaftClient.Listener` should not use `RaftClient.leaderAndEpoch` to determine if it is the leader. It should instead use `RaftClient.Listener.handleLeaderChange`.
   
   For this state machine `ReplicatedCounter` we should look at the `claimedEpoch` variable.
   
   I am going to create an issue to remove this method. cc @hachikuji 

##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +162,27 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalled = false;
+        handleSnapshotCalls = 0;
+    }
+
+    public boolean isLeader() {
+        return this.client.leaderAndEpoch().isLeader(nodeId);

Review comment:
       https://issues.apache.org/jira/browse/KAFKA-13006




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] showuon commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r662155569



##########
File path: raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
##########
@@ -1014,6 +1015,26 @@ public void verify() {
         }
     }
 
+    private static class LeaderNeverLoadSnapshot implements Invariant {
+        final Cluster cluster;
+
+        private LeaderNeverLoadSnapshot(Cluster cluster) {
+            this.cluster = cluster;
+        }
+
+        @Override
+        public void verify() {
+            for (RaftNode raftNode : cluster.running()) {
+                if (raftNode.counter.isWritable()) {
+                    assertEquals(0, raftNode.counter.handleSnapshotCalls());
+                } else {
+                    assertTrue(raftNode.counter.handleSnapshotCalls() >= 0);

Review comment:
       Is there any way we can make sure it's `> 0`? If we can't, I think we should remove this verification since the `handleSnapshotCalls` must be `>= 0` no matter what happened, right?




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#issuecomment-872330694


   > @zhaohaidao , thanks for the PR. Left some comments. Thanks.
   
   Thanks for your advice. Comments have been addressed. Please continue to review if you have time.


-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r660772174



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +162,27 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalled = false;
+        handleSnapshotCalls = 0;
+    }
+
+    public boolean isLeader() {
+        return this.client.leaderAndEpoch().isLeader(nodeId);

Review comment:
       done




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r661038035



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +160,18 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalls = 0;
+    }
+
+    public int getHandleSnapshotCalls() {

Review comment:
       done




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] showuon commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r662157582



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +160,18 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);

Review comment:
       nit: wrong indent.




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#issuecomment-873551428


   @abbccdda Hi, Could you review this pr if you have time.


-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] zhaohaidao commented on pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
zhaohaidao commented on pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#issuecomment-872975936


   @jsancio @showuon Thank you. Could you please merge this pr to trunk if it's approved. 


-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] showuon commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked to load snapshot

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r662158289



##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +160,18 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalls = 0;
+    }
+
+    public int handleSnapshotCalls() {

Review comment:
       nit: Could we add some java doc here to explain why we need this? Thanks.




-- 
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-unsubscribe@kafka.apache.org

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



[GitHub] [kafka] jsancio commented on a change in pull request #10932: KAFKA-12958: add an invariant that notified leaders are never asked t…

Posted by GitBox <gi...@apache.org>.
jsancio commented on a change in pull request #10932:
URL: https://github.com/apache/kafka/pull/10932#discussion_r660788266



##########
File path: raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
##########
@@ -1014,6 +1015,26 @@ public void verify() {
         }
     }
 
+    private static class LeaderNeverLoadSnapshot implements Invariant {
+        final Cluster cluster;
+
+        private LeaderNeverLoadSnapshot(Cluster cluster) {
+            this.cluster = cluster;
+        }
+
+        @Override
+        public void verify() {
+            for (RaftNode raftNode : cluster.running()) {
+                if (raftNode.counter.isWritable()) {
+                    assertTrue(raftNode.counter.getHandleSnapshotCalls() == 0);

Review comment:
       Let's use `assertEquals`:
   ```suggestion
                       assertEquals(0, raftNode.counter.getHandleSnapshotCalls());
   ```

##########
File path: raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java
##########
@@ -157,13 +160,18 @@ public synchronized void handleSnapshot(SnapshotReader<Integer> reader) {
     public synchronized void handleLeaderChange(LeaderAndEpoch newLeader) {
         if (newLeader.isLeader(nodeId)) {
             log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}",
-                committed, newLeader);
+                    committed, newLeader);
             uncommitted = committed;
             claimedEpoch = OptionalInt.of(newLeader.epoch());
         } else {
             log.debug("Counter uncommitted value reset after resigning leadership");
             uncommitted = -1;
             claimedEpoch = OptionalInt.empty();
         }
+        handleSnapshotCalls = 0;
+    }
+
+    public int getHandleSnapshotCalls() {

Review comment:
       The Kafka project tends not to prefix accessors with `get`:
   ```suggestion
       public int handleSnapshotCalls() {
   ```




-- 
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-unsubscribe@kafka.apache.org

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