New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
raftstore: gc abnormal snapshots and destroy peer if failed to apply snapshots. #16992
base: master
Are you sure you want to change the base?
raftstore: gc abnormal snapshots and destroy peer if failed to apply snapshots. #16992
Conversation
…a snapshot. Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
[REVIEW NOTIFICATION] This pull request has not been approved. To complete the pull request process, please ask the reviewers in the list to review by filling The full list of commands accepted by this bot can be found here. Reviewer can indicate their review by submitting an approval review. |
Skipping CI for Draft Pull Request. |
Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
I have 2 questions about this PR:
|
|
By the way, as for the point 2, I agree what u mentioned before. But for safety, this pr takes current implementation.
I'll have a try for the |
I don't think we can directly mark it as
There are two ways to fix the panic:
|
Why need this extra RPC? At the leader side, it will switch the peer's state to normal after finishing send the snapshot. At the follower side, when apply snapshot failed, it is also doable to restore the raft state to its previous state before this snapshot. Thus, If I understand correctly, the leader should start another snapshot without any extra operation? |
Do you mean persisting the previous state so it can be restored even after restarting TiKV? That's doable (without introducing a new RPC), but it does add extra complexity to raftstore (and we'll need to review every code path related to snapshot handling). |
Yes. But I think just update the peer's state to its previous status should be simpler than tombstone or let pd remove this peer and later add it back. |
Correct me if I'm wrong, but I think that removing the peer and making PD add one fresh peer sounds valid ? As the peer, which panics due to loading the abnormal snapshot, has already advanced its |
You are right, the raft state is reset to the snapshot's state when receives the snapshot msg, so it not easy to revert to the previous state. And makes the raft state revert may cause other side-effects. Thus, if it's doable, I prefer the current idea to just destroy the current peer and let newer raft message to trigger create it automatically. |
@LykxSassinator from tikv/components/raftstore/src/store/fsm/store.rs Lines 2114 to 2130 in 06eed73
It seems after tombstoning the peer, the region will drop all following raft messages, thus the peer won't be recovered automatically. |
Yes, thank you for the reminder. It seems that we may need to utilize |
Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
/cc @glorv @overvenus PTAL, now this pr is ready. |
status.swap(JOB_STATUS_FAILED, Ordering::SeqCst); | ||
SNAP_COUNTER.apply.fail.inc(); | ||
// As the snapshot failed, it should be cleared and the related peer should be |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why not handle it when JOB_STATUS_FAILED
in check_applying_snap
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
IMO, check_applying_snap
should be responsible solely for checking and returning the latest applying status, which is updated by the RegionRunner
thread. The RegionRunner
thread is the first to obtain the applying status.
tombstone && self.fsm.peer.peer_id() == peer_id && !self.fsm.peer.is_leader(); | ||
if apply_snap_failed { | ||
// Send ConfChange to the leader to make the region tombstone the peer. | ||
self.fsm.peer.send_tombstone_peer_msg(self.ctx); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What if send_tombstone_peer_msg
failed or the message is dropped due to epoch change or leadership change. If current peer is not destroyed directly, we need some mechanism to mark that it is abnormal and should not be used for read/write anymore.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Make senses. The absence of messages can result in abnormal peer residue on this node.
Let me dive deeper in it.
@LykxSassinator: The following test failed, say
Full PR test history. Your PR dashboard. 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/test-infra repository. I understand the commands that are listed here. |
Out of curiosity and just trying to learn, what are the typical scenarios where applying snapshots may fail? In those cases, would it help if we retry a few times? If we destroy the peer and add a new one, it's possible that the new peer may hit the same issue again, right? |
Yep. One typical case is that loading snapshots encounters IO errors. In this case, the issue can be attributed to physical errors on the disk or system errors, causing the TiKV panic. |
What is changed and how it works?
Issue Number: Close #15292
What's Changed:
Previously, there were pending tasks to address the scenario where TiKV would panic if applying snapshots failed due to abnormal conditions such as IO errors or unexpected issues.
This pull request resolves the issue by introducing additional traits
tombstone: bool
toSnapshotApplied
, indicating whether the failure occurred due to abnormal snapshots.Additionally, the abnormal peer will send
ExtraMessageType::MsgGcPeerRequest
to related leader of this region, trigger a newConfChange
withRemoveNode
to gc the associated peer. Finally, this peer will be destroyed later to ensure the cluster will add one new peer by sending a fresh snapshot to the affected node.Related changes
pingcap/docs
/pingcap/docs-cn
:Check List
Tests
Side effects
Release note