mumrah commented on code in PR #13643:
URL: https://github.com/apache/kafka/pull/13643#discussion_r1275381984


##########
raft/src/main/java/org/apache/kafka/raft/RaftClient.java:
##########
@@ -172,15 +173,17 @@ default void beginShutdown() {}
      * uncommitted entries after observing an epoch change.

Review Comment:
   It's probably worth adding a sentence or two about the new optimistic 
concurrency.



##########
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java:
##########
@@ -1168,15 +1165,15 @@ private void updateWriteOffset(long offset) {
         }
     }
 
-    private void claim(int epoch) {
+    private void claim(int epoch, long newLastWriteOffset) {
         try {
             if (curClaimEpoch != -1) {
                 throw new RuntimeException("Cannot claim leadership because we 
are already the " +
                         "active controller.");
             }
             curClaimEpoch = epoch;
             controllerMetrics.setActive(true);
-            updateWriteOffset(lastCommittedOffset);
+            updateWriteOffset(newLastWriteOffset);

Review Comment:
   Previously, we would update `lastCommittedOffset` as we got the 
`handleCommit` callback from our RaftClient. Since we process Raft events 
sequentially (and they are delivered sequentially from a single thread), we 
always process any commit callbacks before a leader change. Which means this 
offset is valid with respect to the end offset when the leadership changed.
   
   Now, while we're processing a leader change, we _ask_ RaftClient for its end 
offset. Is there any possibility that commits could be made that would make 
this end offset greater than we expect? Basically, can we be sure that the end 
offset doesn't change between the time Raft becomes the leader and this event 
is processed?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to