-
Notifications
You must be signed in to change notification settings - Fork 2.1k
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). |
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. |
Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
@@ -244,6 +244,8 @@ procinfo = { git = "https://github.com/tikv/procinfo-rs", rev = "7693954bd1dd86e | |||
# After the PR to rust-rocksdb is merged, remember to comment this out and run `cargo update -p rocksdb`. | |||
# [patch.'https://github.com/tikv/rust-rocksdb'] | |||
# rocksdb = { git = "https://github.com/your_github_id/rust-rocksdb", branch = "your_branch" } | |||
[patch.'https://github.com/pingcap/kvproto'] | |||
kvproto = { git = "https://github.com/LykxSassinator/kvproto", branch = "tombstone_peer" } |
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.
Will be updated with latest if this pr is approved.
@@ -99,3 +99,7 @@ exceptions = [ | |||
unknown-git = "deny" | |||
unknown-registry = "deny" | |||
allow-org = { github = ["tikv", "pingcap", "rust-lang"] } | |||
# Allow list. | |||
allow-git = [ |
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.
Will be cleared when this pr is approved.
self.fsm.has_ready = true; | ||
// If failed on applying snapshot, it should tomebstone the peer. | ||
self.fsm.peer.should_tombstone = |
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 send msg here?
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.
To prevent the loss of messages in case of sending failures due to unexpected issues. Furthermore, in the event of message loss, this abnormal peer will be left in a dangled state and unable to process any messages from the leader.
The flag should_tombstone
is used to ensure that the abnormal peer continues to send the TombStone message to the leader until it is removed from the raft group.
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.
still can send it here
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.
FYI, the abnormal peer will stop sending messages once it is destroyed, which is triggered by receiving a RaftMessage
with the flag RaftMessage::is_tombstone
set to true.
Mark `unavailable == true` if the abnormal peer failed to apply snapshot and could not be normarlly removed by ConfChange. Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
// If failed on applying snapshot, send ConfChange to the leader to make the | ||
// region tombstone the peer. | ||
if self.fsm.peer.should_tombstone { | ||
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.
how to avoid sending too many msgs?
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.
After the abnormal peer is destoryed, it will not send message anymore.
self.fsm.has_ready = true; | ||
// If failed on applying snapshot, it should tomebstone the peer. | ||
self.fsm.peer.should_tombstone = |
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.
still can send it here
This reverts commit cad382d. Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
Signed-off-by: lucasliang <nkcs_lykx@hotmail.com>
This pr needs extra support from PD side to make the implementation more valid. Hold until tikv/pd#8266 is merged. |
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 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::MsgForcelyRemovePeerRequest
to related leader of this region, trigger a newConfChange
withRemoveNode
to remove 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