raftstore: fix reset follow apply_unpersisted_log_limit when demoting from leader#18241
raftstore: fix reset follow apply_unpersisted_log_limit when demoting from leader#18241ti-chi-bot[bot] merged 7 commits intotikv:masterfrom
Conversation
Signed-off-by: glorv <glorvs@163.com>
|
/hold waiting tikv/raft-rs#561. |
|
@glorv: GitHub didn't allow me to request PR reviews from the following users: hhwyt. Note that only tikv members and repo collaborators can review this PR, and authors cannot review their own PRs. DetailsIn response to this: Instructions for interacting with me using PR comments are available here. If you have questions or suggestions related to my behavior, please file an issue against the kubernetes-sigs/prow repository. |
Signed-off-by: glorv <glorvs@163.com>
Signed-off-by: glorv <glorvs@163.com>
| mem_size_change -= | ||
| (bytes_capacity(&e.data) + bytes_capacity(&e.context)) as i64; | ||
| } | ||
| // NOTE: if we are going to support apply unpersisted entries on follower, |
There was a problem hiding this comment.
It's better to move this line to the line before the "assert" line.
| must_get_equal(&cluster.get_engine(3), b"k150", b"v150"); | ||
| } | ||
|
|
||
| // Test the consistency of EntryCache when partioned leader constains |
There was a problem hiding this comment.
| // Test the consistency of EntryCache when partioned leader constains | |
| // Test the consistency of EntryCache when partitioned leader constains |
| } | ||
|
|
||
| // Test the consistency of EntryCache when partioned leader constains | ||
| // uncommitted propose, after the partion is recovered, it can replicate raft |
There was a problem hiding this comment.
| // uncommitted propose, after the partion is recovered, it can replicate raft | |
| // uncommitted propose, after the partition is recovered, it can replicate raft |
| let mut msg = dropped_append.lock().unwrap().clone(); | ||
| assert_eq!(msg.get_message().to, 1); | ||
| // Advance the committed index of the Append msg to trigger the corner case. | ||
| // I don't find an easy way to trigger this kind of msg, so direct modify the |
|
typo in the PR descrption: "on_role_change" -> "on_role_changed" |
Signed-off-by: glorv <glorvs@163.com>
| // Test the consistency of EntryCache when partitioned leader constains | ||
| // uncommitted propose, and after the partition is recovered, it can replicate | ||
| // raft entries for new leader correctly. This case test the corner scenario | ||
| // that partitioned leader receive a new Append msg for new elected leader and | ||
| // the new entries are already committed and overlap with existing entries | ||
| // uncomitted entries in the entry cache, it may cause panic if handles | ||
| // incorrectly. See issue https://github.com/tikv/tikv/issues/17868 for more details. |
There was a problem hiding this comment.
i'm just a spell-checking bot 🤖
| // Test the consistency of EntryCache when partitioned leader constains | |
| // uncommitted propose, and after the partition is recovered, it can replicate | |
| // raft entries for new leader correctly. This case test the corner scenario | |
| // that partitioned leader receive a new Append msg for new elected leader and | |
| // the new entries are already committed and overlap with existing entries | |
| // uncomitted entries in the entry cache, it may cause panic if handles | |
| // incorrectly. See issue https://github.com/tikv/tikv/issues/17868 for more details. | |
| // Test the consistency of EntryCache when partitioned leader contains | |
| // uncommitted propose, and after the partition is recovered, it can replicate | |
| // raft entries for new leader correctly. This case tests the corner scenario | |
| // that partitioned leader receives a new Append msg from new elected leader and | |
| // the new entries are already committed and overlap with existing | |
| // uncommitted entries in the entry cache, it may cause panic if handled | |
| // incorrectly. See issue https://github.com/tikv/tikv/issues/17868 for more details. |
| block_on_timeout(ch, Duration::from_millis(100)).unwrap_err(); | ||
|
|
||
| let fp = "pause_on_peer_collect_message"; | ||
| // pause peer 1 to wait for leader timeout, should new leader |
There was a problem hiding this comment.
| // pause peer 1 to wait for leader timeout, should new leader | |
| // pause peer 1 to wait for leader timeout |
overvenus
left a comment
There was a problem hiding this comment.
LGTM, wait for raft-rs change.
| let cached_last = cached.range.end - 1; | ||
| // TODO: if we are going to support apply unpersisted entries on | ||
| // follower, this assert will not longer be correct. | ||
| assert!(cached_last < trunc_to_idx); |
There was a problem hiding this comment.
| assert!(cached_last < trunc_to_idx); | |
| assert!(cached_last < trunc_to_idx, "cached_last: {}, trunc_to_idx: {}", cached_last, trunc_to_idx); |
Signed-off-by: glorv <glorvs@163.com>
|
/unhold |
|
@hbisheng @LykxSassinator PTAL again as tikv/raft-rs#561 is merged. |
[LGTM Timeline notifier]Timeline:
|
|
[APPROVALNOTIFIER] This PR is APPROVED This pull-request has been approved by: Connor1996, hbisheng, LykxSassinator The full list of commands accepted by this bot can be found here. The pull request process is described here DetailsNeeds approval from an approver in each of these files:
Approvers can indicate their approval by writing |
|
In response to a cherrypick label: new pull request created to branch |
|
In response to a cherrypick label: new pull request created to branch |
… from leader (#18241) (#18260) ref #16717, close #17868 This PR adopt the changes in tikv/raft-rs#561 that directly reset the apply_unpersisted_log_limit limit in raft when leader is demoted. It fix the bug that in the previous impl, the reset is done in "on_role_change" which is called in handle_raft_ready, in the round that leader is demoted, it still return unpersisted comitted entries as at the time, the limit is not reset. NOTE: once we want to support this feature on follower, we need to remove the original assert as it is incorrect in this case. Signed-off-by: glorv <glorvs@163.com> Co-authored-by: glorv <glorvs@163.com>
… from leader (#18241) (#18261) ref #16717, close #17868 This PR adopt the changes in tikv/raft-rs#561 that directly reset the apply_unpersisted_log_limit limit in raft when leader is demoted. It fix the bug that in the previous impl, the reset is done in "on_role_change" which is called in handle_raft_ready, in the round that leader is demoted, it still return unpersisted comitted entries as at the time, the limit is not reset. NOTE: once we want to support this feature on follower, we need to remove the original assert as it is incorrect in this case. Signed-off-by: glorv <glorvs@163.com> Co-authored-by: glorv <glorvs@163.com>
What is changed and how it works?
Issue Number: Close #17868, ref #16717
What's Changed:
NOTE: this PR depends on PR tikv/raft-rs#561
Related changes
pingcap/docs/pingcap/docs-cn:Check List
Tests
Side effects
Release note