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-v2: support tracing peer lifetime #14056
Changes from all commits
868f1cd
ef3a2bb
e70ea42
68290d5
84f0957
7c0246c
b227f9f
a42a587
45bd91c
df107c5
a5b9ef9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -49,6 +49,12 @@ pub struct ConfChangeResult { | |
pub region_state: RegionLocalState, | ||
} | ||
|
||
#[derive(Debug)] | ||
pub struct UpdateGcPeersResult { | ||
index: u64, | ||
region_state: RegionLocalState, | ||
} | ||
|
||
impl<EK: KvEngine, ER: RaftEngine> Peer<EK, ER> { | ||
#[inline] | ||
pub fn propose_conf_change<T>( | ||
|
@@ -177,10 +183,13 @@ impl<EK: KvEngine, ER: RaftEngine> Peer<EK, ER> { | |
} | ||
} | ||
} | ||
if has_new_peer.is_some() { | ||
// Speed up snapshot instead of waiting another heartbeat. | ||
self.raft_group_mut().ping(); | ||
self.set_has_ready(); | ||
if self.is_leader() { | ||
if has_new_peer.is_some() { | ||
// Speed up snapshot instead of waiting another heartbeat. | ||
self.raft_group_mut().ping(); | ||
self.set_has_ready(); | ||
} | ||
self.maybe_schedule_gc_peer_tick(); | ||
} | ||
} | ||
ctx.coprocessor_host.on_region_changed( | ||
|
@@ -199,6 +208,15 @@ impl<EK: KvEngine, ER: RaftEngine> Peer<EK, ER> { | |
self.set_has_extra_write(); | ||
} | ||
} | ||
|
||
pub fn on_apply_res_update_gc_peers(&mut self, result: UpdateGcPeersResult) { | ||
let region_id = self.region_id(); | ||
self.state_changes_mut() | ||
.put_region_state(region_id, result.index, &result.region_state) | ||
.unwrap(); | ||
self.set_has_extra_write(); | ||
self.storage_mut().set_region_state(result.region_state); | ||
} | ||
} | ||
|
||
impl<EK: KvEngine, R> Apply<EK, R> { | ||
|
@@ -279,7 +297,28 @@ impl<EK: KvEngine, R> Apply<EK, R> { | |
); | ||
let my_id = self.peer().get_id(); | ||
let state = self.region_state_mut(); | ||
let mut removed_records: Vec<_> = state.take_removed_records().into(); | ||
for p0 in state.get_region().get_peers() { | ||
// No matching store ID means the peer must be removed. | ||
if new_region | ||
.get_peers() | ||
.iter() | ||
.all(|p1| p1.get_store_id() != p0.get_store_id()) | ||
{ | ||
removed_records.push(p0.clone()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Does it need dedup? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, it's impossible to be duplicated. |
||
} | ||
} | ||
// If a peer is replaced in the same store, the leader will keep polling the | ||
// new peer on the same store, which implies that the old peer must be | ||
// tombstone in the end. | ||
removed_records.retain(|p0| { | ||
new_region | ||
.get_peers() | ||
.iter() | ||
.all(|p1| p1.get_store_id() != p0.get_store_id()) | ||
}); | ||
state.set_region(new_region.clone()); | ||
state.set_removed_records(removed_records.into()); | ||
let new_peer = new_region | ||
.get_peers() | ||
.iter() | ||
|
@@ -534,4 +573,33 @@ impl<EK: KvEngine, R> Apply<EK, R> { | |
.inc(); | ||
Ok(()) | ||
} | ||
|
||
pub fn apply_update_gc_peer( | ||
&mut self, | ||
log_index: u64, | ||
admin_req: &AdminRequest, | ||
) -> (AdminResponse, AdminCmdResult) { | ||
let mut removed_records: Vec<_> = self.region_state_mut().take_removed_records().into(); | ||
let mut merged_records: Vec<_> = self.region_state_mut().take_merged_records().into(); | ||
let updates = admin_req.get_update_gc_peers().get_peer_id(); | ||
info!(self.logger, "update gc peer"; "index" => log_index, "updates" => ?updates, "gc_peers" => ?removed_records, "merged_peers" => ?merged_records); | ||
removed_records.retain(|p| !updates.contains(&p.get_id())); | ||
merged_records.retain_mut(|r| { | ||
let mut sources: Vec<_> = r.take_source_peers().into(); | ||
sources.retain(|p| !updates.contains(&p.get_id())); | ||
r.set_source_peers(sources.into()); | ||
!r.get_source_peers().is_empty() | ||
}); | ||
self.region_state_mut() | ||
.set_removed_records(removed_records.into()); | ||
self.region_state_mut() | ||
.set_merged_records(merged_records.into()); | ||
( | ||
AdminResponse::default(), | ||
AdminCmdResult::UpdateGcPeers(UpdateGcPeersResult { | ||
index: log_index, | ||
region_state: self.region_state().clone(), | ||
}), | ||
) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -7,7 +7,7 @@ mod transfer_leader; | |
|
||
pub use compact_log::CompactLogContext; | ||
use compact_log::CompactLogResult; | ||
use conf_change::ConfChangeResult; | ||
use conf_change::{ConfChangeResult, UpdateGcPeersResult}; | ||
use engine_traits::{KvEngine, RaftEngine}; | ||
use kvproto::raft_cmdpb::{AdminCmdType, RaftCmdRequest}; | ||
use protobuf::Message; | ||
|
@@ -28,6 +28,7 @@ pub enum AdminCmdResult { | |
ConfChange(ConfChangeResult), | ||
TransferLeader(u64), | ||
CompactLog(CompactLogResult), | ||
UpdateGcPeers(UpdateGcPeersResult), | ||
} | ||
|
||
impl<EK: KvEngine, ER: RaftEngine> Peer<EK, ER> { | ||
|
@@ -110,6 +111,10 @@ impl<EK: KvEngine, ER: RaftEngine> Peer<EK, ER> { | |
} | ||
} | ||
AdminCmdType::CompactLog => self.propose_compact_log(ctx, req), | ||
AdminCmdType::UpdateGcPeer => { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If I understand it correctly, we rely on this cmd to propagate the removed peer list. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The list is part of region local state and snapshot state. Every node should have the exact same peer list when they applies to the same index. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The protocol updates can be found in pingcap/kvproto#1040. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I see. So it's part of snapshot. Thanks. |
||
let data = req.write_to_bytes().unwrap(); | ||
self.propose(ctx, data) | ||
} | ||
_ => unimplemented!(), | ||
} | ||
}; | ||
|
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.
Where is this set? And FYI there's already a shutdown in
StoreSystem
.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.
It's the exact same bool flag. It's for query only.