You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
The leader's matchIndex is not monotonically increasing, which violates the requirements outlined in the Raft paper and results in node states that could lead to misconceptions.
#53
Open
liang636600 opened this issue
Jan 8, 2025
· 0 comments
Hi, @wenweihu86@loveheaven@guohao@wangwg1, I found that the leader's matchIndex is not monotonically increasing, which violates the requirements outlined in the Raft paper and leads to node states that could cause misunderstandings. I will explain the details below. How to reproduce the bug: Reproducing this bug requires only two nodes. Below, I will explain how to reproduce the bug in detail with reference to Figure 1. This is a 3-node cluster, but for simplicity, only Node 1 and Node 2 are shown in the diagram. The figure skips the leader election process, which has already completed, and Node 1 becomes the leader. Node 1 then sends the first AppendEntriesRequest (AEReq) to Node 2, but this message is delayed in transit.
Figure 1. Diagram illustrating how to reproduce the bug.
Subsequently, Node 1 receives a write request from the client, and let's assume the value being written is 5. Node 1 writes the value 5 to its log and persists it. Next, Node 1 sends the second AEReq to Node 2, which contains an entry with the value 5. Node 2 first receives the second AEReq, processes it, writes the value 5 to its log, persists it, and responds to Node 1.
When Node 1 receives the response from Node 2, it updates the matchIndex and nextIndex for the peer (Node 2). Although both matchIndex and nextIndex are problematic, for simplicity, I'll focus on explaining the issue with matchIndex. Node 1 sets the matchIndex(Node 2) to 1 and then calls the advanceCommitIndex function to update the commitIndex to 1.
Afterward, Node 2 receives the delayed first AEReq message from Node 1. Node 2 processes it and returns an AEResponse, part of which contains: [term: 1, lastLogIndex: 1, success: true]. When Node 1 processes this response, the code for updating the matchIndex is: peer.setMatchIndex(prevLogIndex + numEntries);(
). Here, both prevLogIndex and numEntries are 0, so Node 1 updates Node 2's matchIndex to 0.
However, Node 2's matchIndex was previously 1, which causes a decrease in the matchIndex value, leading to the issue.
Harmful impact: The first harmful effect is that the decrease in matchIndex violates the Raft paper's description of matchIndex, as shown in Figure 2. According to the Raft paper, matchIndex is supposed to be monotonically increasing. The second harmful effect is that it causes some node states to become highly confusing and incorrect. For example, in the earlier Figure 1, when matchIndex[2] = 0, Node 1's state is problematic. At this point, Node 1's matchIndex values are: matchIndex[1]: 1 (for clarity), matchIndex[2]: 0, and matchIndex[3]: 0. However, Node 1's commitIndex is 1, which makes this state invalid.
Figure 2. The description of matchIndex in the Raft paper.
Suggested fix: Fixing the bug is straightforward. Simply add a conditional check before peer.setMatchIndex. If prevLogIndex + numEntries > peer.getMatchIndex(), then update the matchIndex.
Hi, @wenweihu86 @loveheaven @guohao @wangwg1, I found that the leader's matchIndex is not monotonically increasing, which violates the requirements outlined in the Raft paper and leads to node states that could cause misunderstandings. I will explain the details below.
How to reproduce the bug: Reproducing this bug requires only two nodes. Below, I will explain how to reproduce the bug in detail with reference to Figure 1. This is a 3-node cluster, but for simplicity, only Node 1 and Node 2 are shown in the diagram. The figure skips the leader election process, which has already completed, and Node 1 becomes the leader. Node 1 then sends the first AppendEntriesRequest (AEReq) to Node 2, but this message is delayed in transit.
Figure 1. Diagram illustrating how to reproduce the bug.
Subsequently, Node 1 receives a write request from the client, and let's assume the value being written is 5. Node 1 writes the value 5 to its log and persists it. Next, Node 1 sends the second AEReq to Node 2, which contains an entry with the value 5. Node 2 first receives the second AEReq, processes it, writes the value 5 to its log, persists it, and responds to Node 1.
When Node 1 receives the response from Node 2, it updates the matchIndex and nextIndex for the peer (Node 2). Although both matchIndex and nextIndex are problematic, for simplicity, I'll focus on explaining the issue with matchIndex. Node 1 sets the matchIndex(Node 2) to 1 and then calls the advanceCommitIndex function to update the commitIndex to 1.
Afterward, Node 2 receives the delayed first AEReq message from Node 1. Node 2 processes it and returns an AEResponse, part of which contains: [term: 1, lastLogIndex: 1, success: true]. When Node 1 processes this response, the code for updating the matchIndex is:
peer.setMatchIndex(prevLogIndex + numEntries);
(raft-java/raft-java-core/src/main/java/com/github/wenweihu86/raft/RaftNode.java
Line 276 in 50761c6
However, Node 2's matchIndex was previously 1, which causes a decrease in the matchIndex value, leading to the issue.
Harmful impact: The first harmful effect is that the decrease in matchIndex violates the Raft paper's description of matchIndex, as shown in Figure 2. According to the Raft paper, matchIndex is supposed to be monotonically increasing. The second harmful effect is that it causes some node states to become highly confusing and incorrect. For example, in the earlier Figure 1, when matchIndex[2] = 0, Node 1's state is problematic. At this point, Node 1's matchIndex values are: matchIndex[1]: 1 (for clarity), matchIndex[2]: 0, and matchIndex[3]: 0. However, Node 1's commitIndex is 1, which makes this state invalid.
Figure 2. The description of matchIndex in the Raft paper.
Suggested fix: Fixing the bug is straightforward. Simply add a conditional check before
peer.setMatchIndex
. IfprevLogIndex + numEntries > peer.getMatchIndex()
, then update thematchIndex
.I'm looking forward to your confirmation, and would be happy to help fix the issue if needed.
The text was updated successfully, but these errors were encountered: