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/04/08 11:25:07 UTC

[GitHub] [kafka] dengziming commented on a change in pull request #10481: KAFKA-12619; Raft leader should expose hw only after committing LeaderChange

dengziming commented on a change in pull request #10481:
URL: https://github.com/apache/kafka/pull/10481#discussion_r609589579



##########
File path: raft/src/main/java/org/apache/kafka/raft/LeaderState.java
##########
@@ -113,13 +113,18 @@ private boolean updateHighWatermark() {
         Optional<LogOffsetMetadata> highWatermarkUpdateOpt = followersByDescendingFetchOffset.get(indexOfHw).endOffset;
 
         if (highWatermarkUpdateOpt.isPresent()) {
-            // When a leader is first elected, it cannot know the high watermark of the previous
-            // leader. In order to avoid exposing a non-monotonically increasing value, we have
-            // to wait for followers to catch up to the start of the leader's epoch.
+
+            // The KRaft protocol requires an extra condition on commitment after a leader
+            // election. The leader must commit one record from its own epoch before it is
+            // allowed to expose records from any previous epoch. This guarantees that its
+            // log will contain the largest record (in terms of epoch/offset) in any log
+            // which ensures that any future leader will have replicated this record as well
+            // as all records from previous epochs that the current leader has committed.
+
             LogOffsetMetadata highWatermarkUpdateMetadata = highWatermarkUpdateOpt.get();
             long highWatermarkUpdateOffset = highWatermarkUpdateMetadata.offset;
 
-            if (highWatermarkUpdateOffset >= epochStartOffset) {
+            if (highWatermarkUpdateOffset > epochStartOffset) {

Review comment:
       I was also confused by this logic for a while and I got an optimization idea from the [Raft dissertation](http://wcl.cs.rpi.edu/pilots/library/papers/consensus/RAFTOngaroPhD.pdf) about this, The sentence below is taken from section "5.4.2 Committing entries from previous terms":
   ```
   There are some situations where a leader could safely conclude that an older log entry is committed (for example, if that entry is stored on every server), but Raft takes a more conservative approach for simplicity.
   ```
   so we can also update commitIndex(highWatermark) if logEndOffset of all followers have passed the highWatermark.
   
   I don't think this is a good idea since it makes the logic opaque but will not necessarily really optimize any performance, so I just mention it here and feel free to ignore it 🙈.
   
   




-- 
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.

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