[ 
https://issues.apache.org/jira/browse/RATIS-748?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Shashikant Banerjee reassigned RATIS-748:
-----------------------------------------

    Assignee: Lokesh Jain

> Follower might not update its commit index
> ------------------------------------------
>
>                 Key: RATIS-748
>                 URL: https://issues.apache.org/jira/browse/RATIS-748
>             Project: Ratis
>          Issue Type: Bug
>            Reporter: Lokesh Jain
>            Assignee: Lokesh Jain
>            Priority: Critical
>         Attachments: RATIS-748.001.patch
>
>
> While updating the commit index, the follower checks whether majority index 
> is present in the raft log. There can be cases where leader is ahead of the 
> follower and follower does not have the entry corresponding to the 
> majorityIndex. In such cases the follower commit index is not updated. Below 
> is the corresponding code snippet.
> {code:java}
> public boolean updateLastCommitted(long majorityIndex, long currentTerm) {
>   try(AutoCloseableLock writeLock = writeLock()) {
>     final long oldCommittedIndex = getLastCommittedIndex();
>     if (oldCommittedIndex < majorityIndex) {
>       // Only update last committed index for current term. See §5.4.2 in
>       // paper for details.
>       final TermIndex entry = getTermIndex(majorityIndex);
>       if (entry != null && entry.getTerm() == currentTerm) {
>         final long newCommitIndex = Math.min(majorityIndex, getFlushIndex());
>         if (newCommitIndex > oldCommittedIndex) {
>           commitIndex.updateIncreasingly(newCommitIndex, traceIndexChange);
>         }
>         return true;
>       }
>     }
>   }
>   return false;
> }{code}
> This function RaftLog#updateLastCommitted is also used by follower to update 
> its commit index. The follower does not require the check of entry.getTerm() 
> == currentTerm and its commitIndex can be updated to min(majorityIndex, 
> getFlushIndex()). It has already verified the entries in the 
> appendEntriesAsync call.
> This can lead to the follower commit being updated in bursts and can lead to 
> failure of watch requests.
> cc [~shashikant] [~szetszwo]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to