From 83d173a2c0058246631f0e71de74238ccff670fd Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Fri, 14 Jan 2022 14:17:42 +0800 Subject: [PATCH 01/16] raftstore: change PeerTicks to enum (#11849) * raftstore: change PeerTicks to enum close #11848 Signed-off-by: Yilin Chen * use bool array for registry Signed-off-by: Yilin Chen * rename PeerTicks to PeerTick Signed-off-by: Yilin Chen * add associated const for number of PeerTick types Signed-off-by: Yilin Chen * change naming Signed-off-by: Yilin Chen Co-authored-by: Ti Chi Robot --- components/raftstore/src/store/fsm/peer.rs | 61 ++++++++++--------- components/raftstore/src/store/fsm/store.rs | 25 ++++---- components/raftstore/src/store/mod.rs | 6 +- components/raftstore/src/store/msg.rs | 66 ++++++++++----------- 4 files changed, 78 insertions(+), 80 deletions(-) diff --git a/components/raftstore/src/store/fsm/peer.rs b/components/raftstore/src/store/fsm/peer.rs index 24a4fddc2c8..1875293b3c1 100644 --- a/components/raftstore/src/store/fsm/peer.rs +++ b/components/raftstore/src/store/fsm/peer.rs @@ -4,13 +4,13 @@ use std::borrow::Cow; use std::cell::Cell; use std::collections::Bound::{Excluded, Unbounded}; -use std::collections::{HashSet, VecDeque}; +use std::collections::VecDeque; use std::iter::Iterator; use std::time::Instant; use std::{cmp, mem, u64}; use batch_system::{BasicMailbox, Fsm}; -use collections::HashMap; +use collections::{HashMap, HashSet}; use engine_traits::{ Engines, KvEngine, RaftEngine, SSTMetaInfo, WriteBatch, WriteBatchExt, WriteOptions, }; @@ -72,7 +72,7 @@ use crate::store::worker::{ }; use crate::store::PdTask; use crate::store::{ - util, AbstractPeer, CasualMessage, Config, MergeResultKind, PeerMsg, PeerTicks, + util, AbstractPeer, CasualMessage, Config, MergeResultKind, PeerMsg, PeerTick, RaftCmdExtraOpts, RaftCommand, SignificantMsg, SnapKey, StoreMsg, }; use crate::{Error, Result}; @@ -97,7 +97,7 @@ where { pub peer: Peer, /// A registry for all scheduled ticks. This can avoid scheduling ticks twice accidentally. - tick_registry: PeerTicks, + tick_registry: [bool; PeerTick::VARIANT_COUNT], /// Ticks for speed up campaign in chaos state. /// /// Followers will keep ticking in Idle mode to measure how many ticks have been skipped. @@ -221,7 +221,7 @@ where tx, Box::new(PeerFsm { peer: Peer::new(store_id, cfg, sched, engines, region, meta_peer)?, - tick_registry: PeerTicks::empty(), + tick_registry: [false; PeerTick::VARIANT_COUNT], missing_ticks: 0, hibernate_state: HibernateState::ordered(), stopped: false, @@ -264,7 +264,7 @@ where tx, Box::new(PeerFsm { peer: Peer::new(store_id, cfg, sched, engines, ®ion, peer)?, - tick_registry: PeerTicks::empty(), + tick_registry: [false; PeerTick::VARIANT_COUNT], missing_ticks: 0, hibernate_state: HibernateState::ordered(), stopped: false, @@ -688,7 +688,7 @@ where } fn on_update_region_for_unsafe_recover(&mut self, region: Region) { - let mut new_peer_list = HashSet::new(); + let mut new_peer_list = HashSet::default(); for peer in region.get_peers() { new_peer_list.insert(peer.get_id()); } @@ -917,7 +917,7 @@ where } } - fn on_tick(&mut self, tick: PeerTicks) { + fn on_tick(&mut self, tick: PeerTick) { if self.fsm.stopped { return; } @@ -927,17 +927,16 @@ where "peer_id" => self.fsm.peer_id(), "region_id" => self.region_id(), ); - self.fsm.tick_registry.remove(tick); + self.fsm.tick_registry[tick as usize] = false; match tick { - PeerTicks::RAFT => self.on_raft_base_tick(), - PeerTicks::RAFT_LOG_GC => self.on_raft_gc_log_tick(false), - PeerTicks::PD_HEARTBEAT => self.on_pd_heartbeat_tick(), - PeerTicks::SPLIT_REGION_CHECK => self.on_split_region_check_tick(), - PeerTicks::CHECK_MERGE => self.on_check_merge(), - PeerTicks::CHECK_PEER_STALE_STATE => self.on_check_peer_stale_state_tick(), - PeerTicks::ENTRY_CACHE_EVICT => self.on_entry_cache_evict_tick(), - PeerTicks::CHECK_LEADER_LEASE => self.on_check_leader_lease_tick(), - _ => unreachable!(), + PeerTick::Raft => self.on_raft_base_tick(), + PeerTick::RaftLogGc => self.on_raft_gc_log_tick(false), + PeerTick::PdHeartbeat => self.on_pd_heartbeat_tick(), + PeerTick::SplitRegionCheck => self.on_split_region_check_tick(), + PeerTick::CheckMerge => self.on_check_merge(), + PeerTick::CheckPeerStaleState => self.on_check_peer_stale_state_tick(), + PeerTick::EntryCacheEvict => self.on_entry_cache_evict_tick(), + PeerTick::CheckLeaderLease => self.on_check_leader_lease_tick(), } } @@ -1281,11 +1280,11 @@ where } #[inline] - fn schedule_tick(&mut self, tick: PeerTicks) { - if self.fsm.tick_registry.contains(tick) { + fn schedule_tick(&mut self, tick: PeerTick) { + let idx = tick as usize; + if self.fsm.tick_registry[idx] { return; } - let idx = tick.bits() as usize; if is_zero_duration(&self.ctx.tick_batch[idx].wait_duration) { return; } @@ -1296,13 +1295,13 @@ where "region_id" => self.region_id(), "peer_id" => self.fsm.peer_id(), ); - self.fsm.tick_registry.insert(tick); + self.fsm.tick_registry[idx] = true; let region_id = self.region_id(); let mb = match self.ctx.router.mailbox(region_id) { Some(mb) => mb, None => { - self.fsm.tick_registry.remove(tick); + self.fsm.tick_registry[idx] = false; error!( "failed to get mailbox"; "region_id" => self.fsm.region_id(), @@ -1333,7 +1332,7 @@ where fn register_raft_base_tick(&mut self) { // If we register raft base tick failed, the whole raft can't run correctly, // TODO: shutdown the store? - self.schedule_tick(PeerTicks::RAFT) + self.schedule_tick(PeerTick::Raft) } fn on_raft_base_tick(&mut self) { @@ -2958,7 +2957,7 @@ where } fn register_merge_check_tick(&mut self) { - self.schedule_tick(PeerTicks::CHECK_MERGE) + self.schedule_tick(PeerTick::CheckMerge) } /// Check if merge target region is staler than the local one in kv engine. @@ -4031,7 +4030,7 @@ where } fn register_raft_gc_log_tick(&mut self) { - self.schedule_tick(PeerTicks::RAFT_LOG_GC) + self.schedule_tick(PeerTick::RaftLogGc) } #[allow(clippy::if_same_then_else)] @@ -4167,7 +4166,7 @@ where } fn register_entry_cache_evict_tick(&mut self) { - self.schedule_tick(PeerTicks::ENTRY_CACHE_EVICT) + self.schedule_tick(PeerTick::EntryCacheEvict) } fn on_entry_cache_evict_tick(&mut self) { @@ -4184,7 +4183,7 @@ where } fn register_check_leader_lease_tick(&mut self) { - self.schedule_tick(PeerTicks::CHECK_LEADER_LEASE) + self.schedule_tick(PeerTick::CheckLeaderLease) } fn on_check_leader_lease_tick(&mut self) { @@ -4197,7 +4196,7 @@ where } fn register_split_region_check_tick(&mut self) { - self.schedule_tick(PeerTicks::SPLIT_REGION_CHECK) + self.schedule_tick(PeerTick::SplitRegionCheck) } #[inline] @@ -4459,7 +4458,7 @@ where } fn register_pd_heartbeat_tick(&mut self) { - self.schedule_tick(PeerTicks::PD_HEARTBEAT) + self.schedule_tick(PeerTick::PdHeartbeat) } fn on_check_peer_stale_state_tick(&mut self) { @@ -4558,7 +4557,7 @@ where } fn register_check_peer_stale_state_tick(&mut self) { - self.schedule_tick(PeerTicks::CHECK_PEER_STALE_STATE) + self.schedule_tick(PeerTick::CheckPeerStaleState) } } diff --git a/components/raftstore/src/store/fsm/store.rs b/components/raftstore/src/store/fsm/store.rs index efa9c0b69c0..98b697703d5 100644 --- a/components/raftstore/src/store/fsm/store.rs +++ b/components/raftstore/src/store/fsm/store.rs @@ -79,7 +79,7 @@ use crate::store::worker::{ }; use crate::store::{ util, Callback, CasualMessage, GlobalReplicationState, InspectedRaftMessage, MergeResultKind, - PdTask, PeerMsg, PeerTicks, RaftCommand, SignificantMsg, SnapManager, StoreMsg, StoreTick, + PdTask, PeerMsg, PeerTick, RaftCommand, SignificantMsg, SnapManager, StoreMsg, StoreTick, }; use crate::Result; use concurrency_manager::ConcurrencyManager; @@ -421,21 +421,20 @@ where } pub fn update_ticks_timeout(&mut self) { - self.tick_batch[PeerTicks::RAFT.bits() as usize].wait_duration = - self.cfg.raft_base_tick_interval.0; - self.tick_batch[PeerTicks::RAFT_LOG_GC.bits() as usize].wait_duration = + self.tick_batch[PeerTick::Raft as usize].wait_duration = self.cfg.raft_base_tick_interval.0; + self.tick_batch[PeerTick::RaftLogGc as usize].wait_duration = self.cfg.raft_log_gc_tick_interval.0; - self.tick_batch[PeerTicks::ENTRY_CACHE_EVICT.bits() as usize].wait_duration = + self.tick_batch[PeerTick::EntryCacheEvict as usize].wait_duration = ENTRY_CACHE_EVICT_TICK_DURATION; - self.tick_batch[PeerTicks::PD_HEARTBEAT.bits() as usize].wait_duration = + self.tick_batch[PeerTick::PdHeartbeat as usize].wait_duration = self.cfg.pd_heartbeat_tick_interval.0; - self.tick_batch[PeerTicks::SPLIT_REGION_CHECK.bits() as usize].wait_duration = + self.tick_batch[PeerTick::SplitRegionCheck as usize].wait_duration = self.cfg.split_region_check_tick_interval.0; - self.tick_batch[PeerTicks::CHECK_PEER_STALE_STATE.bits() as usize].wait_duration = + self.tick_batch[PeerTick::CheckPeerStaleState as usize].wait_duration = self.cfg.peer_stale_state_check_interval.0; - self.tick_batch[PeerTicks::CHECK_MERGE.bits() as usize].wait_duration = + self.tick_batch[PeerTick::CheckMerge as usize].wait_duration = self.cfg.merge_check_tick_interval.0; - self.tick_batch[PeerTicks::CHECK_LEADER_LEASE.bits() as usize].wait_duration = + self.tick_batch[PeerTick::CheckLeaderLease as usize].wait_duration = self.cfg.check_leader_lease_interval.0; } } @@ -662,8 +661,8 @@ impl RaftPoller { } fn flush_ticks(&mut self) { - for t in PeerTicks::get_all_ticks() { - let idx = t.bits() as usize; + for t in PeerTick::get_all_ticks() { + let idx = *t as usize; if self.poll_ctx.tick_batch[idx].ticks.is_empty() { continue; } @@ -1165,7 +1164,7 @@ where .engines .kv .get_perf_context(self.cfg.value().perf_level, PerfContextKind::RaftstoreStore), - tick_batch: vec![PeerTickBatch::default(); 256], + tick_batch: vec![PeerTickBatch::default(); PeerTick::VARIANT_COUNT], node_start_time: Some(TiInstant::now_coarse()), feature_gate: self.feature_gate.clone(), self_disk_usage: DiskUsage::Normal, diff --git a/components/raftstore/src/store/mod.rs b/components/raftstore/src/store/mod.rs index 472171dccfb..65d94684cf9 100644 --- a/components/raftstore/src/store/mod.rs +++ b/components/raftstore/src/store/mod.rs @@ -35,9 +35,9 @@ pub use self::fsm::{DestroyPeerJob, RaftRouter, StoreInfo}; pub use self::hibernate_state::{GroupState, HibernateState}; pub use self::memory::*; pub use self::msg::{ - Callback, CasualMessage, ExtCallback, InspectedRaftMessage, MergeResultKind, PeerMsg, - PeerTicks, RaftCmdExtraOpts, RaftCommand, ReadCallback, ReadResponse, SignificantMsg, StoreMsg, - StoreTick, WriteCallback, WriteResponse, + Callback, CasualMessage, ExtCallback, InspectedRaftMessage, MergeResultKind, PeerMsg, PeerTick, + RaftCmdExtraOpts, RaftCommand, ReadCallback, ReadResponse, SignificantMsg, StoreMsg, StoreTick, + WriteCallback, WriteResponse, }; pub use self::peer::{ AbstractPeer, Peer, PeerStat, ProposalContext, RequestInspector, RequestPolicy, diff --git a/components/raftstore/src/store/msg.rs b/components/raftstore/src/store/msg.rs index 7005719c605..8d3db1ca37f 100644 --- a/components/raftstore/src/store/msg.rs +++ b/components/raftstore/src/store/msg.rs @@ -4,7 +4,6 @@ use std::borrow::Cow; use std::fmt; -use bitflags::bitflags; use engine_traits::{CompactedEvent, KvEngine, Snapshot}; use kvproto::kvrpcpb::ExtraOp as TxnExtraOp; use kvproto::metapb; @@ -179,45 +178,46 @@ where } } -bitflags! { - pub struct PeerTicks: u8 { - const RAFT = 0b00000001; - const RAFT_LOG_GC = 0b00000010; - const SPLIT_REGION_CHECK = 0b00000100; - const PD_HEARTBEAT = 0b00001000; - const CHECK_MERGE = 0b00010000; - const CHECK_PEER_STALE_STATE = 0b00100000; - const ENTRY_CACHE_EVICT = 0b01000000; - const CHECK_LEADER_LEASE = 0b10000000; - } +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[repr(u8)] +pub enum PeerTick { + Raft = 0, + RaftLogGc = 1, + SplitRegionCheck = 2, + PdHeartbeat = 3, + CheckMerge = 4, + CheckPeerStaleState = 5, + EntryCacheEvict = 6, + CheckLeaderLease = 7, } -impl PeerTicks { +impl PeerTick { + pub const VARIANT_COUNT: usize = Self::get_all_ticks().len(); + #[inline] pub fn tag(self) -> &'static str { match self { - PeerTicks::RAFT => "raft", - PeerTicks::RAFT_LOG_GC => "raft_log_gc", - PeerTicks::SPLIT_REGION_CHECK => "split_region_check", - PeerTicks::PD_HEARTBEAT => "pd_heartbeat", - PeerTicks::CHECK_MERGE => "check_merge", - PeerTicks::CHECK_PEER_STALE_STATE => "check_peer_stale_state", - PeerTicks::ENTRY_CACHE_EVICT => "entry_cache_evict", - PeerTicks::CHECK_LEADER_LEASE => "check_leader_lease", - _ => unreachable!(), + PeerTick::Raft => "raft", + PeerTick::RaftLogGc => "raft_log_gc", + PeerTick::SplitRegionCheck => "split_region_check", + PeerTick::PdHeartbeat => "pd_heartbeat", + PeerTick::CheckMerge => "check_merge", + PeerTick::CheckPeerStaleState => "check_peer_stale_state", + PeerTick::EntryCacheEvict => "entry_cache_evict", + PeerTick::CheckLeaderLease => "check_leader_lease", } } - pub fn get_all_ticks() -> &'static [PeerTicks] { - const TICKS: &[PeerTicks] = &[ - PeerTicks::RAFT, - PeerTicks::RAFT_LOG_GC, - PeerTicks::SPLIT_REGION_CHECK, - PeerTicks::PD_HEARTBEAT, - PeerTicks::CHECK_MERGE, - PeerTicks::CHECK_PEER_STALE_STATE, - PeerTicks::ENTRY_CACHE_EVICT, - PeerTicks::CHECK_LEADER_LEASE, + pub const fn get_all_ticks() -> &'static [PeerTick] { + const TICKS: &[PeerTick] = &[ + PeerTick::Raft, + PeerTick::RaftLogGc, + PeerTick::SplitRegionCheck, + PeerTick::PdHeartbeat, + PeerTick::CheckMerge, + PeerTick::CheckPeerStaleState, + PeerTick::EntryCacheEvict, + PeerTick::CheckLeaderLease, ]; TICKS } @@ -492,7 +492,7 @@ pub enum PeerMsg { RaftCommand(RaftCommand), /// Tick is periodical task. If target peer doesn't exist there is a potential /// that the raft node will not work anymore. - Tick(PeerTicks), + Tick(PeerTick), /// Result of applying committed entries. The message can't be lost. ApplyRes { res: ApplyTaskRes, From c778d6ec2224c43b2a57a35e497cd7c63ecb2b91 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Mon, 17 Jan 2022 16:21:44 +0800 Subject: [PATCH 02/16] *: check memory limit when inserting in-memory pessimistic locks (#11841) * *: check memory limit when inserting in-memory pessimistic locks ref #11452 Signed-off-by: Yilin Chen * fix merged upstream Signed-off-by: Yilin Chen * fix test Signed-off-by: Yilin Chen * fix typo Signed-off-by: Yilin Chen --- components/raftstore/src/store/fsm/peer.rs | 109 +----- components/raftstore/src/store/mod.rs | 2 +- components/raftstore/src/store/peer.rs | 2 +- components/raftstore/src/store/txn_ext.rs | 331 +++++++++++++++++- components/tikv_kv/src/lib.rs | 18 +- components/txn_types/src/lock.rs | 17 + src/storage/mod.rs | 6 +- src/storage/mvcc/reader/reader.rs | 2 +- src/storage/txn/scheduler.rs | 17 +- tests/failpoints/cases/test_split_region.rs | 4 +- tests/failpoints/cases/test_transaction.rs | 4 +- .../failpoints/cases/test_transfer_leader.rs | 46 ++- tests/integrations/raftstore/test_multi.rs | 6 +- .../raftstore/test_split_region.rs | 18 +- .../raftstore/test_transfer_leader.rs | 2 +- 15 files changed, 408 insertions(+), 176 deletions(-) diff --git a/components/raftstore/src/store/fsm/peer.rs b/components/raftstore/src/store/fsm/peer.rs index 1875293b3c1..0de30cffb5d 100644 --- a/components/raftstore/src/store/fsm/peer.rs +++ b/components/raftstore/src/store/fsm/peer.rs @@ -45,7 +45,7 @@ use tikv_util::time::{duration_to_sec, monotonic_raw_now, Instant as TiInstant}; use tikv_util::worker::{ScheduleError, Scheduler}; use tikv_util::{box_err, debug, defer, error, info, trace, warn}; use tikv_util::{escape, is_zero_duration, Either}; -use txn_types::{Key, PessimisticLock, WriteBatchFlags}; +use txn_types::WriteBatchFlags; use self::memtrace::*; use crate::coprocessor::RegionChangeEvent; @@ -2340,7 +2340,7 @@ where pessimistic_locks.is_valid = false; // 2. Propose pessimistic locks - if pessimistic_locks.map.is_empty() { + if pessimistic_locks.is_empty() { return false; } // FIXME: Raft command has size limit. Either limit the total size of pessimistic locks @@ -2350,7 +2350,7 @@ where // Downgrade to a read guard, do not block readers in the scheduler as far as possible. let pessimistic_locks = RwLockWriteGuard::downgrade(pessimistic_locks); fail_point!("invalidate_locks_before_transfer_leader"); - for (key, (lock, deleted)) in &pessimistic_locks.map { + for (key, (lock, deleted)) in &*pessimistic_locks { if *deleted { continue; } @@ -2765,7 +2765,7 @@ where // Update the version so the concurrent reader will fail due to EpochNotMatch // instead of PessimisticLockNotFound. pessimistic_locks.version = derived.get_region_epoch().get_version(); - group_locks_by_regions(&mut pessimistic_locks.map, ®ions, &derived) + pessimistic_locks.group_by_regions(®ions, &derived) }; fail_point!("on_split_invalidate_locks"); @@ -2908,7 +2908,7 @@ where if is_leader { new_peer.peer.approximate_size = estimated_size; new_peer.peer.approximate_keys = estimated_keys; - new_peer.peer.txn_ext.pessimistic_locks.write().map = locks; + *new_peer.peer.txn_ext.pessimistic_locks.write() = locks; // The new peer is likely to become leader, send a heartbeat immediately to reduce // client query miss. new_peer.peer.heartbeat_pd(self.ctx); @@ -4820,47 +4820,6 @@ fn new_compact_log_request( request } -/// Group pessimistic locks in the original region to the split regions. -/// -/// The given regions MUST be sorted by key in the ascending order. The returned -/// `HashMap`s are in the same order of the given regions. -/// -/// The locks belonging to the derived region will be kept in the given `locks` map, -/// and the corresponding position in the returned `Vec` will be an empty map. -fn group_locks_by_regions( - locks: &mut HashMap, - regions: &[metapb::Region], - derived: &metapb::Region, -) -> Vec> { - // Assert regions are sorted by key in ascending order. - if cfg!(debug_assertions) { - for (r1, r2) in regions.iter().zip(regions.iter().skip(1)) { - assert!(r1.get_start_key() < r2.get_start_key()); - } - } - - let mut res: Vec> = - regions.iter().map(|_| HashMap::default()).collect(); - // Locks that are marked deleted still need to be moved to the new regions, - // and the deleted mark should also be cleared. - // Refer to the comment in `PeerPessimisticLocks` for details. - let removed_locks = locks.drain_filter(|key, _| { - let key = &**key.as_encoded(); - let (start_key, end_key) = (derived.get_start_key(), derived.get_end_key()); - key < start_key || (!end_key.is_empty() && key >= end_key) - }); - for (key, (lock, _)) in removed_locks { - let idx = match regions - .binary_search_by_key(&&**key.as_encoded(), |region| region.get_start_key()) - { - Ok(idx) => idx, - Err(idx) => idx - 1, - }; - res[idx].insert(key, (lock, false)); - } - res -} - impl<'a, EK, ER, T: Transport> PeerFsmDelegate<'a, EK, ER, T> where EK: KvEngine, @@ -5106,62 +5065,4 @@ mod tests { assert!(flag.load(Ordering::Acquire)); } } - - #[test] - fn test_group_locks_by_regions() { - fn lock(key: &[u8], deleted: bool) -> (Key, (PessimisticLock, bool)) { - ( - Key::from_raw(key), - ( - PessimisticLock { - primary: key.to_vec().into_boxed_slice(), - start_ts: 10.into(), - ttl: 1000, - for_update_ts: 10.into(), - min_commit_ts: 20.into(), - }, - deleted, - ), - ) - } - fn region(start_key: &[u8], end_key: &[u8]) -> metapb::Region { - let mut region = metapb::Region::default(); - region.set_start_key(start_key.to_vec()); - region.set_end_key(end_key.to_vec()); - region - } - let mut locks: HashMap = vec![ - lock(b"a", true), - lock(b"c", false), - lock(b"e", true), - lock(b"g", false), - lock(b"i", false), - ] - .into_iter() - .collect(); - let regions = vec![ - region(b"", b"b"), // test leftmost region - region(b"b", b"c"), // no lock inside - region(b"c", b"d"), // test key equals to start_key - region(b"d", b"h"), // test multiple locks inside - region(b"h", b""), // test rightmost region - ]; - let output = group_locks_by_regions(&mut locks, ®ions, ®ions[4]); - let expected: Vec> = vec![ - vec![lock(b"a", false)], - vec![], - vec![lock(b"c", false)], - vec![lock(b"e", false), lock(b"g", false)], - vec![], // the position of the derived region is empty - ] - .into_iter() - .map(|locks| locks.into_iter().collect()) - .collect(); - assert_eq!(output, expected); - // The lock that belongs to the derived region is kept in the original map. - assert_eq!( - locks.into_iter().collect::>(), - vec![lock(b"i", false)] - ); - } } diff --git a/components/raftstore/src/store/mod.rs b/components/raftstore/src/store/mod.rs index 65d94684cf9..74476c0cc3e 100644 --- a/components/raftstore/src/store/mod.rs +++ b/components/raftstore/src/store/mod.rs @@ -57,7 +57,7 @@ pub use self::snap::{ Snapshot, SnapshotStatistics, }; pub use self::transport::{CasualRouter, ProposalRouter, StoreRouter, Transport}; -pub use self::txn_ext::{PeerPessimisticLocks, TxnExt}; +pub use self::txn_ext::{PeerPessimisticLocks, PessimisticLockPair, TxnExt}; pub use self::util::{RegionReadProgress, RegionReadProgressRegistry}; pub use self::worker::RefreshConfigTask; pub use self::worker::{ diff --git a/components/raftstore/src/store/peer.rs b/components/raftstore/src/store/peer.rs index 456f15ba23d..e1b0c16c485 100644 --- a/components/raftstore/src/store/peer.rs +++ b/components/raftstore/src/store/peer.rs @@ -4438,7 +4438,7 @@ where fn clear_in_memory_pessimistic_locks(&mut self) { let mut pessimistic_locks = self.txn_ext.pessimistic_locks.write(); pessimistic_locks.is_valid = false; // Not necessary, but just make it safer. - pessimistic_locks.map = Default::default(); + pessimistic_locks.clear(); pessimistic_locks.term = self.term(); pessimistic_locks.version = self.region().get_region_epoch().get_version(); } diff --git a/components/raftstore/src/store/txn_ext.rs b/components/raftstore/src/store/txn_ext.rs index fdab29ce843..e3430ff5c34 100644 --- a/components/raftstore/src/store/txn_ext.rs +++ b/components/raftstore/src/store/txn_ext.rs @@ -6,7 +6,10 @@ use std::{ }; use collections::HashMap; +use kvproto::metapb; +use lazy_static::lazy_static; use parking_lot::RwLock; +use prometheus::{register_int_gauge, IntGauge}; use txn_types::{Key, PessimisticLock}; /// Transaction extensions related to a peer. @@ -45,7 +48,21 @@ impl fmt::Debug for TxnExt { } } +lazy_static! { + pub static ref GLOBAL_MEM_SIZE: IntGauge = register_int_gauge!( + "tikv_pessimistic_lock_memory_size", + "Total memory size of pessimistic locks in bytes." + ) + .unwrap(); +} + +const GLOBAL_MEM_SIZE_LIMIT: usize = 100 << 20; // 100 MiB + +// 512 KiB, so pessimistic locks in one region can be proposed in a single command. +const PEER_MEM_SIZE_LIMIT: usize = 512 << 10; + /// Pessimistic locks of a region peer. +#[derive(PartialEq)] pub struct PeerPessimisticLocks { /// The table that stores pessimistic locks. /// @@ -87,7 +104,7 @@ pub struct PeerPessimisticLocks { /// Applying the write will be skipped because of version mismatch. So, no lock should /// be deleted. It's correct that we include the locks that are marked deleted in the /// commit merge request. - pub map: HashMap, + map: HashMap, /// Whether the pessimistic lock map is valid to read or write. If it is invalid, /// the in-memory pessimistic lock feature cannot be used at the moment. pub is_valid: bool, @@ -95,12 +112,15 @@ pub struct PeerPessimisticLocks { pub term: u64, /// Refers to the region version in which the pessimistic lock table is valid. pub version: u64, + /// Estimated memory used by the pessimistic locks. + pub memory_size: usize, } impl fmt::Debug for PeerPessimisticLocks { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("PeerPessimisticLocks") - .field("pessimistic_locks_size", &self.map.len()) + .field("count", &self.map.len()) + .field("memory_size", &self.memory_size) .field("is_valid", &self.is_valid) .field("term", &self.term) .field("version", &self.version) @@ -115,12 +135,315 @@ impl Default for PeerPessimisticLocks { is_valid: true, term: 0, version: 0, + memory_size: 0, } } } impl PeerPessimisticLocks { - pub fn insert(&mut self, key: Key, lock: PessimisticLock) { - self.map.insert(key, (lock, false)); + /// Inserts pessimistic locks into the map. + /// + /// Returns whether the operation succeeds. + pub fn insert(&mut self, pairs: Vec) -> bool { + let mut incr = 0; + // Pre-check the memory limit of pessimistic locks. + for pair in &pairs { + let (key, lock) = pair.as_pair(); + // If the key already exists in the map, it's an overwrite. + // The primary lock does not change during an overwrite, so we don't need to update + // the memory size. + if !self.map.contains_key(key) { + incr += key.len() + lock.memory_size(); + } + } + if self.memory_size + incr > PEER_MEM_SIZE_LIMIT + || GLOBAL_MEM_SIZE.get() as usize + incr > GLOBAL_MEM_SIZE_LIMIT + { + return false; + } + // Insert after check has passed. + for pair in pairs { + let (key, lock) = pair.into_pair(); + self.map.insert(key, (lock, false)); + } + self.memory_size += incr; + GLOBAL_MEM_SIZE.add(incr as i64); + true + } + + pub fn remove(&mut self, key: &Key) { + if let Some((lock, _)) = self.map.remove(key) { + let desc = key.len() + lock.memory_size(); + self.memory_size -= desc; + GLOBAL_MEM_SIZE.sub(desc as i64); + } + } + + pub fn clear(&mut self) { + self.map = HashMap::default(); + GLOBAL_MEM_SIZE.sub(self.memory_size as i64); + self.memory_size = 0; + } + + pub fn is_empty(&self) -> bool { + self.map.is_empty() + } + + pub fn get(&self, key: &Key) -> Option<&(PessimisticLock, bool)> { + self.map.get(key) + } + + pub fn get_mut(&mut self, key: &Key) -> Option<&mut (PessimisticLock, bool)> { + self.map.get_mut(key) + } + + /// Group pessimistic locks in the original region to the split regions. + /// + /// The given regions MUST be sorted by key in the ascending order. The returned + /// `HashMap`s are in the same order of the given regions. + /// + /// The locks belonging to the derived region will be kept in the given `locks` map, + /// and the corresponding position in the returned `Vec` will be an empty map. + pub fn group_by_regions( + &mut self, + regions: &[metapb::Region], + derived: &metapb::Region, + ) -> Vec { + // Assert regions are sorted by key in ascending order. + if cfg!(debug_assertions) { + for (r1, r2) in regions.iter().zip(regions.iter().skip(1)) { + assert!(r1.get_start_key() < r2.get_start_key()); + } + } + + let mut res: Vec = regions + .iter() + .map(|_| PeerPessimisticLocks::default()) + .collect(); + // Locks that are marked deleted still need to be moved to the new regions, + // and the deleted mark should also be cleared. + // Refer to the comment in `PeerPessimisticLocks` for details. + let removed_locks = self.map.drain_filter(|key, _| { + let key = &**key.as_encoded(); + let (start_key, end_key) = (derived.get_start_key(), derived.get_end_key()); + key < start_key || (!end_key.is_empty() && key >= end_key) + }); + for (key, (lock, _)) in removed_locks { + let idx = match regions + .binary_search_by_key(&&**key.as_encoded(), |region| region.get_start_key()) + { + Ok(idx) => idx, + Err(idx) => idx - 1, + }; + let size = key.len() + lock.memory_size(); + self.memory_size -= size; + res[idx].map.insert(key, (lock, false)); + res[idx].memory_size += size; + } + res + } + + #[cfg(test)] + fn from_locks(locks: impl IntoIterator) -> Self { + let mut res = PeerPessimisticLocks::default(); + for (key, (locks, is_deleted)) in locks { + res.memory_size += key.len() + locks.memory_size(); + res.map.insert(key, (locks, is_deleted)); + } + res + } +} + +impl<'a> IntoIterator for &'a PeerPessimisticLocks { + type Item = (&'a Key, &'a (PessimisticLock, bool)); + type IntoIter = std::collections::hash_map::Iter<'a, Key, (PessimisticLock, bool)>; + + fn into_iter(self) -> Self::IntoIter { + self.map.iter() + } +} + +impl Drop for PeerPessimisticLocks { + fn drop(&mut self) { + GLOBAL_MEM_SIZE.sub(self.memory_size as i64); + } +} + +pub trait PessimisticLockPair { + fn as_pair(&self) -> (&Key, &PessimisticLock); + + fn into_pair(self) -> (Key, PessimisticLock); +} + +impl PessimisticLockPair for (Key, PessimisticLock) { + fn as_pair(&self) -> (&Key, &PessimisticLock) { + (&self.0, &self.1) + } + + fn into_pair(self) -> (Key, PessimisticLock) { + self + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Mutex; + use tikv_util::defer; + + lazy_static! { + static ref TEST_MUTEX: Mutex<()> = Mutex::new(()); + } + + fn lock(primary: &[u8]) -> PessimisticLock { + PessimisticLock { + primary: primary.to_vec().into_boxed_slice(), + start_ts: 100.into(), + ttl: 3000, + for_update_ts: 100.into(), + min_commit_ts: Default::default(), + } + } + + #[test] + fn test_memory_size() { + let _guard = TEST_MUTEX.lock().unwrap(); + + let mut locks1 = PeerPessimisticLocks::default(); + let mut locks2 = PeerPessimisticLocks::default(); + let k1 = Key::from_raw(b"k1"); + let k2 = Key::from_raw(b"k22"); + let k3 = Key::from_raw(b"k333"); + + // Test the memory size of peer pessimistic locks after inserting. + locks1.insert(vec![(k1.clone(), lock(b"k1"))]); + assert_eq!(locks1.get(&k1), Some(&(lock(b"k1"), false))); + assert_eq!(locks1.memory_size, k1.len() + lock(b"k1").memory_size()); + locks1.insert(vec![(k2.clone(), lock(b"k1"))]); + assert_eq!(locks1.get(&k2), Some(&(lock(b"k1"), false))); + assert_eq!( + locks1.memory_size, + k1.len() + k2.len() + 2 * lock(b"k1").memory_size() + ); + + // Test the global memory size after inserting. + locks2.insert(vec![(k3.clone(), lock(b"k1"))]); + assert_eq!(locks2.get(&k3), Some(&(lock(b"k1"), false))); + assert_eq!( + GLOBAL_MEM_SIZE.get() as usize, + locks1.memory_size + locks2.memory_size + ); + + // Test the memory size after replacing, it should not change. + locks1.insert(vec![(k2.clone(), lock(b"k2"))]); + assert_eq!(locks1.get(&k2), Some(&(lock(b"k2"), false))); + assert_eq!( + locks1.memory_size, + k1.len() + k2.len() + 2 * lock(b"k1").memory_size() + ); + assert_eq!( + GLOBAL_MEM_SIZE.get() as usize, + locks1.memory_size + locks2.memory_size + ); + + // Test the memory size after removing. + locks1.remove(&k1); + assert!(locks1.get(&k1).is_none()); + assert_eq!(locks1.memory_size, k2.len() + lock(b"k2").memory_size()); + assert_eq!( + GLOBAL_MEM_SIZE.get() as usize, + locks1.memory_size + locks2.memory_size + ); + + // Test the memory size after clearing. + locks2.clear(); + assert!(locks2.is_empty()); + assert_eq!(locks2.memory_size, 0); + assert_eq!(GLOBAL_MEM_SIZE.get() as usize, locks1.memory_size); + + // Test the global memory size after dropping. + drop(locks1); + drop(locks2); + assert_eq!(GLOBAL_MEM_SIZE.get(), 0); + } + + #[test] + fn test_insert_checking_memory_limit() { + let _guard = TEST_MUTEX.lock().unwrap(); + defer!(GLOBAL_MEM_SIZE.set(0)); + + let mut locks = PeerPessimisticLocks::default(); + let success = locks.insert(vec![(Key::from_raw(b"k1"), lock(&[0; 512000]))]); + assert!(success); + + // Exceeding the region limit + let success = locks.insert(vec![(Key::from_raw(b"k2"), lock(&[0; 32000]))]); + assert!(!success); + assert!(locks.get(&Key::from_raw(b"k2")).is_none()); + + // Not exceeding the region limit, but exceeding the global limit + GLOBAL_MEM_SIZE.set(101 << 20); + let success = locks.insert(vec![(Key::from_raw(b"k2"), lock(b"abc"))]); + assert!(!success); + assert!(locks.get(&Key::from_raw(b"k2")).is_none()); + } + + #[test] + fn test_group_locks_by_regions() { + fn lock(key: &[u8], deleted: bool) -> (Key, (PessimisticLock, bool)) { + ( + Key::from_raw(key), + ( + PessimisticLock { + primary: key.to_vec().into_boxed_slice(), + start_ts: 10.into(), + ttl: 1000, + for_update_ts: 10.into(), + min_commit_ts: 20.into(), + }, + deleted, + ), + ) + } + fn region(start_key: &[u8], end_key: &[u8]) -> metapb::Region { + let mut region = metapb::Region::default(); + region.set_start_key(start_key.to_vec()); + region.set_end_key(end_key.to_vec()); + region + } + let _guard = TEST_MUTEX.lock().unwrap(); + defer!(GLOBAL_MEM_SIZE.set(0)); + + let mut original = PeerPessimisticLocks::from_locks(vec![ + lock(b"a", true), + lock(b"c", false), + lock(b"e", true), + lock(b"g", false), + lock(b"i", false), + ]); + let regions = vec![ + region(b"", b"b"), // test leftmost region + region(b"b", b"c"), // no lock inside + region(b"c", b"d"), // test key equals to start_key + region(b"d", b"h"), // test multiple locks inside + region(b"h", b""), // test rightmost region + ]; + let output = original.group_by_regions(®ions, ®ions[4]); + let expected: Vec<_> = vec![ + vec![lock(b"a", false)], + vec![], + vec![lock(b"c", false)], + vec![lock(b"e", false), lock(b"g", false)], + vec![], // the position of the derived region is empty + ] + .into_iter() + .map(PeerPessimisticLocks::from_locks) + .collect(); + assert_eq!(output, expected); + // The lock that belongs to the derived region is kept in the original map. + assert_eq!( + original, + PeerPessimisticLocks::from_locks(vec![lock(b"i", false)]) + ); } } diff --git a/components/tikv_kv/src/lib.rs b/components/tikv_kv/src/lib.rs index 6d71eea7f34..6ebb45f7b22 100644 --- a/components/tikv_kv/src/lib.rs +++ b/components/tikv_kv/src/lib.rs @@ -39,7 +39,7 @@ use engine_traits::{ use futures::prelude::*; use kvproto::errorpb::Error as ErrorHeader; use kvproto::kvrpcpb::{Context, DiskFullOpt, ExtraOp as TxnExtraOp, KeyRange}; -use raftstore::store::TxnExt; +use raftstore::store::{PessimisticLockPair, TxnExt}; use thiserror::Error; use tikv_util::{deadline::Deadline, escape}; use txn_types::{Key, PessimisticLock, TimeStamp, TxnExtra, Value}; @@ -92,6 +92,22 @@ impl Modify { } } +impl PessimisticLockPair for Modify { + fn as_pair(&self) -> (&Key, &PessimisticLock) { + match self { + Modify::PessimisticLock(k, lock) => (k, lock), + _ => panic!("not a pessimistic lock"), + } + } + + fn into_pair(self) -> (Key, PessimisticLock) { + match self { + Modify::PessimisticLock(k, lock) => (k, lock), + _ => panic!("not a pessimistic lock"), + } + } +} + #[derive(Default)] pub struct WriteData { pub modifies: Vec, diff --git a/components/txn_types/src/lock.rs b/components/txn_types/src/lock.rs index 78a1a647260..d4955f3d229 100644 --- a/components/txn_types/src/lock.rs +++ b/components/txn_types/src/lock.rs @@ -409,6 +409,10 @@ impl PessimisticLock { self.min_commit_ts, ) } + + pub fn memory_size(&self) -> usize { + self.primary.len() + size_of::() + } } impl std::fmt::Debug for PessimisticLock { @@ -847,4 +851,17 @@ mod tests { for_update_ts: TimeStamp(10), min_commit_ts: TimeStamp(20) }" ); } + + #[test] + fn test_pessimistic_lock_memory_size() { + let lock = PessimisticLock { + primary: b"primary".to_vec().into_boxed_slice(), + start_ts: 5.into(), + ttl: 1000, + for_update_ts: 10.into(), + min_commit_ts: 20.into(), + }; + // 7 bytes for primary key, 16 bytes for Box<[u8]>, and 4 8-byte integers. + assert_eq!(lock.memory_size(), 7 + 16 + 4 * 8); + } } diff --git a/src/storage/mod.rs b/src/storage/mod.rs index e2f6cf6e1fa..34a06b9b24a 100644 --- a/src/storage/mod.rs +++ b/src/storage/mod.rs @@ -8171,7 +8171,7 @@ mod tests { { let pessimistic_locks = txn_ext.pessimistic_locks.read(); - let lock = pessimistic_locks.map.get(&k1).unwrap(); + let lock = pessimistic_locks.get(&k1).unwrap(); assert_eq!( lock, &( @@ -8232,7 +8232,7 @@ mod tests { // After prewrite, the memory lock should be removed. { let pessimistic_locks = txn_ext.pessimistic_locks.read(); - assert!(!pessimistic_locks.map.contains_key(&k1)); + assert!(pessimistic_locks.get(&k1).is_none()); } } @@ -8261,7 +8261,7 @@ mod tests { .unwrap(); rx.recv().unwrap(); // When disabling in-memory pessimistic lock, the lock map should remain unchanged. - assert!(txn_ext.pessimistic_locks.read().map.is_empty()); + assert!(txn_ext.pessimistic_locks.read().is_empty()); let (tx, rx) = channel(); storage diff --git a/src/storage/mvcc/reader/reader.rs b/src/storage/mvcc/reader/reader.rs index f3fca08f439..966aa20896c 100644 --- a/src/storage/mvcc/reader/reader.rs +++ b/src/storage/mvcc/reader/reader.rs @@ -241,7 +241,7 @@ impl MvccReader { return Some(Err(KvError::from(err).into())); } - locks.map.get(key).map(|(lock, _)| { + locks.get(key).map(|(lock, _)| { // For write commands that are executed in serial, it should be impossible // to read a deleted lock. // For read commands in the scheduler, it should read the lock marked deleted diff --git a/src/storage/txn/scheduler.rs b/src/storage/txn/scheduler.rs index 6ae9c4a40d5..873a3cc033e 100644 --- a/src/storage/txn/scheduler.rs +++ b/src/storage/txn/scheduler.rs @@ -925,14 +925,14 @@ impl Scheduler { let removed_pessimistic_locks = match pessimistic_locks_guard.as_mut() { Some(locks) // If there is a leader or region change, removing the locks is unnecessary. - if locks.term == term && locks.version == version && !locks.map.is_empty() => + if locks.term == term && locks.version == version && !locks.is_empty() => { to_be_write .modifies .iter() .filter_map(|write| match write { Modify::Put(cf, key, ..) | Modify::Delete(cf, key) if *cf == CF_LOCK => { - locks.map.get_mut(key).map(|(_, deleted)| { + locks.get_mut(key).map(|(_, deleted)| { *deleted = true; key.to_owned() }) @@ -968,7 +968,7 @@ impl Scheduler { // so we needn't remove the key. if pessimistic_locks.term == term && pessimistic_locks.version == version { for key in removed_pessimistic_locks { - pessimistic_locks.map.remove(&key); + pessimistic_locks.remove(&key); } } } @@ -1039,16 +1039,7 @@ impl Scheduler { { return false; } - // TODO: add memory limit check - for modify in mem::take(&mut to_be_write.modifies) { - match modify { - Modify::PessimisticLock(key, lock) => { - pessimistic_locks.insert(key, lock); - } - _ => panic!("all modifies should be PessimisticLock"), - } - } - true + pessimistic_locks.insert(mem::take(&mut to_be_write.modifies)) } /// If the task has expired, return `true` and call the callback of diff --git a/tests/failpoints/cases/test_split_region.rs b/tests/failpoints/cases/test_split_region.rs index beb273af3a4..ce7b47bcf0a 100644 --- a/tests/failpoints/cases/test_split_region.rs +++ b/tests/failpoints/cases/test_split_region.rs @@ -869,8 +869,8 @@ fn test_split_pessimistic_locks_with_concurrent_prewrite() { }; { let mut locks = txn_ext.pessimistic_locks.write(); - locks.insert(Key::from_raw(b"a"), lock_a); - locks.insert(Key::from_raw(b"c"), lock_c); + locks.insert(vec![(Key::from_raw(b"a"), lock_a)]); + locks.insert(vec![(Key::from_raw(b"c"), lock_c)]); } let mut mutation = Mutation::default(); diff --git a/tests/failpoints/cases/test_transaction.rs b/tests/failpoints/cases/test_transaction.rs index baf4a0ca4e6..996a3d6b6f6 100644 --- a/tests/failpoints/cases/test_transaction.rs +++ b/tests/failpoints/cases/test_transaction.rs @@ -531,7 +531,7 @@ fn test_pessimistic_lock_check_valid() { // There should be no region error. assert!(!resp.has_region_error()); // The lock should not be written to the in-memory pessimistic lock table. - assert!(txn_ext.pessimistic_locks.read().map.is_empty()); + assert!(txn_ext.pessimistic_locks.read().is_empty()); } #[test] @@ -559,7 +559,7 @@ fn test_concurrent_write_after_transfer_leader_invalidates_locks() { txn_ext .pessimistic_locks .write() - .insert(Key::from_raw(b"key"), lock.clone()); + .insert(vec![(Key::from_raw(b"key"), lock.clone())]); let region = cluster.get_region(b""); let leader = region.get_peers()[0].clone(); diff --git a/tests/failpoints/cases/test_transfer_leader.rs b/tests/failpoints/cases/test_transfer_leader.rs index 8e2d418837f..11afeee874f 100644 --- a/tests/failpoints/cases/test_transfer_leader.rs +++ b/tests/failpoints/cases/test_transfer_leader.rs @@ -119,7 +119,7 @@ fn test_delete_lock_proposed_after_proposing_locks_impl(transfer_msg_count: usiz let snapshot = cluster.must_get_snapshot_of_region(region_id); let txn_ext = snapshot.txn_ext.unwrap(); - txn_ext.pessimistic_locks.write().insert( + txn_ext.pessimistic_locks.write().insert(vec![( Key::from_raw(b"key"), PessimisticLock { primary: b"key".to_vec().into_boxed_slice(), @@ -128,7 +128,7 @@ fn test_delete_lock_proposed_after_proposing_locks_impl(transfer_msg_count: usiz for_update_ts: 10.into(), min_commit_ts: 20.into(), }, - ); + )]); let addr = cluster.sim.rl().get_addr(1); let env = Arc::new(Environment::new(1)); @@ -192,19 +192,16 @@ fn test_delete_lock_proposed_before_proposing_locks() { let snapshot = cluster.must_get_snapshot_of_region(region_id); let txn_ext = snapshot.txn_ext.unwrap(); - txn_ext.pessimistic_locks.write().map.insert( + txn_ext.pessimistic_locks.write().insert(vec![( Key::from_raw(b"key"), - ( - PessimisticLock { - primary: b"key".to_vec().into_boxed_slice(), - start_ts: 10.into(), - ttl: 1000, - for_update_ts: 10.into(), - min_commit_ts: 20.into(), - }, - false, - ), - ); + PessimisticLock { + primary: b"key".to_vec().into_boxed_slice(), + start_ts: 10.into(), + ttl: 1000, + for_update_ts: 10.into(), + min_commit_ts: 20.into(), + }, + )]); let addr = cluster.sim.rl().get_addr(1); let env = Arc::new(Environment::new(1)); @@ -273,19 +270,16 @@ fn test_read_lock_after_become_follower() { let snapshot = cluster.must_get_snapshot_of_region(region_id); let txn_ext = snapshot.txn_ext.unwrap(); let for_update_ts = block_on(cluster.pd_client.get_tso()).unwrap(); - txn_ext.pessimistic_locks.write().map.insert( + txn_ext.pessimistic_locks.write().insert(vec![( Key::from_raw(b"key"), - ( - PessimisticLock { - primary: b"key".to_vec().into_boxed_slice(), - start_ts, - ttl: 1000, - for_update_ts, - min_commit_ts: for_update_ts, - }, - false, - ), - ); + PessimisticLock { + primary: b"key".to_vec().into_boxed_slice(), + start_ts, + ttl: 1000, + for_update_ts, + min_commit_ts: for_update_ts, + }, + )]); let addr = cluster.sim.rl().get_addr(3); let env = Arc::new(Environment::new(1)); diff --git a/tests/integrations/raftstore/test_multi.rs b/tests/integrations/raftstore/test_multi.rs index 6cc149c8668..54fdda71140 100644 --- a/tests/integrations/raftstore/test_multi.rs +++ b/tests/integrations/raftstore/test_multi.rs @@ -839,7 +839,7 @@ fn test_leader_drop_with_pessimistic_lock() { .get_txn_ext() .unwrap() .clone(); - txn_ext.pessimistic_locks.write().insert( + txn_ext.pessimistic_locks.write().insert(vec![( Key::from_raw(b"k1"), PessimisticLock { primary: b"k1".to_vec().into_boxed_slice(), @@ -848,7 +848,7 @@ fn test_leader_drop_with_pessimistic_lock() { for_update_ts: 10.into(), min_commit_ts: 10.into(), }, - ); + )]); // Isolate node 1, leader should be transferred to another node. cluster.add_send_filter(IsolationFilterFactory::new(1)); @@ -858,5 +858,5 @@ fn test_leader_drop_with_pessimistic_lock() { // When peer 1 becomes leader again, the pessimistic locks should be cleared before. cluster.clear_send_filters(); cluster.must_transfer_leader(1, new_peer(1, 1)); - assert!(txn_ext.pessimistic_locks.write().map.is_empty()); + assert!(txn_ext.pessimistic_locks.read().is_empty()); } diff --git a/tests/integrations/raftstore/test_split_region.rs b/tests/integrations/raftstore/test_split_region.rs index 5bf097977be..c96cf5883fc 100644 --- a/tests/integrations/raftstore/test_split_region.rs +++ b/tests/integrations/raftstore/test_split_region.rs @@ -929,8 +929,8 @@ fn test_split_with_in_memory_pessimistic_locks() { }; { let mut locks = txn_ext.pessimistic_locks.write(); - locks.insert(Key::from_raw(b"a"), lock_a.clone()); - locks.insert(Key::from_raw(b"c"), lock_c.clone()); + locks.insert(vec![(Key::from_raw(b"a"), lock_a.clone())]); + locks.insert(vec![(Key::from_raw(b"c"), lock_c.clone())]); } let region = cluster.get_region(b""); @@ -945,13 +945,8 @@ fn test_split_with_in_memory_pessimistic_locks() { .get_txn_ext() .unwrap() .clone(); - assert_eq!(txn_ext.pessimistic_locks.read().map.len(), 1); assert_eq!( - txn_ext - .pessimistic_locks - .read() - .map - .get(&Key::from_raw(b"a")), + txn_ext.pessimistic_locks.read().get(&Key::from_raw(b"a")), Some(&(lock_a, false)) ); @@ -962,13 +957,8 @@ fn test_split_with_in_memory_pessimistic_locks() { .get_txn_ext() .unwrap() .clone(); - assert_eq!(txn_ext.pessimistic_locks.read().map.len(), 1); assert_eq!( - txn_ext - .pessimistic_locks - .read() - .map - .get(&Key::from_raw(b"c")), + txn_ext.pessimistic_locks.read().get(&Key::from_raw(b"c")), Some(&(lock_c, false)) ); } diff --git a/tests/integrations/raftstore/test_transfer_leader.rs b/tests/integrations/raftstore/test_transfer_leader.rs index ce81e5dcb33..19d37647b23 100644 --- a/tests/integrations/raftstore/test_transfer_leader.rs +++ b/tests/integrations/raftstore/test_transfer_leader.rs @@ -241,7 +241,7 @@ fn test_propose_in_memory_pessimistic_locks() { { let mut pessimistic_locks = txn_ext.pessimistic_locks.write(); assert!(pessimistic_locks.is_valid); - pessimistic_locks.insert(Key::from_raw(b"key"), lock.clone()); + pessimistic_locks.insert(vec![(Key::from_raw(b"key"), lock.clone())]); } cluster.must_transfer_leader(1, new_peer(2, 2)); From 6bc13b42119ef74d99f262dd401b474fa32b2154 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Mon, 17 Jan 2022 17:03:45 +0800 Subject: [PATCH 03/16] *: fix invalid failpoint caused by typo (#11709) * fix invalid failpoint caused by typo close #11734 Signed-off-by: Ryan Leung * address the comment Signed-off-by: Ryan Leung * address the comment Signed-off-by: Ryan Leung Co-authored-by: Ti Chi Robot --- components/raftstore/src/store/fsm/apply.rs | 3 +-- components/resolved_ts/src/advance.rs | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/components/raftstore/src/store/fsm/apply.rs b/components/raftstore/src/store/fsm/apply.rs index acc9b714ce8..53c292c8a1b 100644 --- a/components/raftstore/src/store/fsm/apply.rs +++ b/components/raftstore/src/store/fsm/apply.rs @@ -3451,7 +3451,7 @@ where } } - #[allow(unused_mut)] + #[allow(unused_mut, clippy::redundant_closure_call)] fn handle_snapshot>( &mut self, apply_ctx: &mut ApplyContext, @@ -3466,7 +3466,6 @@ where .iter() .any(|res| res.region_id == self.delegate.region_id()) && self.delegate.last_flush_applied_index != applied_index; - #[cfg(feature = "failpoint")] (|| fail_point!("apply_on_handle_snapshot_sync", |_| { need_sync = true }))(); if need_sync { if apply_ctx.timer.is_none() { diff --git a/components/resolved_ts/src/advance.rs b/components/resolved_ts/src/advance.rs index 4a62d1f6f19..e1a2830eeed 100644 --- a/components/resolved_ts/src/advance.rs +++ b/components/resolved_ts/src/advance.rs @@ -6,6 +6,7 @@ use std::time::Duration; use collections::{HashMap, HashSet}; use concurrency_manager::ConcurrencyManager; use engine_traits::KvEngine; +use fail::fail_point; use futures::compat::Future01CompatExt; use futures::future::select_all; use futures::FutureExt; @@ -158,8 +159,7 @@ pub async fn region_resolved_ts_store( tikv_clients: Arc>>, min_ts: TimeStamp, ) -> Vec { - #[cfg(feature = "failpoint")] - (|| fail_point!("before_sync_replica_read_state", |_| regions))(); + fail_point!("before_sync_replica_read_state", |_| regions.clone()); let store_id = match store_meta.lock().unwrap().store_id { Some(id) => id, From da1c069a2f9b5222ddbd32820a5a2ae1de50c78c Mon Sep 17 00:00:00 2001 From: Connor Date: Mon, 17 Jan 2022 19:59:45 +0800 Subject: [PATCH 04/16] raftstore: Extract significant router (#11750) * ref#11409 add significant router Signed-off-by: Connor1996 * make format Signed-off-by: Connor1996 * address comment Signed-off-by: Connor1996 --- components/raftstore/src/router.rs | 62 +++++++------------ components/raftstore/src/store/mod.rs | 4 +- components/raftstore/src/store/transport.rs | 35 ++++++++++- components/test_raftstore/src/router.rs | 16 +++-- .../test_raftstore/src/transport_simulate.rs | 12 ++-- src/server/server.rs | 14 +++-- tests/benches/misc/raftkv/mod.rs | 15 ++--- 7 files changed, 91 insertions(+), 67 deletions(-) diff --git a/components/raftstore/src/router.rs b/components/raftstore/src/router.rs index 40816fe1a16..eb5fd0a7297 100644 --- a/components/raftstore/src/router.rs +++ b/components/raftstore/src/router.rs @@ -3,16 +3,15 @@ // #[PerformanceCriticalPath] use std::cell::RefCell; -use crossbeam::channel::{SendError, TrySendError}; +use crossbeam::channel::TrySendError; use engine_traits::{KvEngine, RaftEngine, Snapshot}; use kvproto::raft_cmdpb::RaftCmdRequest; use kvproto::raft_serverpb::RaftMessage; use raft::SnapshotStatus; -use tikv_util::error; use tikv_util::time::ThreadReadId; use crate::store::fsm::RaftRouter; -use crate::store::transport::{CasualRouter, ProposalRouter, StoreRouter}; +use crate::store::transport::{CasualRouter, ProposalRouter, SignificantRouter, StoreRouter}; use crate::store::{ Callback, CasualMessage, LocalReader, PeerMsg, RaftCmdExtraOpts, RaftCommand, SignificantMsg, StoreMsg, @@ -21,20 +20,18 @@ use crate::{DiscardReason, Error as RaftStoreError, Result as RaftStoreResult}; /// Routes messages to the raftstore. pub trait RaftStoreRouter: - StoreRouter + ProposalRouter + CasualRouter + Send + Clone + StoreRouter + + ProposalRouter + + CasualRouter + + SignificantRouter + + Send + + Clone where EK: KvEngine, { /// Sends RaftMessage to local store. fn send_raft_msg(&self, msg: RaftMessage) -> RaftStoreResult<()>; - /// Sends a significant message. We should guarantee that the message can't be dropped. - fn significant_send( - &self, - region_id: u64, - msg: SignificantMsg, - ) -> RaftStoreResult<()>; - /// Broadcast a message generated by `msg_gen` to all Raft groups. fn broadcast_normal(&self, msg_gen: impl FnMut() -> PeerMsg); @@ -136,6 +133,12 @@ impl CasualRouter for RaftStoreBlackHole { } } +impl SignificantRouter for RaftStoreBlackHole { + fn significant_send(&self, _: u64, _: SignificantMsg) -> RaftStoreResult<()> { + Ok(()) + } +} + impl ProposalRouter for RaftStoreBlackHole { fn send(&self, _: RaftCommand) -> std::result::Result<(), TrySendError>> { Ok(()) @@ -160,11 +163,6 @@ where Ok(()) } - /// Sends a significant message. We should guarantee that the message can't be dropped. - fn significant_send(&self, _: u64, _: SignificantMsg) -> RaftStoreResult<()> { - Ok(()) - } - fn broadcast_normal(&self, _: impl FnMut() -> PeerMsg) {} } @@ -218,18 +216,19 @@ impl CasualRouter for ServerRaftStoreRouter RaftStoreRouter for ServerRaftStoreRouter { - fn send_raft_msg(&self, msg: RaftMessage) -> RaftStoreResult<()> { - RaftStoreRouter::send_raft_msg(&self.router, msg) - } - - /// Sends a significant message. We should guarantee that the message can't be dropped. +impl SignificantRouter for ServerRaftStoreRouter { fn significant_send( &self, region_id: u64, msg: SignificantMsg, ) -> RaftStoreResult<()> { - RaftStoreRouter::significant_send(&self.router, region_id, msg) + self.router.significant_send(region_id, msg) + } +} + +impl RaftStoreRouter for ServerRaftStoreRouter { + fn send_raft_msg(&self, msg: RaftMessage) -> RaftStoreResult<()> { + RaftStoreRouter::send_raft_msg(&self.router, msg) } fn broadcast_normal(&self, msg_gen: impl FnMut() -> PeerMsg) { @@ -270,23 +269,6 @@ impl RaftStoreRouter for RaftRouter { .map_err(|e| handle_send_error(region_id, e)) } - fn significant_send( - &self, - region_id: u64, - msg: SignificantMsg, - ) -> RaftStoreResult<()> { - if let Err(SendError(msg)) = self - .router - .force_send(region_id, PeerMsg::SignificantMsg(msg)) - { - // TODO: panic here once we can detect system is shutting down reliably. - error!("failed to send significant msg"; "msg" => ?msg); - return Err(RaftStoreError::RegionNotFound(region_id)); - } - - Ok(()) - } - fn broadcast_normal(&self, msg_gen: impl FnMut() -> PeerMsg) { batch_system::Router::broadcast_normal(self, msg_gen) } diff --git a/components/raftstore/src/store/mod.rs b/components/raftstore/src/store/mod.rs index 74476c0cc3e..f73446c0746 100644 --- a/components/raftstore/src/store/mod.rs +++ b/components/raftstore/src/store/mod.rs @@ -56,7 +56,9 @@ pub use self::snap::{ ApplyOptions, Error as SnapError, SnapEntry, SnapKey, SnapManager, SnapManagerBuilder, Snapshot, SnapshotStatistics, }; -pub use self::transport::{CasualRouter, ProposalRouter, StoreRouter, Transport}; +pub use self::transport::{ + CasualRouter, ProposalRouter, SignificantRouter, StoreRouter, Transport, +}; pub use self::txn_ext::{PeerPessimisticLocks, PessimisticLockPair, TxnExt}; pub use self::util::{RegionReadProgress, RegionReadProgressRegistry}; pub use self::worker::RefreshConfigTask; diff --git a/components/raftstore/src/store/transport.rs b/components/raftstore/src/store/transport.rs index 31dfcd35c4c..1d38b2f9638 100644 --- a/components/raftstore/src/store/transport.rs +++ b/components/raftstore/src/store/transport.rs @@ -1,12 +1,13 @@ // Copyright 2016 TiKV Project Authors. Licensed under Apache-2.0. // #[PerformanceCriticalPath] -use crate::store::{CasualMessage, PeerMsg, RaftCommand, RaftRouter, StoreMsg}; +use crate::store::{CasualMessage, PeerMsg, RaftCommand, RaftRouter, SignificantMsg, StoreMsg}; use crate::{DiscardReason, Error, Result}; -use crossbeam::channel::TrySendError; +use crossbeam::channel::{SendError, TrySendError}; use engine_traits::{KvEngine, RaftEngine, Snapshot}; use kvproto::raft_serverpb::RaftMessage; use std::sync::mpsc; +use tikv_util::error; /// Transports messages between different Raft peers. pub trait Transport: Send + Clone { @@ -27,6 +28,16 @@ where fn send(&self, region_id: u64, msg: CasualMessage) -> Result<()>; } +/// Routes message to target region. +/// +/// Messages aret guaranteed to be delivered by this trait. +pub trait SignificantRouter: Send +where + EK: KvEngine, +{ + fn significant_send(&self, region_id: u64, msg: SignificantMsg) -> Result<()>; +} + /// Routes proposal to target region. pub trait ProposalRouter where @@ -60,6 +71,26 @@ where } } +impl SignificantRouter for RaftRouter +where + EK: KvEngine, + ER: RaftEngine, +{ + #[inline] + fn significant_send(&self, region_id: u64, msg: SignificantMsg) -> Result<()> { + if let Err(SendError(msg)) = self + .router + .force_send(region_id, PeerMsg::SignificantMsg(msg)) + { + // TODO: panic here once we can detect system is shutting down reliably. + error!("failed to send significant msg"; "msg" => ?msg); + return Err(Error::RegionNotFound(region_id)); + } + + Ok(()) + } +} + impl ProposalRouter for RaftRouter where EK: KvEngine, diff --git a/components/test_raftstore/src/router.rs b/components/test_raftstore/src/router.rs index 8f8e5af92eb..a11a55981ef 100644 --- a/components/test_raftstore/src/router.rs +++ b/components/test_raftstore/src/router.rs @@ -9,7 +9,9 @@ use kvproto::raft_serverpb::RaftMessage; use raftstore::errors::{Error as RaftStoreError, Result as RaftStoreResult}; use raftstore::router::{handle_send_error, RaftStoreRouter}; use raftstore::store::msg::{CasualMessage, PeerMsg, SignificantMsg}; -use raftstore::store::{CasualRouter, ProposalRouter, RaftCommand, StoreMsg, StoreRouter}; +use raftstore::store::{ + CasualRouter, ProposalRouter, RaftCommand, SignificantRouter, StoreMsg, StoreRouter, +}; use tikv_util::mpsc::{loose_bounded, LooseBoundedSender, Receiver}; #[derive(Clone)] @@ -64,11 +66,7 @@ impl CasualRouter for MockRaftStoreRouter { } } -impl RaftStoreRouter for MockRaftStoreRouter { - fn send_raft_msg(&self, _: RaftMessage) -> RaftStoreResult<()> { - unimplemented!() - } - +impl SignificantRouter for MockRaftStoreRouter { fn significant_send( &self, region_id: u64, @@ -83,6 +81,12 @@ impl RaftStoreRouter for MockRaftStoreRouter { Err(RaftStoreError::RegionNotFound(region_id)) } } +} + +impl RaftStoreRouter for MockRaftStoreRouter { + fn send_raft_msg(&self, _: RaftMessage) -> RaftStoreResult<()> { + unimplemented!() + } fn broadcast_normal(&self, _: impl FnMut() -> PeerMsg) {} } diff --git a/components/test_raftstore/src/transport_simulate.rs b/components/test_raftstore/src/transport_simulate.rs index e0a9ea307c3..b7c95d75eda 100644 --- a/components/test_raftstore/src/transport_simulate.rs +++ b/components/test_raftstore/src/transport_simulate.rs @@ -16,7 +16,7 @@ use raft::eraftpb::MessageType; use raftstore::router::{LocalReadRouter, RaftStoreRouter}; use raftstore::store::{ Callback, CasualMessage, CasualRouter, PeerMsg, ProposalRouter, RaftCommand, SignificantMsg, - StoreMsg, StoreRouter, Transport, + SignificantRouter, StoreMsg, StoreRouter, Transport, }; use raftstore::Result as RaftStoreResult; use raftstore::{DiscardReason, Error, Result}; @@ -230,15 +230,17 @@ impl> CasualRouter for SimulateTran } } +impl> SignificantRouter for SimulateTransport { + fn significant_send(&self, region_id: u64, msg: SignificantMsg) -> Result<()> { + self.ch.significant_send(region_id, msg) + } +} + impl> RaftStoreRouter for SimulateTransport { fn send_raft_msg(&self, msg: RaftMessage) -> Result<()> { filter_send(&self.filters, msg, |m| self.ch.send_raft_msg(m)) } - fn significant_send(&self, region_id: u64, msg: SignificantMsg) -> Result<()> { - self.ch.significant_send(region_id, msg) - } - fn broadcast_normal(&self, _: impl FnMut() -> PeerMsg) {} } diff --git a/src/server/server.rs b/src/server/server.rs index 7d83d793025..39f36ca2d13 100644 --- a/src/server/server.rs +++ b/src/server/server.rs @@ -374,12 +374,7 @@ pub mod test_router { } } - impl RaftStoreRouter for TestRaftStoreRouter { - fn send_raft_msg(&self, _: RaftMessage) -> RaftStoreResult<()> { - let _ = self.tx.send(1); - Ok(()) - } - + impl SignificantRouter for TestRaftStoreRouter { fn significant_send( &self, _: u64, @@ -388,6 +383,13 @@ pub mod test_router { let _ = self.significant_msg_sender.send(msg); Ok(()) } + } + + impl RaftStoreRouter for TestRaftStoreRouter { + fn send_raft_msg(&self, _: RaftMessage) -> RaftStoreResult<()> { + let _ = self.tx.send(1); + Ok(()) + } fn broadcast_normal(&self, _: impl FnMut() -> PeerMsg) { let _ = self.tx.send(1); diff --git a/tests/benches/misc/raftkv/mod.rs b/tests/benches/misc/raftkv/mod.rs index 53f7fb133a5..739ec3ec7c3 100644 --- a/tests/benches/misc/raftkv/mod.rs +++ b/tests/benches/misc/raftkv/mod.rs @@ -13,8 +13,8 @@ use kvproto::raft_serverpb::RaftMessage; use raftstore::router::{LocalReadRouter, RaftStoreRouter}; use raftstore::store::{ cmd_resp, util, Callback, CasualMessage, CasualRouter, PeerMsg, ProposalRouter, - RaftCmdExtraOpts, RaftCommand, ReadResponse, RegionSnapshot, SignificantMsg, StoreMsg, - StoreRouter, WriteResponse, + RaftCmdExtraOpts, RaftCommand, ReadResponse, RegionSnapshot, SignificantMsg, SignificantRouter, + StoreMsg, StoreRouter, WriteResponse, }; use raftstore::Result; use tempfile::{Builder, TempDir}; @@ -73,6 +73,12 @@ impl CasualRouter for SyncBenchRouter { } } +impl SignificantRouter for SyncBenchRouter { + fn significant_send(&self, _: u64, _: SignificantMsg) -> Result<()> { + Ok(()) + } +} + impl ProposalRouter for SyncBenchRouter { fn send( &self, @@ -93,11 +99,6 @@ impl RaftStoreRouter for SyncBenchRouter { Ok(()) } - /// Sends a significant message. We should guarantee that the message can't be dropped. - fn significant_send(&self, _: u64, _: SignificantMsg) -> Result<()> { - Ok(()) - } - fn broadcast_normal(&self, _: impl FnMut() -> PeerMsg) {} fn send_command( From 6170255604aff4a52b365c91d09859e4315045c5 Mon Sep 17 00:00:00 2001 From: Jay Date: Wed, 19 Jan 2022 14:27:44 +0800 Subject: [PATCH 05/16] raftstore: move scan delete to raft log gc worker (#11853) * raftstore: move scan delete to raft log gc worker When clearing raft metas, raftstore will scan raft logs and delete them one by one. Seeking can be slow if there are a lot of tombstone keys. This PR moves the operation to raft log gc worker to reduce the impact. The final solution should be also moving remaining IO operations to async write IO threads. Close #10210. Signed-off-by: Jay Lee * address comment Signed-off-by: Jay Lee * speed up destroy Signed-off-by: Jay Lee * fix compile Signed-off-by: Jay Lee * further speed up Signed-off-by: Jay Lee * revert test case configuration Signed-off-by: Jay Lee * address comment Signed-off-by: Jay Lee * address comment Signed-off-by: Jay Lee Co-authored-by: Ti Chi Robot --- components/engine_panic/src/raft_engine.rs | 5 + components/engine_rocks/src/raft_engine.rs | 43 +++-- components/engine_traits/src/raft_engine.rs | 4 + components/raft_log_engine/src/engine.rs | 10 ++ components/raftstore/src/store/bootstrap.rs | 2 +- components/raftstore/src/store/fsm/peer.rs | 148 ++++++++++++++---- components/raftstore/src/store/fsm/store.rs | 2 +- components/raftstore/src/store/msg.rs | 1 + components/raftstore/src/store/peer.rs | 45 +++++- .../raftstore/src/store/peer_storage.rs | 61 ++++++-- .../raftstore/src/store/worker/raftlog_gc.rs | 100 +++++++----- components/server/src/raft_engine_switch.rs | 2 +- components/tikv_util/src/yatp_pool/mod.rs | 5 +- tests/failpoints/cases/test_snap.rs | 61 ++++++++ tests/failpoints/cases/test_stale_peer.rs | 47 +++++- tests/integrations/raftstore/test_snap.rs | 35 ++++- .../integrations/raftstore/test_tombstone.rs | 75 ++++++++- 17 files changed, 541 insertions(+), 105 deletions(-) diff --git a/components/engine_panic/src/raft_engine.rs b/components/engine_panic/src/raft_engine.rs index b7f05365bfe..54e394a22ac 100644 --- a/components/engine_panic/src/raft_engine.rs +++ b/components/engine_panic/src/raft_engine.rs @@ -25,6 +25,10 @@ impl RaftEngineReadOnly for PanicEngine { ) -> Result { panic!() } + + fn get_all_entries_to(&self, region_id: u64, buf: &mut Vec) -> Result<()> { + panic!() + } } impl RaftEngine for PanicEngine { @@ -55,6 +59,7 @@ impl RaftEngine for PanicEngine { fn clean( &self, raft_group_id: u64, + first_index: u64, state: &RaftLocalState, batch: &mut Self::LogBatch, ) -> Result<()> { diff --git a/components/engine_rocks/src/raft_engine.rs b/components/engine_rocks/src/raft_engine.rs index 26558643154..df64211f090 100644 --- a/components/engine_rocks/src/raft_engine.rs +++ b/components/engine_rocks/src/raft_engine.rs @@ -97,6 +97,23 @@ impl RaftEngineReadOnly for RocksEngine { // Here means we don't fetch enough entries. Err(Error::EntriesUnavailable) } + + fn get_all_entries_to(&self, region_id: u64, buf: &mut Vec) -> Result<()> { + let start_key = keys::raft_log_key(region_id, 0); + let end_key = keys::raft_log_key(region_id, u64::MAX); + self.scan( + &start_key, + &end_key, + false, // fill_cache + |_, value| { + let mut entry = Entry::default(); + entry.merge_from_bytes(value)?; + buf.push(entry); + Ok(true) + }, + )?; + Ok(()) + } } impl RocksEngine { fn gc_impl( @@ -171,21 +188,29 @@ impl RaftEngine for RocksEngine { fn clean( &self, raft_group_id: u64, + mut first_index: u64, state: &RaftLocalState, batch: &mut Self::LogBatch, ) -> Result<()> { batch.delete(&keys::raft_state_key(raft_group_id))?; - let seek_key = keys::raft_log_key(raft_group_id, 0); - let prefix = keys::raft_log_prefix(raft_group_id); - if let Some((key, _)) = self.seek(&seek_key)? { - if !key.starts_with(&prefix) { - // No raft logs for the raft group. + if first_index == 0 { + let seek_key = keys::raft_log_key(raft_group_id, 0); + let prefix = keys::raft_log_prefix(raft_group_id); + fail::fail_point!("engine_rocks_raft_engine_clean_seek", |_| Ok(())); + if let Some((key, _)) = self.seek(&seek_key)? { + if !key.starts_with(&prefix) { + // No raft logs for the raft group. + return Ok(()); + } + first_index = match keys::raft_log_index(&key) { + Ok(index) => index, + Err(_) => return Ok(()), + }; + } else { return Ok(()); } - let first_index = match keys::raft_log_index(&key) { - Ok(index) => index, - Err(_) => return Ok(()), - }; + } + if first_index <= state.last_index { for index in first_index..=state.last_index { let key = keys::raft_log_key(raft_group_id, index); batch.delete(&key)?; diff --git a/components/engine_traits/src/raft_engine.rs b/components/engine_traits/src/raft_engine.rs index 1d73d16b3f0..0325bd397b5 100644 --- a/components/engine_traits/src/raft_engine.rs +++ b/components/engine_traits/src/raft_engine.rs @@ -18,6 +18,9 @@ pub trait RaftEngineReadOnly: Sync + Send + 'static { max_size: Option, to: &mut Vec, ) -> Result; + + /// Get all available entries in the region. + fn get_all_entries_to(&self, region_id: u64, buf: &mut Vec) -> Result<()>; } pub struct RaftLogGCTask { @@ -50,6 +53,7 @@ pub trait RaftEngine: RaftEngineReadOnly + Clone + Sync + Send + 'static { fn clean( &self, raft_group_id: u64, + first_index: u64, state: &RaftLocalState, batch: &mut Self::LogBatch, ) -> Result<()>; diff --git a/components/raft_log_engine/src/engine.rs b/components/raft_log_engine/src/engine.rs index 93a81372ab9..7d9c1b9cb2c 100644 --- a/components/raft_log_engine/src/engine.rs +++ b/components/raft_log_engine/src/engine.rs @@ -242,6 +242,15 @@ impl RaftEngineReadOnly for RaftLogEngine { .fetch_entries_to::(raft_group_id, begin, end, max_size, to) .map_err(transfer_error) } + + fn get_all_entries_to(&self, raft_group_id: u64, buf: &mut Vec) -> Result<()> { + if let Some(first) = self.0.first_index(raft_group_id) { + let last = self.0.last_index(raft_group_id).unwrap(); + buf.reserve((last - first + 1) as usize); + self.fetch_entries_to(raft_group_id, first, last + 1, None, buf)?; + } + Ok(()) + } } impl RaftEngine for RaftLogEngine { @@ -272,6 +281,7 @@ impl RaftEngine for RaftLogEngine { fn clean( &self, raft_group_id: u64, + _: u64, _: &RaftLocalState, batch: &mut RaftLogBatch, ) -> Result<()> { diff --git a/components/raftstore/src/store/bootstrap.rs b/components/raftstore/src/store/bootstrap.rs index 46700056b20..e49efdb01e9 100644 --- a/components/raftstore/src/store/bootstrap.rs +++ b/components/raftstore/src/store/bootstrap.rs @@ -97,7 +97,7 @@ pub fn clear_prepare_bootstrap_cluster( box_try!( engines .raft - .clean(region_id, &RaftLocalState::default(), &mut wb) + .clean(region_id, 0, &RaftLocalState::default(), &mut wb) ); box_try!(engines.raft.consume(&mut wb, true)); diff --git a/components/raftstore/src/store/fsm/peer.rs b/components/raftstore/src/store/fsm/peer.rs index 0de30cffb5d..06e1092bff6 100644 --- a/components/raftstore/src/store/fsm/peer.rs +++ b/components/raftstore/src/store/fsm/peer.rs @@ -77,6 +77,19 @@ use crate::store::{ }; use crate::{Error, Result}; +#[derive(Clone, Copy, Debug)] +pub struct DelayDestroy { + merged_by_target: bool, + reason: DelayReason, +} + +#[derive(Clone, Copy, Debug, PartialEq)] +enum DelayReason { + UnPersistedReady, + UnFlushLogGc, + Shutdown, +} + /// Limits the maximum number of regions returned by error. /// /// Another choice is using coprocessor batch limit, but 10 should be a good fit in most case. @@ -125,7 +138,10 @@ where /// Destroy is delayed because of some unpersisted readies in Peer. /// Should call `destroy_peer` again after persisting all readies. - delayed_destroy: Option, + delayed_destroy: Option, + /// Before actually destroying a peer, ensure all log gc tasks are finished, so we + /// can start destroying without seeking. + logs_gc_flushed: bool, } pub struct BatchRaftCmdRequestBuilder @@ -233,6 +249,7 @@ where batch_req_builder: BatchRaftCmdRequestBuilder::new(), trace: PeerMemoryTrace::default(), delayed_destroy: None, + logs_gc_flushed: false, }), )) } @@ -276,6 +293,7 @@ where batch_req_builder: BatchRaftCmdRequestBuilder::new(), trace: PeerMemoryTrace::default(), delayed_destroy: None, + logs_gc_flushed: false, }), )) } @@ -1141,6 +1159,9 @@ where SignificantMsg::LeaderCallback(cb) => { self.on_leader_callback(cb); } + SignificantMsg::RaftLogGcFlushed => { + self.on_raft_log_gc_flushed(); + } } } @@ -1166,9 +1187,11 @@ where self.fsm.has_ready = true; - if let Some(mbt) = self.fsm.delayed_destroy { - if !self.fsm.peer.has_unpersisted_ready() { - self.destroy_peer(mbt); + if let Some(delay) = self.fsm.delayed_destroy { + if delay.reason == DelayReason::UnPersistedReady + && !self.fsm.peer.has_unpersisted_ready() + { + self.destroy_peer(delay.merged_by_target); } } } @@ -2392,24 +2415,106 @@ where } } + /// Check if destroy can be executed immediately. If it can't, the reason is returned. + fn maybe_delay_destroy(&mut self) -> Option { + if self.fsm.peer.has_unpersisted_ready() { + assert!(self.ctx.sync_write_worker.is_none()); + // The destroy must be delayed if there are some unpersisted readies. + // Otherwise there is a race of writting kv db and raft db between here + // and write worker. + return Some(DelayReason::UnPersistedReady); + } + + if !self.fsm.logs_gc_flushed { + let start_index = self.fsm.peer.last_compacted_idx; + let mut end_index = start_index; + if end_index == 0 { + // Technically, all logs between first index and last index should be accessible + // before being destroyed. + end_index = self.fsm.peer.get_store().first_index(); + self.fsm.peer.last_compacted_idx = end_index; + } + let region_id = self.region_id(); + let peer_id = self.fsm.peer.peer_id(); + let mb = match self.ctx.router.mailbox(region_id) { + Some(mb) => mb, + None => { + if tikv_util::thread_group::is_shutdown(!cfg!(test)) { + // It's shutting down, nothing we can do. + return Some(DelayReason::Shutdown); + } + panic!("{} failed to get mailbox", self.fsm.peer.tag); + } + }; + let task = RaftlogGcTask::gc( + self.fsm.peer.get_store().get_region_id(), + start_index, + end_index, + ) + .flush() + .when_done(move || { + if let Err(e) = + mb.force_send(PeerMsg::SignificantMsg(SignificantMsg::RaftLogGcFlushed)) + { + if tikv_util::thread_group::is_shutdown(!cfg!(test)) { + return; + } + panic!( + "[region {}] {} failed to respond flush message {:?}", + region_id, peer_id, e + ); + } + }); + if let Err(e) = self.ctx.raftlog_gc_scheduler.schedule(task) { + if tikv_util::thread_group::is_shutdown(!cfg!(test)) { + // It's shutting down, nothing we can do. + return Some(DelayReason::Shutdown); + } + panic!( + "{} failed to schedule raft log task {:?}", + self.fsm.peer.tag, e + ); + } + // We need to delete all logs entries to avoid introducing race between + // new peers and old peers. Flushing gc logs allow last_compact_index be + // used directly without seeking. + return Some(DelayReason::UnFlushLogGc); + } + None + } + + fn on_raft_log_gc_flushed(&mut self) { + self.fsm.logs_gc_flushed = true; + let delay = match self.fsm.delayed_destroy { + Some(delay) => delay, + None => panic!("{} a delayed destroy should not recover", self.fsm.peer.tag), + }; + self.destroy_peer(delay.merged_by_target); + } + // [PerformanceCriticalPath] TODO: spin off the I/O code (self.fsm.peer.destroy) fn destroy_peer(&mut self, merged_by_target: bool) -> bool { fail_point!("destroy_peer"); // Mark itself as pending_remove self.fsm.peer.pending_remove = true; + fail_point!("destroy_peer_after_pending_move", |_| { true }); - if self.fsm.peer.has_unpersisted_ready() { - assert!(self.ctx.sync_write_worker.is_none()); - // The destroy must be delayed if there are some unpersisted readies. - // Otherwise there is a race of writting kv db and raft db between here - // and write worker. - if let Some(mbt) = self.fsm.delayed_destroy { + + if let Some(reason) = self.maybe_delay_destroy() { + if self + .fsm + .delayed_destroy + .map_or(false, |delay| delay.reason == reason) + { panic!( - "{} destroy peer twice with some unpersisted readies, original {}, now {}", - self.fsm.peer.tag, mbt, merged_by_target + "{} destroy peer twice with same delay reason, original {:?}, now {}", + self.fsm.peer.tag, self.fsm.delayed_destroy, merged_by_target ); } - self.fsm.delayed_destroy = Some(merged_by_target); + self.fsm.delayed_destroy = Some(DelayDestroy { + merged_by_target, + reason, + }); // TODO: The destroy process can also be asynchronous as snapshot process, // if so, all write db operations are removed in store thread. info!( @@ -2417,6 +2522,7 @@ where "region_id" => self.fsm.region_id(), "peer_id" => self.fsm.peer_id(), "merged_by_target" => merged_by_target, + "reason" => ?reason, ); return false; } @@ -2730,21 +2836,9 @@ where self.fsm.peer.raft_log_size_hint = self.fsm.peer.raft_log_size_hint * remain_cnt / total_cnt; let compact_to = state.get_index() + 1; - let task = RaftlogGcTask::gc( - self.fsm.peer.get_store().get_region_id(), - self.fsm.peer.last_compacted_idx, - compact_to, - ); + self.fsm.peer.schedule_raftlog_gc(self.ctx, compact_to); self.fsm.peer.last_compacted_idx = compact_to; self.fsm.peer.mut_store().compact_to(compact_to); - if let Err(e) = self.ctx.raftlog_gc_scheduler.schedule(task) { - error!( - "failed to schedule compact task"; - "region_id" => self.fsm.region_id(), - "peer_id" => self.fsm.peer_id(), - "err" => %e, - ); - } } fn on_ready_split_region( @@ -3526,7 +3620,7 @@ where ); } MergeResultKind::FromTargetSnapshotStep2 => { - // `merge_by_target` is true because this region's range already belongs to + // `merged_by_target` is true because this region's range already belongs to // its target region so we must not clear data otherwise its target region's // data will corrupt. self.destroy_peer(true); diff --git a/components/raftstore/src/store/fsm/store.rs b/components/raftstore/src/store/fsm/store.rs index 98b697703d5..e6b44ec3f52 100644 --- a/components/raftstore/src/store/fsm/store.rs +++ b/components/raftstore/src/store/fsm/store.rs @@ -1078,7 +1078,7 @@ impl RaftPollerBuilder { None => return, Some(value) => value, }; - peer_storage::clear_meta(&self.engines, kv_wb, raft_wb, rid, &raft_state).unwrap(); + peer_storage::clear_meta(&self.engines, kv_wb, raft_wb, rid, 0, &raft_state).unwrap(); let key = keys::region_state_key(rid); kv_wb.put_msg_cf(CF_RAFT, &key, origin_state).unwrap(); } diff --git a/components/raftstore/src/store/msg.rs b/components/raftstore/src/store/msg.rs index 8d3db1ca37f..d994ccabd11 100644 --- a/components/raftstore/src/store/msg.rs +++ b/components/raftstore/src/store/msg.rs @@ -302,6 +302,7 @@ where callback: Callback, }, LeaderCallback(Callback), + RaftLogGcFlushed, } /// Message that will be sent to a peer. diff --git a/components/raftstore/src/store/peer.rs b/components/raftstore/src/store/peer.rs index e1b0c16c485..73f020ee92e 100644 --- a/components/raftstore/src/store/peer.rs +++ b/components/raftstore/src/store/peer.rs @@ -53,9 +53,12 @@ use crate::store::hibernate_state::GroupState; use crate::store::memory::{needs_evict_entry_cache, MEMTRACE_RAFT_ENTRIES}; use crate::store::msg::RaftCommand; use crate::store::util::{admin_cmd_epoch_lookup, RegionReadProgress}; -use crate::store::worker::{HeartbeatTask, ReadDelegate, ReadExecutor, ReadProgress, RegionTask}; +use crate::store::worker::{ + HeartbeatTask, RaftlogGcTask, ReadDelegate, ReadExecutor, ReadProgress, RegionTask, +}; use crate::store::{ Callback, Config, GlobalReplicationState, PdTask, ReadIndexContext, ReadResponse, TxnExt, + RAFT_INIT_LOG_INDEX, }; use crate::{Error, Result}; use collections::{HashMap, HashSet}; @@ -937,7 +940,11 @@ where // Set Tombstone state explicitly let mut kv_wb = engines.kv.write_batch(); let mut raft_wb = engines.raft.log_batch(1024); - self.mut_store().clear_meta(&mut kv_wb, &mut raft_wb)?; + // Raft log gc should be flushed before being destroyed, so last_compacted_idx has to be + // the minimal index that may still have logs. + let last_compacted_idx = self.last_compacted_idx; + self.mut_store() + .clear_meta(last_compacted_idx, &mut kv_wb, &mut raft_wb)?; // StoreFsmDelegate::check_msg use both epoch and region peer list to check whether // a message is targing a staled peer. But for an uninitialized peer, both epoch and @@ -1816,6 +1823,31 @@ where && !self.replication_sync } + pub fn schedule_raftlog_gc( + &mut self, + ctx: &mut PollContext, + to: u64, + ) -> bool { + let task = RaftlogGcTask::gc(self.region_id, self.last_compacted_idx, to); + debug!( + "scheduling raft log gc task"; + "region_id" => self.region_id, + "peer_id" => self.peer_id(), + "task" => %task, + ); + if let Err(e) = ctx.raftlog_gc_scheduler.schedule(task) { + error!( + "failed to schedule raft log gc task"; + "region_id" => self.region_id, + "peer_id" => self.peer_id(), + "err" => %e, + ); + false + } else { + true + } + } + /// Check the current snapshot status. /// Returns whether it's valid to handle raft ready. /// @@ -2185,6 +2217,7 @@ where msgs, snap_region, destroy_regions, + last_first_index, } = res { // When applying snapshot, there is no log applied and not compacted yet. @@ -2200,6 +2233,14 @@ where destroy_regions, }), }); + if self.last_compacted_idx == 0 && last_first_index >= RAFT_INIT_LOG_INDEX { + // There may be stale logs in raft engine, so schedule a task to clean it + // up. This is a best effort, if TiKV is shutdown before the task is + // handled, there can still be stale logs not being deleted until next + // log gc command is executed. This will delete range [0, last_first_index). + self.schedule_raftlog_gc(ctx, last_first_index); + self.last_compacted_idx = last_first_index; + } // Pause `read_progress` to prevent serving stale read while applying snapshot self.read_progress.pause(); } diff --git a/components/raftstore/src/store/peer_storage.rs b/components/raftstore/src/store/peer_storage.rs index db26f175fe4..03829522700 100644 --- a/components/raftstore/src/store/peer_storage.rs +++ b/components/raftstore/src/store/peer_storage.rs @@ -423,6 +423,8 @@ pub enum HandleReadyResult { snap_region: metapb::Region, /// The regions whose range are overlapped with this region destroy_regions: Vec, + /// The first index before applying the snapshot. + last_first_index: u64, }, NoIOTask, } @@ -1210,7 +1212,18 @@ where if self.is_initialized() { // we can only delete the old data when the peer is initialized. - self.clear_meta(kv_wb, raft_wb)?; + let first_index = self.first_index(); + // It's possible that logs between `last_compacted_idx` and `first_index` are + // being deleted in raftlog_gc worker. But it's OK as: + // 1. If the peer accepts a new snapshot, it must start with an index larger than + // this `first_index`; + // 2. If the peer accepts new entries after this snapshot or new snapshot, it must + // start with the new applied index, which is larger than `first_index`. + // So new logs won't be deleted by on going raftlog_gc task accidentally. + // It's possible that there will be some logs between `last_compacted_idx` and + // `first_index` are not deleted. So a cleanup task for the range should be triggered + // after applying the snapshot. + self.clear_meta(first_index, kv_wb, raft_wb)?; } // Write its source peers' `RegionLocalState` together with itself for atomicity for r in destroy_regions { @@ -1256,11 +1269,19 @@ where /// Delete all meta belong to the region. Results are stored in `wb`. pub fn clear_meta( &mut self, + first_index: u64, kv_wb: &mut EK::WriteBatch, raft_wb: &mut ER::LogBatch, ) -> Result<()> { let region_id = self.get_region_id(); - clear_meta(&self.engines, kv_wb, raft_wb, region_id, &self.raft_state)?; + clear_meta( + &self.engines, + kv_wb, + raft_wb, + region_id, + first_index, + &self.raft_state, + )?; self.cache = EntryCache::default(); Ok(()) } @@ -1465,6 +1486,7 @@ where let mut res = HandleReadyResult::SendIOTask; if !ready.snapshot().is_empty() { fail_point!("raft_before_apply_snap"); + let last_first_index = self.first_index(); let snap_region = self.apply_snapshot(ready.snapshot(), &mut write_task, &destroy_regions)?; @@ -1472,6 +1494,7 @@ where msgs: ready.take_persisted_messages(), snap_region, destroy_regions, + last_first_index, }; fail_point!("raft_after_apply_snap"); }; @@ -1568,6 +1591,7 @@ pub fn clear_meta( kv_wb: &mut EK::WriteBatch, raft_wb: &mut ER::LogBatch, region_id: u64, + first_index: u64, raft_state: &RaftLocalState, ) -> Result<()> where @@ -1577,7 +1601,11 @@ where let t = Instant::now(); box_try!(kv_wb.delete_cf(CF_RAFT, &keys::region_state_key(region_id))); box_try!(kv_wb.delete_cf(CF_RAFT, &keys::apply_state_key(region_id))); - box_try!(engines.raft.clean(region_id, raft_state, raft_wb)); + box_try!( + engines + .raft + .clean(region_id, first_index, raft_state, raft_wb) + ); info!( "finish clear peer meta"; @@ -1941,21 +1969,26 @@ mod tests { #[test] fn test_storage_clear_meta() { - let td = Builder::new().prefix("tikv-store").tempdir().unwrap(); let worker = Worker::new("snap-manager").lazy_build("snap-manager"); - let sched = worker.scheduler(); - let mut store = new_storage_from_ents(sched, &td, &[new_entry(3, 3), new_entry(4, 4)]); - append_ents(&mut store, &[new_entry(5, 5), new_entry(6, 6)]); + let cases = vec![(0, 0), (5, 1)]; + for (first_index, left) in cases { + let td = Builder::new().prefix("tikv-store").tempdir().unwrap(); + let sched = worker.scheduler(); + let mut store = new_storage_from_ents(sched, &td, &[new_entry(3, 3), new_entry(4, 4)]); + append_ents(&mut store, &[new_entry(5, 5), new_entry(6, 6)]); - assert_eq!(6, get_meta_key_count(&store)); + assert_eq!(6, get_meta_key_count(&store)); - let mut kv_wb = store.engines.kv.write_batch(); - let mut raft_wb = store.engines.raft.write_batch(); - store.clear_meta(&mut kv_wb, &mut raft_wb).unwrap(); - kv_wb.write().unwrap(); - raft_wb.write().unwrap(); + let mut kv_wb = store.engines.kv.write_batch(); + let mut raft_wb = store.engines.raft.write_batch(); + store + .clear_meta(first_index, &mut kv_wb, &mut raft_wb) + .unwrap(); + kv_wb.write().unwrap(); + raft_wb.write().unwrap(); - assert_eq!(0, get_meta_key_count(&store)); + assert_eq!(left, get_meta_key_count(&store)); + } } #[test] diff --git a/components/raftstore/src/store/worker/raftlog_gc.rs b/components/raftstore/src/store/worker/raftlog_gc.rs index d8fc9f2d492..05d10f03dfb 100644 --- a/components/raftstore/src/store/worker/raftlog_gc.rs +++ b/components/raftstore/src/store/worker/raftlog_gc.rs @@ -15,39 +15,48 @@ use tikv_util::{box_try, debug, error, warn}; use crate::store::worker::metrics::*; const MAX_GC_REGION_BATCH: usize = 512; -const MAX_REGION_NORMAL_GC_LOG_NUBER: u64 = 10240; +const MAX_REGION_NORMAL_GC_LOG_NUMBER: u64 = 10240; -pub enum Task { - Gc { - region_id: u64, - start_idx: u64, - end_idx: u64, - }, +pub struct Task { + region_id: u64, + start_idx: u64, + end_idx: u64, + flush: bool, + cb: Option>, } impl Task { pub fn gc(region_id: u64, start: u64, end: u64) -> Self { - Task::Gc { + Task { region_id, start_idx: start, end_idx: end, + flush: false, + cb: None, } } + + pub fn flush(mut self) -> Self { + self.flush = true; + self + } + + pub fn when_done(mut self, callback: impl FnOnce() + Send + 'static) -> Self { + self.cb = Some(Box::new(callback)); + self + } } impl Display for Task { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - match self { - Task::Gc { - region_id, - start_idx, - end_idx, - } => write!( - f, - "GC Raft Logs [region: {}, from: {}, to: {}]", - region_id, start_idx, end_idx - ), - } + write!( + f, + "GC Raft Logs [region: {}, from: {}, to: {}, has_cb: {}]", + self.region_id, + self.start_idx, + self.end_idx, + self.cb.is_some() + ) } } @@ -76,6 +85,9 @@ impl Runner { /// Does the GC job and returns the count of logs collected. fn gc_raft_log(&mut self, regions: Vec) -> Result { + fail::fail_point!("worker_gc_raft_log", |s| { + Ok(s.and_then(|s| s.parse().ok()).unwrap_or(0)) + }); let deleted = box_try!(self.engines.raft.batch_gc(regions)); Ok(deleted) } @@ -98,28 +110,31 @@ impl Runner { RAFT_LOG_GC_KV_SYNC_DURATION_HISTOGRAM.observe(start.saturating_elapsed_secs()); let tasks = std::mem::take(&mut self.tasks); let mut groups = Vec::with_capacity(tasks.len()); + let mut cbs = Vec::new(); for t in tasks { - match t { - Task::Gc { - region_id, - start_idx, - end_idx, - } => { - debug!("gc raft log"; "region_id" => region_id, "end_index" => end_idx); - if start_idx == 0 { - RAFT_LOG_GC_SEEK_OPERATIONS.inc(); - } else if end_idx > start_idx + MAX_REGION_NORMAL_GC_LOG_NUBER { - warn!("gc raft log with a large range"; "region_id" => region_id, - "start_index" => start_idx, - "end_index" => end_idx); - } - groups.push(RaftLogGCTask { - raft_group_id: region_id, - from: start_idx, - to: end_idx, - }); - } + debug!("gc raft log"; "region_id" => t.region_id, "start_index" => t.start_idx, "end_index" => t.end_idx); + if let Some(cb) = t.cb { + cbs.push(cb); } + if t.start_idx == t.end_idx { + // It's only for flush. + continue; + } + if t.start_idx == 0 { + RAFT_LOG_GC_SEEK_OPERATIONS.inc(); + } else if t.end_idx > t.start_idx + MAX_REGION_NORMAL_GC_LOG_NUMBER { + warn!( + "gc raft log with a large range"; + "region_id" => t.region_id, + "start_index" => t.start_idx, + "end_index" => t.end_idx, + ); + } + groups.push(RaftLogGCTask { + raft_group_id: t.region_id, + from: t.start_idx, + to: t.end_idx, + }); } let start = Instant::now(); match self.gc_raft_log(groups) { @@ -135,6 +150,9 @@ impl Runner { } } RAFT_LOG_GC_WRITE_DURATION_HISTOGRAM.observe(start.saturating_elapsed_secs()); + for cb in cbs { + cb() + } } } @@ -147,8 +165,10 @@ where fn run(&mut self, task: Task) { let _io_type_guard = WithIOType::new(IOType::ForegroundWrite); + let flush_now = task.flush; self.tasks.push(task); - if self.tasks.len() > MAX_GC_REGION_BATCH { + // TODO: maybe they should also be batched even `flush_now` is true. + if flush_now || self.tasks.len() > MAX_GC_REGION_BATCH { self.flush(); } } diff --git a/components/server/src/raft_engine_switch.rs b/components/server/src/raft_engine_switch.rs index ad9359374ef..da9c0598e8e 100644 --- a/components/server/src/raft_engine_switch.rs +++ b/components/server/src/raft_engine_switch.rs @@ -47,7 +47,7 @@ fn clear_raft_engine(engine: &RaftLogEngine) -> Result<(), EngineError> { let mut batch_to_clean = engine.log_batch(0); for id in engine.raft_groups() { let state = engine.get_raft_state(id)?.unwrap(); - engine.clean(id, &state, &mut batch_to_clean)?; + engine.clean(id, 0, &state, &mut batch_to_clean)?; } engine.consume(&mut batch_to_clean, true).map(|_| ()) } diff --git a/components/tikv_util/src/yatp_pool/mod.rs b/components/tikv_util/src/yatp_pool/mod.rs index adc5bc4117b..fc7013b578f 100644 --- a/components/tikv_util/src/yatp_pool/mod.rs +++ b/components/tikv_util/src/yatp_pool/mod.rs @@ -251,8 +251,9 @@ impl YatpPoolBuilder { } fn create_builder(&mut self) -> (yatp::Builder, YatpPoolRunner) { - let mut builder = - yatp::Builder::new(self.name_prefix.clone().unwrap_or_else(|| "".to_string())); + let mut builder = yatp::Builder::new(thd_name!( + self.name_prefix.clone().unwrap_or_else(|| "".to_string()) + )); builder .stack_size(self.stack_size) .min_thread_count(self.min_thread_count) diff --git a/tests/failpoints/cases/test_snap.rs b/tests/failpoints/cases/test_snap.rs index f9e5f2b0ce1..254b8a6cb4b 100644 --- a/tests/failpoints/cases/test_snap.rs +++ b/tests/failpoints/cases/test_snap.rs @@ -5,6 +5,7 @@ use std::sync::{mpsc, Arc, Mutex}; use std::time::Duration; use std::{fs, io, thread}; +use engine_traits::RaftEngineReadOnly; use raft::eraftpb::MessageType; use std::fs::File; use std::io::prelude::*; @@ -197,6 +198,12 @@ fn test_destroy_peer_on_pending_snapshot() { let pd_client = Arc::clone(&cluster.pd_client); pd_client.disable_default_operator(); + fail::cfg_callback("engine_rocks_raft_engine_clean_seek", move || { + if std::thread::current().name().unwrap().contains("raftstore") { + panic!("seek should not happen in raftstore threads"); + } + }) + .unwrap(); let r1 = cluster.run_conf_change(); pd_client.must_add_peer(r1, new_peer(2, 2)); pd_client.must_add_peer(r1, new_peer(3, 3)); @@ -668,3 +675,57 @@ fn test_sending_fail_with_net_error() { must_get_none(&engine2, b"k1"); assert_eq!(cluster.get_snap_mgr(2).stats().receiving_count, 0); } + +/// Logs scan are now moved to raftlog gc threads. The case is to test if logs +/// are still cleaned up when there is stale logs before first index during applying +/// snapshot. It's expected to schedule a gc task after applying snapshot. +#[test] +fn test_snapshot_clean_up_logs_with_unfinished_log_gc() { + let mut cluster = new_node_cluster(0, 3); + cluster.cfg.raft_store.raft_log_gc_count_limit = 15; + cluster.cfg.raft_store.raft_log_gc_threshold = 15; + // Speed up log gc. + cluster.cfg.raft_store.raft_log_compact_sync_interval = ReadableDuration::millis(1); + let pd_client = cluster.pd_client.clone(); + + // Disable default max peer number check. + pd_client.disable_default_operator(); + cluster.run(); + // Simulate raft log gc are pending in queue. + let fp = "worker_gc_raft_log"; + fail::cfg(fp, "return(0)").unwrap(); + + let state = cluster.truncated_state(1, 3); + for i in 0..30 { + let b = format!("k{}", i).into_bytes(); + cluster.must_put(&b, &b); + } + must_get_equal(&cluster.get_engine(3), b"k29", b"k29"); + cluster.wait_log_truncated(1, 3, state.get_index() + 1); + cluster.stop_node(3); + let truncated_index = cluster.truncated_state(1, 3).get_index(); + let raft_engine = cluster.engines[&3].raft.clone(); + // Make sure there are stale logs. + raft_engine.get_entry(1, truncated_index).unwrap().unwrap(); + + let last_index = cluster.raft_local_state(1, 3).get_last_index(); + for i in 30..60 { + let b = format!("k{}", i).into_bytes(); + cluster.must_put(&b, &b); + } + cluster.wait_log_truncated(1, 2, last_index + 1); + + fail::remove(fp); + // So peer (3, 3) will accept a snapshot. And all stale logs before first + // index should be cleaned up. + cluster.run_node(3).unwrap(); + must_get_equal(&cluster.get_engine(3), b"k59", b"k59"); + cluster.must_put(b"k60", b"v60"); + must_get_equal(&cluster.get_engine(3), b"k60", b"v60"); + + let truncated_index = cluster.truncated_state(1, 3).get_index(); + let mut dest = vec![]; + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + // Only previous log should be cleaned up. + assert!(dest[0].get_index() > truncated_index, "{:?}", dest); +} diff --git a/tests/failpoints/cases/test_stale_peer.rs b/tests/failpoints/cases/test_stale_peer.rs index 1acce37a491..1bb55f0ac79 100644 --- a/tests/failpoints/cases/test_stale_peer.rs +++ b/tests/failpoints/cases/test_stale_peer.rs @@ -6,7 +6,7 @@ use std::thread; use std::time::Duration; use engine_rocks::Compat; -use engine_traits::Peekable; +use engine_traits::{Peekable, RaftEngineReadOnly}; use futures::executor::block_on; use kvproto::raft_serverpb::{PeerState, RaftLocalState, RaftMessage}; use pd_client::PdClient; @@ -289,3 +289,48 @@ fn test_destroy_uninitialized_peer_when_there_exists_old_peer() { let region = block_on(pd_client.get_region_by_id(r1)).unwrap(); must_region_cleared(&cluster.get_all_engines(3), ®ion.unwrap()); } + +/// Logs scan are now moved to raftlog gc threads. The case is to test if logs +/// are still cleaned up when there is stale logs before first index during destroy. +#[test] +fn test_destroy_clean_up_logs_with_unfinished_log_gc() { + let mut cluster = new_node_cluster(0, 3); + cluster.cfg.raft_store.raft_log_gc_count_limit = 15; + cluster.cfg.raft_store.raft_log_gc_threshold = 15; + let pd_client = cluster.pd_client.clone(); + + // Disable default max peer number check. + pd_client.disable_default_operator(); + cluster.run(); + // Simulate raft log gc are pending in queue. + let fp = "worker_gc_raft_log"; + fail::cfg(fp, "return(0)").unwrap(); + + let state = cluster.truncated_state(1, 3); + for i in 0..30 { + let b = format!("k{}", i).into_bytes(); + cluster.must_put(&b, &b); + } + must_get_equal(&cluster.get_engine(3), b"k29", b"k29"); + cluster.wait_log_truncated(1, 3, state.get_index() + 1); + cluster.stop_node(3); + let truncated_index = cluster.truncated_state(1, 3).get_index(); + let raft_engine = cluster.engines[&3].raft.clone(); + // Make sure there are stale logs. + raft_engine.get_entry(1, truncated_index).unwrap().unwrap(); + + pd_client.must_remove_peer(1, new_peer(3, 3)); + cluster.must_put(b"k30", b"v30"); + must_get_equal(&cluster.get_engine(1), b"k30", b"v30"); + + fail::remove(fp); + // So peer (3, 3) will be destroyed by gc message. And all stale logs before first + // index should be cleaned up. + cluster.run_node(3).unwrap(); + must_get_none(&cluster.get_engine(3), b"k29"); + + let mut dest = vec![]; + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + // All logs should be deleted. + assert!(dest.is_empty(), "{:?}", dest); +} diff --git a/tests/integrations/raftstore/test_snap.rs b/tests/integrations/raftstore/test_snap.rs index 2ad59e8bbd9..ac271e6815f 100644 --- a/tests/integrations/raftstore/test_snap.rs +++ b/tests/integrations/raftstore/test_snap.rs @@ -7,7 +7,7 @@ use std::sync::mpsc::{self, Sender}; use std::sync::{Arc, Mutex, RwLock}; use std::time::Duration; -use engine_traits::KvEngine; +use engine_traits::{KvEngine, RaftEngineReadOnly}; use file_system::{IOOp, IOType}; use futures::executor::block_on; use grpcio::Environment; @@ -670,3 +670,36 @@ fn test_correct_snapshot_term() { // If peer 4 panicks, it won't be able to apply new writes. must_get_equal(&cluster.get_engine(4), b"k1", b"v1"); } + +/// Test when applying a snapshot, old logs should be cleaned up. +#[test] +fn test_snapshot_clean_up_logs_with_log_gc() { + let mut cluster = new_node_cluster(0, 4); + cluster.cfg.raft_store.raft_log_gc_count_limit = 50; + cluster.cfg.raft_store.raft_log_gc_threshold = 50; + // Speed up log gc. + cluster.cfg.raft_store.raft_log_compact_sync_interval = ReadableDuration::millis(1); + let pd_client = cluster.pd_client.clone(); + + // Disable default max peer number check. + pd_client.disable_default_operator(); + let r = cluster.run_conf_change(); + pd_client.must_add_peer(r, new_peer(2, 2)); + pd_client.must_add_peer(r, new_peer(3, 3)); + cluster.add_send_filter(IsolationFilterFactory::new(2)); + pd_client.must_add_peer(r, new_peer(4, 4)); + pd_client.must_remove_peer(r, new_peer(3, 3)); + cluster.must_transfer_leader(r, new_peer(4, 4)); + cluster.must_put(b"k1", b"v1"); + must_get_equal(&cluster.get_engine(4), b"k1", b"v1"); + cluster.clear_send_filters(); + cluster.add_send_filter(IsolationFilterFactory::new(1)); + // Peer (4, 4) must become leader at the end and send snapshot to 2. + must_get_equal(&cluster.get_engine(2), b"k1", b"v1"); + + let raft_engine = cluster.engines[&2].raft.clone(); + let mut dest = vec![]; + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + // No new log is proposed, so there should be no log at all. + assert!(dest.is_empty(), "{:?}", dest); +} diff --git a/tests/integrations/raftstore/test_tombstone.rs b/tests/integrations/raftstore/test_tombstone.rs index 449c39c7ee3..e67e87616b0 100644 --- a/tests/integrations/raftstore/test_tombstone.rs +++ b/tests/integrations/raftstore/test_tombstone.rs @@ -5,18 +5,17 @@ use std::thread; use std::time::Duration; use crossbeam::channel; +use engine_rocks::raw::Writable; +use engine_rocks::Compat; +use engine_traits::{Iterable, Peekable, RaftEngineReadOnly}; +use engine_traits::{SyncMutable, CF_RAFT}; use kvproto::raft_serverpb::{PeerState, RaftMessage, RegionLocalState, StoreIdent}; use protobuf::Message; use raft::eraftpb::MessageType; +use test_raftstore::*; use tikv_util::config::*; use tikv_util::time::Instant; -use engine_rocks::raw::Writable; -use engine_rocks::Compat; -use engine_traits::{Iterable, Peekable}; -use engine_traits::{SyncMutable, CF_RAFT}; -use test_raftstore::*; - fn test_tombstone(cluster: &mut Cluster) { let pd_client = Arc::clone(&cluster.pd_client); // Disable default max peer number check. @@ -337,3 +336,67 @@ fn test_safe_tombstone_gc() { thread::sleep(base_tick_interval * tick as u32 * 3); must_get_equal(&cluster.get_engine(5), b"k1", b"v1"); } + +/// Logs scan are now moved to raftlog gc threads. The case is to test if logs +/// are cleaned up no mater whether log gc task has been executed. +#[test] +fn test_destroy_clean_up_logs_with_log_gc() { + let mut cluster = new_node_cluster(0, 3); + cluster.cfg.raft_store.raft_log_gc_count_limit = 50; + cluster.cfg.raft_store.raft_log_gc_threshold = 50; + let pd_client = cluster.pd_client.clone(); + + // Disable default max peer number check. + pd_client.disable_default_operator(); + cluster.run(); + cluster.must_put(b"k1", b"v1"); + cluster.must_put(b"k2", b"v2"); + must_get_equal(&cluster.get_engine(3), b"k1", b"v1"); + let raft_engine = cluster.engines[&3].raft.clone(); + let mut dest = vec![]; + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + assert!(!dest.is_empty()); + + pd_client.must_remove_peer(1, new_peer(3, 3)); + must_get_none(&cluster.get_engine(3), b"k1"); + dest.clear(); + // Normally destroy peer should cleanup all logs. + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + assert!(dest.is_empty(), "{:?}", dest); + + pd_client.must_add_peer(1, new_peer(3, 4)); + must_get_equal(&cluster.get_engine(3), b"k1", b"v1"); + cluster.must_put(b"k3", b"v3"); + must_get_equal(&cluster.get_engine(3), b"k3", b"v3"); + dest.clear(); + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + assert!(!dest.is_empty()); + + pd_client.must_remove_peer(1, new_peer(3, 4)); + must_get_none(&cluster.get_engine(3), b"k1"); + dest.clear(); + // Peer created by snapshot should also cleanup all logs. + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + assert!(dest.is_empty(), "{:?}", dest); + + pd_client.must_add_peer(1, new_peer(3, 5)); + must_get_equal(&cluster.get_engine(3), b"k1", b"v1"); + cluster.must_put(b"k4", b"v4"); + must_get_equal(&cluster.get_engine(3), b"k4", b"v4"); + dest.clear(); + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + assert!(!dest.is_empty()); + + let state = cluster.truncated_state(1, 3); + for _ in 0..50 { + cluster.must_put(b"k5", b"v5"); + } + cluster.wait_log_truncated(1, 3, state.get_index() + 1); + + pd_client.must_remove_peer(1, new_peer(3, 5)); + must_get_none(&cluster.get_engine(3), b"k1"); + dest.clear(); + // Peer destroy after log gc should also cleanup all logs. + raft_engine.get_all_entries_to(1, &mut dest).unwrap(); + assert!(dest.is_empty(), "{:?}", dest); +} From 5f0e9aeb036bbf3d0cb7213fa46da7a70a75faa1 Mon Sep 17 00:00:00 2001 From: Xiang Zhang Date: Wed, 19 Jan 2022 21:55:45 +0800 Subject: [PATCH 06/16] doc: update new rules for linking issue and commit message (#11832) close #11097, close #11831 Signed-off-by: zhangyangyu Co-authored-by: Mini256 Co-authored-by: Mini256 Co-authored-by: Xiaoguang Sun --- CONTRIBUTING.md | 62 ++++++++++++++++++++++++++++++++++--------------- 1 file changed, 43 insertions(+), 19 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index eb03f507ca0..4fcf04d8277 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -115,7 +115,7 @@ Read our configuration guide to learn about various [configuration options](http This is a rough outline of what a contributor's workflow looks like: -- Make sure what you want to contribute is already traced as an issue. +- Make sure what you want to contribute is already traced as an issue (see below for linking issue). * We may discuss the problem and solution in the issue. - Create a Git branch from where you want to base your work. This is usually master. - Write code, add test cases, and commit your work (see below for message format). @@ -150,45 +150,69 @@ The TiKV team actively develops and maintains a bunch of dependencies used in Ti See more in [TiKV Community](https://github.com/tikv/community). -### Format of the commit message +### Linking issues + +Code repositories in TiKV community require **ALL** the pull requests referring to its corresponding issues. In the pull request body, there **MUST** be one line starting with `Issue Number: ` and linking the relevant issues via the [keyword](https://docs.github.com/en/issues/tracking-your-work-with-issues/linking-a-pull-request-to-an-issue#linking-a-pull-request-to-an-issue-using-a-keyword), for example: -We follow a rough convention for commit messages that is designed to answer two -questions: what changed and why. The subject line should feature the what and -the body of the commit should describe the why. +If the pull request resolves the relevant issues, and you want GitHub to close these issues automatically after it merged into the default branch, you can use the syntax (`KEYWORD #ISSUE-NUMBER`) like this: ``` -engine/raftkv: add comment for variable declaration. +Issue Number: close #123 +``` -Improve documentation. +If the pull request links an issue but does not close it, you can use the keyword `ref` like this: -Close #1234. +``` +Issue Number: ref #456 ``` -The format can be described more formally as follows: +Multiple issues should use full syntax for each issue and separate by a comma, like: ``` -: - - - - - -Signed-off-by: +Issue Number: close #123, ref #456 +``` + +For pull requests trying to close issues in a different repository, contributors need to first create an issue in the same repository and use this issue to track. + +If the pull request body does not provide the required content, the bot will add the `do-not-merge/needs-linked-issue` label to the pull request to prevent it from being merged. + +### Format of the commit message + +The bot we use will extract the pull request title as the one-line subject and messages inside the `commit-message` code block as commit message body. For example, a pull request with title `pkg: what's changed in this one package` and body containing: + + ```commit-message + any multiple line commit messages that go into + the final commit message body + + * fix something 1 + * fix something 2 + ``` + +will get a final commit message: + +``` +pkg: what's changed in this one package (#12345) + +any multiple line commit messages that go into +the final commit message body + +* fix something 1 +* fix something 2 ``` -The first line is the subject and should be no longer than 50 characters, the other lines should be wrapped at 72 characters (see [this blog post](https://preslav.me/2015/02/21/what-s-with-the-50-72-rule/) for why). +The first line is the subject (the pull request title) and should be no longer than 50 characters, the other lines should be wrapped at 72 characters (see [this blog post](https://preslav.me/2015/02/21/what-s-with-the-50-72-rule/) for why). If the change affects more than one subsystem, you can use comma to separate them like `util/codec,util/types:`. If the change affects many subsystems, you can use ```*``` instead, like ```*:```. -The body of the commit message should describe why the change was made and at a high level, how the code works. It should also reference the issue for problem context. +The body of the commit message should describe why the change was made and at a high level, how the code works. ### Signing off the Commit The project uses [DCO check](https://github.com/probot/dco#how-it-works) and the commit message must contain a `Signed-off-by` line for [Developer Certificate of Origin](https://developercertificate.org/). -Use option `git commit -s` to sign off your commits. +Use option `git commit -s` to sign off your commits. The bot will group and distinguish the signatures from all your commits in the pull request and append them to the final commit message body. ### Testing AWS From a401f78bc86f7e6ea6a55ad9f453ae31be835b55 Mon Sep 17 00:00:00 2001 From: Xinye Tao Date: Thu, 20 Jan 2022 13:38:30 +0800 Subject: [PATCH 07/16] *: update rust toolchain to 2022-01-07 (#11875) * update rust toolchain to 2022-01-17 Signed-off-by: tabokie * address comment: clean up unnecessary `as_ref()` Signed-off-by: tabokie * try fixing the underflow Signed-off-by: tabokie * mute underflow warning for raft metrics Signed-off-by: tabokie * clean up unused data members Signed-off-by: tabokie * format Signed-off-by: tabokie * step back to 2022-01-07 Signed-off-by: tabokie --- components/backup/src/endpoint.rs | 21 +++---------- components/cdc/src/channel.rs | 4 +-- components/cdc/tests/mod.rs | 2 ++ components/encryption/src/file_dict_file.rs | 8 ++--- components/encryption/src/io.rs | 5 +-- components/engine_rocks/src/config.rs | 2 +- components/engine_rocks/src/properties.rs | 14 ++++----- components/engine_traits/src/options.rs | 11 +------ components/engine_traits/src/sst.rs | 5 +++ components/file_system/src/lib.rs | 10 ++---- .../src/coprocessor/split_check/table.rs | 4 +-- components/raftstore/src/lib.rs | 1 - components/raftstore/src/store/fsm/apply.rs | 2 -- components/raftstore/src/store/fsm/peer.rs | 27 +++++++--------- components/raftstore/src/store/fsm/store.rs | 20 +++++++++--- .../raftstore/src/store/local_metrics.rs | 30 ++---------------- components/raftstore/src/store/snap.rs | 3 ++ components/raftstore/src/store/snap/io.rs | 4 +-- .../raftstore/src/store/worker/query_stats.rs | 1 + .../raftstore/src/store/worker/region.rs | 3 +- .../src/store/worker/split_controller.rs | 10 +----- components/resolved_ts/src/cmd.rs | 1 + components/resource_metering/src/lib.rs | 1 - components/resource_metering/src/model.rs | 1 + .../resource_metering/src/recorder/mod.rs | 2 ++ .../resource_metering/tests/summary_test.rs | 6 ++-- components/security/src/lib.rs | 16 +--------- components/test_coprocessor/src/column.rs | 4 +++ components/test_coprocessor/src/dag.rs | 18 +++++++++++ components/test_coprocessor/src/store.rs | 1 + components/test_coprocessor/src/table.rs | 1 + components/test_raftstore/src/cluster.rs | 7 ++--- .../test_raftstore/src/transport_simulate.rs | 7 +++++ components/test_storage/src/sync_storage.rs | 6 ++-- .../src/builder/field_type.rs | 6 ++++ .../src/codec/batch/lazy_column.rs | 1 + .../src/codec/batch/lazy_column_vec.rs | 2 ++ .../src/codec/data_type/chunked_vec_bytes.rs | 5 +-- .../src/codec/data_type/chunked_vec_json.rs | 5 +-- .../src/codec/data_type/vector.rs | 1 + .../src/codec/mysql/duration.rs | 3 ++ .../src/codec/mysql/json/json_modify.rs | 2 +- .../src/codec/mysql/time/mod.rs | 15 +-------- .../src/codec/mysql/time/weekmode.rs | 1 + .../src/codec/row/v2/encoder_for_test.rs | 3 ++ components/tidb_query_executors/src/runner.rs | 17 +++++----- .../src/slow_hash_aggr_executor.rs | 2 +- .../src/top_n_executor.rs | 2 +- components/tidb_query_expr/src/impl_cast.rs | 31 ++++++------------- .../tidb_query_expr/src/impl_encryption.rs | 4 +-- .../tidb_query_expr/src/types/expr_builder.rs | 5 +++ .../tidb_query_expr/src/types/test_util.rs | 6 ++++ components/tikv_kv/src/cursor.rs | 8 +++++ components/tikv_kv/src/lib.rs | 13 +------- components/tikv_kv/src/mock_engine.rs | 6 ++++ components/tikv_util/src/lib.rs | 1 + components/tikv_util/src/lru.rs | 7 +---- components/tikv_util/src/worker/pool.rs | 2 ++ .../tipb_helper/src/expr_def_builder.rs | 1 + components/txn_types/src/lock.rs | 2 ++ components/txn_types/src/timestamp.rs | 2 ++ components/txn_types/src/types.rs | 1 + components/txn_types/src/write.rs | 1 + rust-toolchain | 2 +- scripts/clippy | 2 +- src/config.rs | 20 ++++++------ src/coprocessor/dag/mod.rs | 1 + src/coprocessor/statistics/analyze.rs | 12 +------ src/coprocessor/statistics/cmsketch.rs | 1 + src/coprocessor_v2/config.rs | 10 +----- src/coprocessor_v2/endpoint.rs | 1 + src/lib.rs | 1 - src/server/gc_worker/gc_manager.rs | 4 +-- src/server/lock_manager/client.rs | 2 -- src/server/lock_manager/waiter_manager.rs | 1 + src/server/service/diagnostics/sys.rs | 2 +- src/storage/mod.rs | 1 + src/storage/mvcc/reader/point_getter.rs | 7 +++++ src/storage/mvcc/reader/scanner/backward.rs | 2 +- src/storage/mvcc/reader/scanner/forward.rs | 3 +- src/storage/mvcc/reader/scanner/mod.rs | 10 ++++++ .../txn/commands/acquire_pessimistic_lock.rs | 8 ++--- .../txn/commands/check_secondary_locks.rs | 4 +-- src/storage/txn/commands/check_txn_status.rs | 4 +-- src/storage/txn/commands/cleanup.rs | 4 +-- src/storage/txn/commands/commit.rs | 4 +-- .../txn/commands/pessimistic_rollback.rs | 8 ++--- src/storage/txn/commands/prewrite.rs | 2 +- src/storage/txn/commands/resolve_lock.rs | 8 ++--- src/storage/txn/commands/resolve_lock_lite.rs | 4 +-- src/storage/txn/commands/rollback.rs | 4 +-- src/storage/txn/commands/txn_heart_beat.rs | 4 +-- src/storage/txn/flow_controller.rs | 1 - src/storage/txn/store.rs | 6 ++-- .../raftstore/test_replication_mode.rs | 2 +- 95 files changed, 258 insertions(+), 300 deletions(-) diff --git a/components/backup/src/endpoint.rs b/components/backup/src/endpoint.rs index 35563c596bd..8cffa7190d4 100644 --- a/components/backup/src/endpoint.rs +++ b/components/backup/src/endpoint.rs @@ -85,12 +85,6 @@ impl fmt::Debug for Task { } } -#[derive(Clone)] -struct LimitedStorage { - limiter: Limiter, - storage: Arc, -} - impl Task { /// Create a backup task based on the given backup request. pub fn new( @@ -483,7 +477,7 @@ impl BackupRange { &self, engine: E, db: Arc, - storage: &LimitedStorage, + limiter: &Limiter, file_name: String, cf: CfNameWrap, compression_type: Option, @@ -495,7 +489,7 @@ impl BackupRange { db, &file_name, cf, - storage.limiter.clone(), + limiter.clone(), compression_type, compression_level, cipher, @@ -793,7 +787,7 @@ impl Endpoint { request: Request, saver_tx: async_channel::Sender, resp_tx: UnboundedSender, - backend: Arc, + _backend: Arc, ) { let start_ts = request.start_ts; let backup_ts = request.end_ts; @@ -806,11 +800,6 @@ impl Endpoint { let limit = self.softlimit.limit(); self.pool.borrow_mut().spawn(async move { - let storage = LimitedStorage { - limiter: request.limiter, - storage: backend, - }; - loop { // when get the guard, release it until we finish scanning a batch, // because if we were suspended during scanning, @@ -868,7 +857,7 @@ impl Endpoint { .backup_raw_kv_to_file( engine, db.clone(), - &storage, + &request.limiter, name, cf.into(), ct, @@ -880,7 +869,7 @@ impl Endpoint { } else { let writer_builder = BackupWriterBuilder::new( store_id, - storage.limiter.clone(), + request.limiter.clone(), brange.region.clone(), db.clone(), ct, diff --git a/components/cdc/src/channel.rs b/components/cdc/src/channel.rs index 5b3cefbb090..5afa03c92c1 100644 --- a/components/cdc/src/channel.rs +++ b/components/cdc/src/channel.rs @@ -387,12 +387,12 @@ impl<'a> Drain { }); let (event_bytes, resolved_ts_bytes) = batcher.statistics(); let resps = batcher.build(); - let last_idx = resps.len() - 1; + let resps_len = resps.len(); // Events are about to be sent, free pending events memory counter. memory_quota.free(bytes as _); for (i, e) in resps.into_iter().enumerate() { // Buffer messages and flush them at once. - let write_flags = WriteFlags::default().buffer_hint(i != last_idx); + let write_flags = WriteFlags::default().buffer_hint(i + 1 != resps_len); sink.feed((e, write_flags)).await?; } sink.flush().await?; diff --git a/components/cdc/tests/mod.rs b/components/cdc/tests/mod.rs index 8368c2792d5..9858e246373 100644 --- a/components/cdc/tests/mod.rs +++ b/components/cdc/tests/mod.rs @@ -104,11 +104,13 @@ impl TestSuiteBuilder { } } + #[must_use] pub fn cluster(mut self, cluster: Cluster) -> TestSuiteBuilder { self.cluster = Some(cluster); self } + #[must_use] pub fn memory_quota(mut self, memory_quota: usize) -> TestSuiteBuilder { self.memory_quota = Some(memory_quota); self diff --git a/components/encryption/src/file_dict_file.rs b/components/encryption/src/file_dict_file.rs index 196a860a861..b2553be3344 100644 --- a/components/encryption/src/file_dict_file.rs +++ b/components/encryption/src/file_dict_file.rs @@ -548,14 +548,14 @@ mod tests { ) .unwrap(); - file_dict.insert(&"f1".to_owned(), &info1).unwrap(); - file_dict.insert(&"f2".to_owned(), &info2).unwrap(); - file_dict.insert(&"f3".to_owned(), &info3).unwrap(); + file_dict.insert("f1", &info1).unwrap(); + file_dict.insert("f2", &info2).unwrap(); + file_dict.insert("f3", &info3).unwrap(); file_dict.insert("f4", &info4).unwrap(); file_dict.remove("f3").unwrap(); - file_dict.remove(&"f2".to_owned()).unwrap(); + file_dict.remove("f2").unwrap(); } // Try open as v1 file. Should fail. { diff --git a/components/encryption/src/io.rs b/components/encryption/src/io.rs index 1cdb1b0560f..842257be372 100644 --- a/components/encryption/src/io.rs +++ b/components/encryption/src/io.rs @@ -399,10 +399,7 @@ impl CrypterCore { fn reset_buffer(&mut self, size: usize) { // OCrypter require the output buffer to have block_size extra bytes, or it will panic. - self.buffer.reserve(size + self.block_size); - unsafe { - self.buffer.set_len(size + self.block_size); - } + self.buffer.resize(size + self.block_size, 0); } pub fn reset_crypter(&mut self, offset: u64) -> IoResult<()> { diff --git a/components/engine_rocks/src/config.rs b/components/engine_rocks/src/config.rs index ec9aa6f9348..369c63530c3 100644 --- a/components/engine_rocks/src/config.rs +++ b/components/engine_rocks/src/config.rs @@ -199,7 +199,7 @@ pub mod compression_type_serde { "disable" => DBCompressionType::Disable, _ => { return Err(E::invalid_value( - Unexpected::Other(&"invalid compression type".to_string()), + Unexpected::Other("invalid compression type"), &self, )); } diff --git a/components/engine_rocks/src/properties.rs b/components/engine_rocks/src/properties.rs index eb244908f51..773ae81dc6a 100644 --- a/components/engine_rocks/src/properties.rs +++ b/components/engine_rocks/src/properties.rs @@ -598,19 +598,19 @@ mod tests { assert_eq!(props.get_approximate_keys_in_range(b"", b"k"), 11_u64); assert_eq!(props.offsets.len(), 7); - let a = props.get(b"a".as_ref()); + let a = props.get(b"a"); assert_eq!(a.size, 1); - let e = props.get(b"e".as_ref()); + let e = props.get(b"e"); assert_eq!(e.size, DEFAULT_PROP_SIZE_INDEX_DISTANCE + 5); - let i = props.get(b"i".as_ref()); + let i = props.get(b"i"); assert_eq!(i.size, DEFAULT_PROP_SIZE_INDEX_DISTANCE / 8 * 17 + 9); - let k = props.get(b"k".as_ref()); + let k = props.get(b"k"); assert_eq!(k.size, DEFAULT_PROP_SIZE_INDEX_DISTANCE / 8 * 25 + 11); - let m = props.get(b"m".as_ref()); + let m = props.get(b"m"); assert_eq!(m.keys, 11 + DEFAULT_PROP_KEYS_INDEX_DISTANCE); - let n = props.get(b"n".as_ref()); + let n = props.get(b"n"); assert_eq!(n.keys, 11 + 2 * DEFAULT_PROP_KEYS_INDEX_DISTANCE); - let o = props.get(b"o".as_ref()); + let o = props.get(b"o"); assert_eq!(o.keys, 12 + 2 * DEFAULT_PROP_KEYS_INDEX_DISTANCE); let empty = RangeOffsets::default(); let cases = [ diff --git a/components/engine_traits/src/options.rs b/components/engine_traits/src/options.rs index 19676d6a313..2891872b77f 100644 --- a/components/engine_traits/src/options.rs +++ b/components/engine_traits/src/options.rs @@ -29,7 +29,7 @@ impl Default for ReadOptions { } } -#[derive(Clone)] +#[derive(Clone, Default)] pub struct WriteOptions { sync: bool, no_slowdown: bool, @@ -60,15 +60,6 @@ impl WriteOptions { } } -impl Default for WriteOptions { - fn default() -> WriteOptions { - WriteOptions { - sync: false, - no_slowdown: false, - } - } -} - #[derive(Clone, PartialEq)] pub enum SeekMode { TotalOrder, diff --git a/components/engine_traits/src/sst.rs b/components/engine_traits/src/sst.rs index 033e7eb2ee5..3c88998148b 100644 --- a/components/engine_traits/src/sst.rs +++ b/components/engine_traits/src/sst.rs @@ -66,17 +66,22 @@ where fn new() -> Self; /// Set DB for the builder. The builder may need some config from the DB. + #[must_use] fn set_db(self, db: &E) -> Self; /// Set CF for the builder. The builder may need some config from the CF. + #[must_use] fn set_cf(self, cf: &str) -> Self; /// Set it to true, the builder builds a in-memory SST builder. + #[must_use] fn set_in_memory(self, in_memory: bool) -> Self; /// set other config specified by writer + #[must_use] fn set_compression_type(self, compression: Option) -> Self; + #[must_use] fn set_compression_level(self, level: i32) -> Self; /// Builder a SstWriter. diff --git a/components/file_system/src/lib.rs b/components/file_system/src/lib.rs index 0f06295f756..dd06443cff1 100644 --- a/components/file_system/src/lib.rs +++ b/components/file_system/src/lib.rs @@ -1,7 +1,7 @@ // Copyright 2020 TiKV Project Authors. Licensed under Apache-2.0. #![feature(test)] -#![feature(duration_consts_2)] +#![feature(duration_consts_float)] #[macro_use] extern crate lazy_static; @@ -108,18 +108,12 @@ impl Drop for WithIOType { } #[repr(C)] -#[derive(Debug, Copy, Clone)] +#[derive(Debug, Copy, Clone, Default)] pub struct IOBytes { read: u64, write: u64, } -impl Default for IOBytes { - fn default() -> Self { - IOBytes { read: 0, write: 0 } - } -} - impl std::ops::Sub for IOBytes { type Output = Self; diff --git a/components/raftstore/src/coprocessor/split_check/table.rs b/components/raftstore/src/coprocessor/split_check/table.rs index e24258ead63..3e6ed691dd4 100644 --- a/components/raftstore/src/coprocessor/split_check/table.rs +++ b/components/raftstore/src/coprocessor/split_check/table.rs @@ -344,8 +344,8 @@ mod tests { type Case = (Option>, Option>, Option); let mut check_cases = |cases: Vec| { for (encoded_start_key, encoded_end_key, table_id) in cases { - region.set_start_key(encoded_start_key.unwrap_or_else(Vec::new)); - region.set_end_key(encoded_end_key.unwrap_or_else(Vec::new)); + region.set_start_key(encoded_start_key.unwrap_or_default()); + region.set_end_key(encoded_end_key.unwrap_or_default()); runnable.run(SplitCheckTask::split_check( region.clone(), true, diff --git a/components/raftstore/src/lib.rs b/components/raftstore/src/lib.rs index 3bfe0990651..844e47e3525 100644 --- a/components/raftstore/src/lib.rs +++ b/components/raftstore/src/lib.rs @@ -2,7 +2,6 @@ #![cfg_attr(test, feature(test))] #![feature(cell_update)] -#![feature(shrink_to)] #![feature(div_duration)] #![feature(min_specialization)] #![feature(box_patterns)] diff --git a/components/raftstore/src/store/fsm/apply.rs b/components/raftstore/src/store/fsm/apply.rs index 53c292c8a1b..1ca6724f17d 100644 --- a/components/raftstore/src/store/fsm/apply.rs +++ b/components/raftstore/src/store/fsm/apply.rs @@ -4794,9 +4794,7 @@ mod tests { #[derive(Clone, Default)] struct ApplyObserver { - pre_admin_count: Arc, pre_query_count: Arc, - post_admin_count: Arc, post_query_count: Arc, cmd_sink: Option>>>, } diff --git a/components/raftstore/src/store/fsm/peer.rs b/components/raftstore/src/store/fsm/peer.rs index 06e1092bff6..378f81a8ac2 100644 --- a/components/raftstore/src/store/fsm/peer.rs +++ b/components/raftstore/src/store/fsm/peer.rs @@ -889,7 +889,7 @@ where // FIXME: should use `bcast_check_stale_peer_message` instead. // Sending a new enum type msg to a old tikv may cause panic during rolling update // we should change the protobuf behavior and check if properly handled in all place - self.fsm.peer.bcast_wake_up_message(&mut self.ctx); + self.fsm.peer.bcast_wake_up_message(self.ctx); } } CasualMessage::SnapshotGenerated => { @@ -920,7 +920,7 @@ where .iter() .any(|p| p.get_id() == self.fsm.peer_id()) { - self.fsm.peer.send_wake_up_message(&mut self.ctx, &leader); + self.fsm.peer.send_wake_up_message(self.ctx, &leader); } } CasualMessage::RejectRaftAppend { peer_id } => { @@ -930,7 +930,7 @@ where msg.from = self.fsm.peer.peer_id(); let raft_msg = self.fsm.peer.build_raft_messages(self.ctx, vec![msg]); - self.fsm.peer.send_raft_messages(&mut self.ctx, raft_msg); + self.fsm.peer.send_raft_messages(self.ctx, raft_msg); } } } @@ -1695,9 +1695,7 @@ where } } - if result.is_err() { - return result; - } + result?; if self.fsm.peer.any_new_peer_catch_up(from_peer_id) { self.fsm.peer.heartbeat_pd(self.ctx); @@ -2289,7 +2287,7 @@ where match self .fsm .peer - .ready_to_transfer_leader(&mut self.ctx, msg.get_index(), &from) + .ready_to_transfer_leader(self.ctx, msg.get_index(), &from) { Some(reason) => { info!( @@ -2337,12 +2335,9 @@ where } } } else { - self.fsm.peer.execute_transfer_leader( - &mut self.ctx, - msg.get_from(), - peer_disk_usage, - false, - ); + self.fsm + .peer + .execute_transfer_leader(self.ctx, msg.get_from(), peer_disk_usage, false); } } @@ -3336,7 +3331,7 @@ where .as_ref() .unwrap() .get_commit(), - &mut self.ctx, + self.ctx, ); } } @@ -4632,7 +4627,7 @@ where "expect" => %self.ctx.cfg.max_leader_missing_duration, ); - self.fsm.peer.bcast_check_stale_peer_message(&mut self.ctx); + self.fsm.peer.bcast_check_stale_peer_message(self.ctx); let task = PdTask::ValidatePeer { peer: self.fsm.peer.peer.clone(), @@ -4736,7 +4731,7 @@ where // As the leader can propose the TransferLeader request successfully, the disk of // the leader is probably not full. self.fsm.peer.execute_transfer_leader( - &mut self.ctx, + self.ctx, self.fsm.peer.leader_id(), DiskUsage::Normal, true, diff --git a/components/raftstore/src/store/fsm/store.rs b/components/raftstore/src/store/fsm/store.rs index e6b44ec3f52..1c68b39053a 100644 --- a/components/raftstore/src/store/fsm/store.rs +++ b/components/raftstore/src/store/fsm/store.rs @@ -859,9 +859,21 @@ impl PollHandler, St self.tag, self.poll_ctx.pending_count, self.poll_ctx.ready_count, - self.poll_ctx.raft_metrics.ready.append - self.previous_metrics.append, - self.poll_ctx.raft_metrics.ready.message - self.previous_metrics.message, - self.poll_ctx.raft_metrics.ready.snapshot - self.previous_metrics.snapshot + self.poll_ctx + .raft_metrics + .ready + .append + .saturating_sub(self.previous_metrics.append), + self.poll_ctx + .raft_metrics + .ready + .message + .saturating_sub(self.previous_metrics.message), + self.poll_ctx + .raft_metrics + .ready + .snapshot + .saturating_sub(self.previous_metrics.snapshot), ); dur } else { @@ -1263,7 +1275,7 @@ impl RaftBatchSystem { } pub fn refresh_config_scheduler(&mut self) -> Scheduler { - assert!(!self.workers.is_none()); + assert!(self.workers.is_some()); self.workers .as_ref() .unwrap() diff --git a/components/raftstore/src/store/local_metrics.rs b/components/raftstore/src/store/local_metrics.rs index 76b9bd6ba1f..9f0544307d8 100644 --- a/components/raftstore/src/store/local_metrics.rs +++ b/components/raftstore/src/store/local_metrics.rs @@ -297,7 +297,7 @@ impl RaftProposeMetrics { } /// The buffered metrics counter for invalid propose -#[derive(Clone)] +#[derive(Clone, Default)] pub struct RaftInvalidProposeMetrics { pub mismatch_store_id: u64, pub region_not_found: u64, @@ -310,22 +310,6 @@ pub struct RaftInvalidProposeMetrics { pub is_applying_snapshot: u64, } -impl Default for RaftInvalidProposeMetrics { - fn default() -> RaftInvalidProposeMetrics { - RaftInvalidProposeMetrics { - mismatch_store_id: 0, - region_not_found: 0, - not_leader: 0, - mismatch_peer_id: 0, - stale_command: 0, - epoch_not_match: 0, - read_index_no_leader: 0, - region_not_initialized: 0, - is_applying_snapshot: 0, - } - } -} - impl RaftInvalidProposeMetrics { fn flush(&mut self) { if self.mismatch_store_id > 0 { @@ -385,23 +369,13 @@ impl RaftInvalidProposeMetrics { } } -#[derive(Clone)] +#[derive(Clone, Default)] pub struct RaftLogGcSkippedMetrics { pub reserve_log: u64, pub threshold_limit: u64, pub compact_idx_too_small: u64, } -impl Default for RaftLogGcSkippedMetrics { - fn default() -> RaftLogGcSkippedMetrics { - RaftLogGcSkippedMetrics { - reserve_log: 0, - threshold_limit: 0, - compact_idx_too_small: 0, - } - } -} - impl RaftLogGcSkippedMetrics { fn flush(&mut self) { if self.reserve_log > 0 { diff --git a/components/raftstore/src/store/snap.rs b/components/raftstore/src/store/snap.rs index 97154d645fa..e7eb56107e4 100644 --- a/components/raftstore/src/store/snap.rs +++ b/components/raftstore/src/store/snap.rs @@ -1490,14 +1490,17 @@ pub struct SnapManagerBuilder { } impl SnapManagerBuilder { + #[must_use] pub fn max_write_bytes_per_sec(mut self, bytes: i64) -> SnapManagerBuilder { self.max_write_bytes_per_sec = bytes; self } + #[must_use] pub fn max_total_size(mut self, bytes: u64) -> SnapManagerBuilder { self.max_total_size = bytes; self } + #[must_use] pub fn encryption_key_manager(mut self, m: Option>) -> SnapManagerBuilder { self.key_manager = m; self diff --git a/components/raftstore/src/store/snap/io.rs b/components/raftstore/src/store/snap/io.rs index 42365db9f85..16c3e8b27b9 100644 --- a/components/raftstore/src/store/snap/io.rs +++ b/components/raftstore/src/store/snap/io.rs @@ -303,8 +303,8 @@ mod tests { cf, 16, |v| { - v.to_owned() - .into_iter() + v.iter() + .cloned() .for_each(|pair| applied_keys.entry(cf).or_default().push(pair)) }, ) diff --git a/components/raftstore/src/store/worker/query_stats.rs b/components/raftstore/src/store/worker/query_stats.rs index 46ea5b37686..01b9ba72c6d 100644 --- a/components/raftstore/src/store/worker/query_stats.rs +++ b/components/raftstore/src/store/worker/query_stats.rs @@ -68,6 +68,7 @@ impl QueryStats { } } + #[must_use] pub fn sub_query_stats(&self, query_stats: &QueryStats) -> QueryStats { let mut res = QueryStats::default(); for kind in QUERY_KINDS { diff --git a/components/raftstore/src/store/worker/region.rs b/components/raftstore/src/store/worker/region.rs index 60000319939..67ee76e70a5 100644 --- a/components/raftstore/src/store/worker/region.rs +++ b/components/raftstore/src/store/worker/region.rs @@ -805,8 +805,7 @@ mod tests { // when we want to insert [g, q), we first extract overlap ranges, // which are [f, i), [m, n), [p, t) let timeout2 = 12; - let overlap_ranges = - pending_delete_ranges.drain_overlap_ranges(&b"g".to_vec(), &b"q".to_vec()); + let overlap_ranges = pending_delete_ranges.drain_overlap_ranges(b"g", b"q"); assert_eq!( overlap_ranges, [ diff --git a/components/raftstore/src/store/worker/split_controller.rs b/components/raftstore/src/store/worker/split_controller.rs index 71c2f2b5c8b..a7f20f42954 100644 --- a/components/raftstore/src/store/worker/split_controller.rs +++ b/components/raftstore/src/store/worker/split_controller.rs @@ -300,7 +300,7 @@ impl Recorder { } } -#[derive(Clone, Debug)] +#[derive(Clone, Debug, Default)] pub struct WriteStats { pub region_infos: HashMap, } @@ -319,14 +319,6 @@ impl WriteStats { } } -impl Default for WriteStats { - fn default() -> WriteStats { - WriteStats { - region_infos: HashMap::default(), - } - } -} - #[derive(Clone, Debug)] pub struct ReadStats { pub region_infos: HashMap, diff --git a/components/resolved_ts/src/cmd.rs b/components/resolved_ts/src/cmd.rs index b9c3cfa3d10..70a930fd7ac 100644 --- a/components/resolved_ts/src/cmd.rs +++ b/components/resolved_ts/src/cmd.rs @@ -33,6 +33,7 @@ pub enum ChangeRow { }, } +#[allow(clippy::large_enum_variant)] pub enum ChangeLog { Error(errorpb::Error), Rows { index: u64, rows: Vec }, diff --git a/components/resource_metering/src/lib.rs b/components/resource_metering/src/lib.rs index 4f045714bd7..472771d64a5 100644 --- a/components/resource_metering/src/lib.rs +++ b/components/resource_metering/src/lib.rs @@ -2,7 +2,6 @@ // TODO(mornyx): crate doc. -#![feature(shrink_to)] #![feature(hash_drain_filter)] #![feature(core_intrinsics)] diff --git a/components/resource_metering/src/model.rs b/components/resource_metering/src/model.rs index f4ae1539934..7d8d23ffd36 100644 --- a/components/resource_metering/src/model.rs +++ b/components/resource_metering/src/model.rs @@ -328,6 +328,7 @@ impl SummaryRecord { } /// Gets the value and writes it to zero. + #[must_use] pub fn take_and_reset(&self) -> Self { Self { read_keys: AtomicU32::new(self.read_keys.swap(0, Relaxed)), diff --git a/components/resource_metering/src/recorder/mod.rs b/components/resource_metering/src/recorder/mod.rs index 56cc699132e..d2cc9e67378 100644 --- a/components/resource_metering/src/recorder/mod.rs +++ b/components/resource_metering/src/recorder/mod.rs @@ -238,12 +238,14 @@ impl Default for RecorderBuilder { impl RecorderBuilder { /// Sets the precision_ms parameter of [Recorder]. + #[must_use] pub fn precision_ms(mut self, precision_ms: u64) -> Self { self.precision_ms = precision_ms; self } /// Add a [SubRecorder] for the execution of [Recorder]. + #[must_use] pub fn add_sub_recorder(mut self, r: Box) -> Self { self.recorders.push(r); self diff --git a/components/resource_metering/tests/summary_test.rs b/components/resource_metering/tests/summary_test.rs index b6f42f12090..1186f3a50d8 100644 --- a/components/resource_metering/tests/summary_test.rs +++ b/components/resource_metering/tests/summary_test.rs @@ -69,7 +69,7 @@ fn test_summary() { resource_metering::record_write_keys(456); } thread::sleep(Duration::from_millis(REPORT_INTERVAL_MS + 500)); // wait report - assert!(data_sink.get(&b"TAG-1".to_vec()).is_none()); + assert!(data_sink.get(b"TAG-1").is_none()); data_sink.clear(); }) .join() @@ -95,7 +95,7 @@ fn test_summary() { } thread::sleep(Duration::from_millis(REPORT_INTERVAL_MS + 500)); // wait report - let r = data_sink.get(&b"TAG-1".to_vec()).unwrap(); + let r = data_sink.get(b"TAG-1").unwrap(); assert_eq!( r.get_record() .get_items() @@ -133,7 +133,7 @@ fn test_summary() { resource_metering::record_write_keys(456); } thread::sleep(Duration::from_millis(REPORT_INTERVAL_MS + 500)); // wait report - assert!(data_sink.get(&b"TAG-1".to_vec()).is_none()); + assert!(data_sink.get(b"TAG-1").is_none()); data_sink.clear(); }) .join() diff --git a/components/security/src/lib.rs b/components/security/src/lib.rs index e4cc41f915a..901a9c78b7e 100644 --- a/components/security/src/lib.rs +++ b/components/security/src/lib.rs @@ -17,7 +17,7 @@ use grpcio::{ ServerCredentialsFetcher, }; -#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Default)] #[serde(default)] #[serde(rename_all = "kebab-case")] pub struct SecurityConfig { @@ -33,20 +33,6 @@ pub struct SecurityConfig { pub encryption: EncryptionConfig, } -impl Default for SecurityConfig { - fn default() -> SecurityConfig { - SecurityConfig { - ca_path: String::new(), - cert_path: String::new(), - key_path: String::new(), - override_ssl_target: String::new(), - cert_allowed_cn: HashSet::default(), - redact_info_log: None, - encryption: EncryptionConfig::default(), - } - } -} - /// Checks and opens key file. Returns `Ok(None)` if the path is empty. /// /// # Arguments diff --git a/components/test_coprocessor/src/column.rs b/components/test_coprocessor/src/column.rs index f17de1e84dc..c10cbb7650c 100644 --- a/components/test_coprocessor/src/column.rs +++ b/components/test_coprocessor/src/column.rs @@ -62,11 +62,13 @@ impl ColumnBuilder { } } + #[must_use] pub fn col_type(mut self, t: i32) -> ColumnBuilder { self.col_type = t; self } + #[must_use] pub fn primary_key(mut self, b: bool) -> ColumnBuilder { if b { self.index = 0; @@ -76,11 +78,13 @@ impl ColumnBuilder { self } + #[must_use] pub fn index_key(mut self, idx_id: i64) -> ColumnBuilder { self.index = idx_id; self } + #[must_use] pub fn default(mut self, val: Datum) -> ColumnBuilder { self.default_val = Some(val); self diff --git a/components/test_coprocessor/src/dag.rs b/components/test_coprocessor/src/dag.rs index dd2d74b9034..1b7812efb20 100644 --- a/components/test_coprocessor/src/dag.rs +++ b/components/test_coprocessor/src/dag.rs @@ -78,11 +78,13 @@ impl DAGSelect { } } + #[must_use] pub fn limit(mut self, n: u64) -> DAGSelect { self.limit = Some(n); self } + #[must_use] pub fn order_by(mut self, col: &Column, desc: bool) -> DAGSelect { let col_offset = offset_for_column(&self.cols, col.id); let mut item = ByItem::default(); @@ -96,10 +98,12 @@ impl DAGSelect { self } + #[must_use] pub fn count(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::Count) } + #[must_use] pub fn aggr_col(mut self, col: &Column, aggr_t: ExprType) -> DAGSelect { let col_offset = offset_for_column(&self.cols, col.id); let mut col_expr = Expr::default(); @@ -119,38 +123,47 @@ impl DAGSelect { self } + #[must_use] pub fn first(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::First) } + #[must_use] pub fn sum(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::Sum) } + #[must_use] pub fn avg(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::Avg) } + #[must_use] pub fn max(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::Max) } + #[must_use] pub fn min(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::Min) } + #[must_use] pub fn bit_and(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::AggBitAnd) } + #[must_use] pub fn bit_or(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::AggBitOr) } + #[must_use] pub fn bit_xor(self, col: &Column) -> DAGSelect { self.aggr_col(col, ExprType::AggBitXor) } + #[must_use] pub fn group_by(mut self, cols: &[&Column]) -> DAGSelect { for col in cols { let offset = offset_for_column(&self.cols, col.id); @@ -163,11 +176,13 @@ impl DAGSelect { self } + #[must_use] pub fn output_offsets(mut self, output_offsets: Option>) -> DAGSelect { self.output_offsets = output_offsets; self } + #[must_use] pub fn where_expr(mut self, expr: Expr) -> DAGSelect { let mut exec = Executor::default(); exec.set_tp(ExecType::TypeSelection); @@ -178,17 +193,20 @@ impl DAGSelect { self } + #[must_use] pub fn desc(mut self, desc: bool) -> DAGSelect { self.execs[0].mut_tbl_scan().set_desc(desc); self } + #[must_use] pub fn paging_size(mut self, paging_size: u64) -> DAGSelect { assert_ne!(paging_size, 0); self.paging_size = Some(paging_size); self } + #[must_use] pub fn key_ranges(mut self, key_ranges: Vec) -> DAGSelect { self.key_ranges = key_ranges; self diff --git a/components/test_coprocessor/src/store.rs b/components/test_coprocessor/src/store.rs index cd7ab240309..a8cce9e3022 100644 --- a/components/test_coprocessor/src/store.rs +++ b/components/test_coprocessor/src/store.rs @@ -32,6 +32,7 @@ impl<'a, E: Engine> Insert<'a, E> { } } + #[must_use] pub fn set(mut self, col: &Column, value: Datum) -> Self { assert!(self.table.column_by_id(col.id).is_some()); self.values.insert(col.id, value); diff --git a/components/test_coprocessor/src/table.rs b/components/test_coprocessor/src/table.rs index fa0660cc716..0cf2646e489 100644 --- a/components/test_coprocessor/src/table.rs +++ b/components/test_coprocessor/src/table.rs @@ -138,6 +138,7 @@ impl TableBuilder { } } + #[must_use] pub fn add_col(mut self, name: impl std::borrow::Borrow, col: Column) -> TableBuilder { use std::cmp::Ordering::*; diff --git a/components/test_raftstore/src/cluster.rs b/components/test_raftstore/src/cluster.rs index ef15ddea433..2dc96a2bdc8 100644 --- a/components/test_raftstore/src/cluster.rs +++ b/components/test_raftstore/src/cluster.rs @@ -737,15 +737,14 @@ impl Cluster { pub fn shutdown(&mut self) { debug!("about to shutdown cluster"); - let keys; - match self.sim.read() { - Ok(s) => keys = s.get_node_ids(), + let keys = match self.sim.read() { + Ok(s) => s.get_node_ids(), Err(_) => { safe_panic!("failed to acquire read lock"); // Leave the resource to avoid double panic. return; } - } + }; for id in keys { self.stop_node(id); } diff --git a/components/test_raftstore/src/transport_simulate.rs b/components/test_raftstore/src/transport_simulate.rs index b7c95d75eda..767304f3f02 100644 --- a/components/test_raftstore/src/transport_simulate.rs +++ b/components/test_raftstore/src/transport_simulate.rs @@ -434,37 +434,44 @@ impl RegionPacketFilter { } } + #[must_use] pub fn direction(mut self, direction: Direction) -> RegionPacketFilter { self.direction = direction; self } // TODO: rename it to `drop`. + #[must_use] pub fn msg_type(mut self, m_type: MessageType) -> RegionPacketFilter { self.drop_type.push(m_type); self } + #[must_use] pub fn skip(mut self, m_type: MessageType) -> RegionPacketFilter { self.skip_type.push(m_type); self } + #[must_use] pub fn allow(mut self, number: usize) -> RegionPacketFilter { self.block = Either::Left(Arc::new(AtomicUsize::new(number))); self } + #[must_use] pub fn when(mut self, condition: Arc) -> RegionPacketFilter { self.block = Either::Right(condition); self } + #[must_use] pub fn reserve_dropped(mut self, dropped: Arc>>) -> RegionPacketFilter { self.dropped_messages = Some(dropped); self } + #[must_use] pub fn set_msg_callback( mut self, cb: Arc, diff --git a/components/test_storage/src/sync_storage.rs b/components/test_storage/src/sync_storage.rs index 2bf7b550a05..60c1492015d 100644 --- a/components/test_storage/src/sync_storage.rs +++ b/components/test_storage/src/sync_storage.rs @@ -59,11 +59,13 @@ impl SyncTestStorageBuilder { } } + #[must_use] pub fn config(mut self, config: Config) -> Self { self.config = Some(config); self } + #[must_use] pub fn gc_config(mut self, gc_config: GcConfig) -> Self { self.gc_config = Some(gc_config); self @@ -154,8 +156,8 @@ impl SyncTestStorage { ) -> Result>>> { let mut ids = vec![]; let requests: Vec = keys - .to_owned() - .into_iter() + .iter() + .copied() .map(|key| { let mut req = GetRequest::default(); req.set_context(ctx.clone()); diff --git a/components/tidb_query_datatype/src/builder/field_type.rs b/components/tidb_query_datatype/src/builder/field_type.rs index 663445bc1e5..8b443ef4921 100644 --- a/components/tidb_query_datatype/src/builder/field_type.rs +++ b/components/tidb_query_datatype/src/builder/field_type.rs @@ -13,31 +13,37 @@ impl FieldTypeBuilder { Default::default() } + #[must_use] pub fn tp(mut self, v: crate::FieldTypeTp) -> Self { FieldTypeAccessor::set_tp(&mut self.0, v); self } + #[must_use] pub fn flag(mut self, v: crate::FieldTypeFlag) -> Self { FieldTypeAccessor::set_flag(&mut self.0, v); self } + #[must_use] pub fn flen(mut self, v: isize) -> Self { FieldTypeAccessor::set_flen(&mut self.0, v); self } + #[must_use] pub fn decimal(mut self, v: isize) -> Self { FieldTypeAccessor::set_decimal(&mut self.0, v); self } + #[must_use] pub fn collation(mut self, v: crate::Collation) -> Self { FieldTypeAccessor::set_collation(&mut self.0, v); self } + #[must_use] pub fn charset(mut self, v: &str) -> Self { self.0.set_charset(String::from(v)); self diff --git a/components/tidb_query_datatype/src/codec/batch/lazy_column.rs b/components/tidb_query_datatype/src/codec/batch/lazy_column.rs index a861aac4dc9..ca57b84f36d 100644 --- a/components/tidb_query_datatype/src/codec/batch/lazy_column.rs +++ b/components/tidb_query_datatype/src/codec/batch/lazy_column.rs @@ -54,6 +54,7 @@ impl LazyBatchColumn { /// Creates a new empty `LazyBatchColumn` with the same schema. #[inline] + #[must_use] pub fn clone_empty(&self, capacity: usize) -> Self { match self { LazyBatchColumn::Raw(_) => Self::raw_with_capacity(capacity), diff --git a/components/tidb_query_datatype/src/codec/batch/lazy_column_vec.rs b/components/tidb_query_datatype/src/codec/batch/lazy_column_vec.rs index e180c22e3f3..e492fa32c91 100644 --- a/components/tidb_query_datatype/src/codec/batch/lazy_column_vec.rs +++ b/components/tidb_query_datatype/src/codec/batch/lazy_column_vec.rs @@ -48,6 +48,7 @@ impl LazyBatchColumnVec { /// Creates a new empty `LazyBatchColumnVec` with the same number of columns and schema. #[inline] + #[must_use] pub fn clone_empty(&self, capacity: usize) -> Self { Self { columns: self @@ -60,6 +61,7 @@ impl LazyBatchColumnVec { /// Creates a new `LazyBatchColumnVec`, which contains `columns_count` number of raw columns. #[cfg(test)] + #[must_use] pub fn with_raw_columns(columns_count: usize) -> Self { let mut columns = Vec::with_capacity(columns_count); for _ in 0..columns_count { diff --git a/components/tidb_query_datatype/src/codec/data_type/chunked_vec_bytes.rs b/components/tidb_query_datatype/src/codec/data_type/chunked_vec_bytes.rs index a5d1c909ef0..4ade7a7dc06 100644 --- a/components/tidb_query_datatype/src/codec/data_type/chunked_vec_bytes.rs +++ b/components/tidb_query_datatype/src/codec/data_type/chunked_vec_bytes.rs @@ -238,10 +238,7 @@ mod tests { None, ]; assert_eq!(ChunkedVecBytes::from_slice(test_bytes).to_vec(), test_bytes); - assert_eq!( - ChunkedVecBytes::from_slice(&test_bytes.to_vec()).to_vec(), - test_bytes - ); + assert_eq!(ChunkedVecBytes::from_slice(test_bytes).to_vec(), test_bytes); } #[test] diff --git a/components/tidb_query_datatype/src/codec/data_type/chunked_vec_json.rs b/components/tidb_query_datatype/src/codec/data_type/chunked_vec_json.rs index a092c301e60..fd03420d45f 100644 --- a/components/tidb_query_datatype/src/codec/data_type/chunked_vec_json.rs +++ b/components/tidb_query_datatype/src/codec/data_type/chunked_vec_json.rs @@ -146,10 +146,7 @@ mod tests { None, ]; assert_eq!(ChunkedVecJson::from_slice(test_json).to_vec(), test_json); - assert_eq!( - ChunkedVecJson::from_slice(&test_json.to_vec()).to_vec(), - test_json - ); + assert_eq!(ChunkedVecJson::from_slice(test_json).to_vec(), test_json); } #[test] diff --git a/components/tidb_query_datatype/src/codec/data_type/vector.rs b/components/tidb_query_datatype/src/codec/data_type/vector.rs index 21b5c01b085..3e545bef6be 100644 --- a/components/tidb_query_datatype/src/codec/data_type/vector.rs +++ b/components/tidb_query_datatype/src/codec/data_type/vector.rs @@ -39,6 +39,7 @@ impl VectorValue { /// Creates a new empty `VectorValue` with the same eval type. #[inline] + #[must_use] pub fn clone_empty(&self, capacity: usize) -> Self { match_template_evaltype! { TT, match self { diff --git a/components/tidb_query_datatype/src/codec/mysql/duration.rs b/components/tidb_query_datatype/src/codec/mysql/duration.rs index c68fc3dbb48..eb522b2981f 100644 --- a/components/tidb_query_datatype/src/codec/mysql/duration.rs +++ b/components/tidb_query_datatype/src/codec/mysql/duration.rs @@ -307,6 +307,7 @@ impl Duration { } #[inline] + #[must_use] pub fn minimize_fsp(self) -> Self { Duration { fsp: MIN_FSP as u8, @@ -315,6 +316,7 @@ impl Duration { } #[inline] + #[must_use] pub fn maximize_fsp(self) -> Self { Duration { fsp: MAX_FSP as u8, @@ -366,6 +368,7 @@ impl Duration { /// Returns the absolute value of `Duration` #[inline] + #[must_use] pub fn abs(self) -> Self { Duration { nanos: self.nanos.abs(), diff --git a/components/tidb_query_datatype/src/codec/mysql/json/json_modify.rs b/components/tidb_query_datatype/src/codec/mysql/json/json_modify.rs index 159ca66530f..71881162da2 100644 --- a/components/tidb_query_datatype/src/codec/mysql/json/json_modify.rs +++ b/components/tidb_query_datatype/src/codec/mysql/json/json_modify.rs @@ -220,7 +220,7 @@ mod tests { json, "#{} expect modified json {:?} == {:?}", i, - json.to_string(), + json, expected.to_string() ); } else { diff --git a/components/tidb_query_datatype/src/codec/mysql/time/mod.rs b/components/tidb_query_datatype/src/codec/mysql/time/mod.rs index 0c047acc8f5..2a81d807518 100755 --- a/components/tidb_query_datatype/src/codec/mysql/time/mod.rs +++ b/components/tidb_query_datatype/src/codec/mysql/time/mod.rs @@ -1964,7 +1964,7 @@ mod tests { use std::sync::Arc; - #[derive(Debug)] + #[derive(Debug, Default)] struct TimeEnv { strict_mode: bool, no_zero_in_date: bool, @@ -1974,19 +1974,6 @@ mod tests { time_zone: Option, } - impl Default for TimeEnv { - fn default() -> TimeEnv { - TimeEnv { - strict_mode: false, - no_zero_in_date: false, - no_zero_date: false, - allow_invalid_date: false, - ignore_truncate: false, - time_zone: None, - } - } - } - impl From for EvalContext { fn from(config: TimeEnv) -> EvalContext { let mut eval_config = EvalConfig::new(); diff --git a/components/tidb_query_datatype/src/codec/mysql/time/weekmode.rs b/components/tidb_query_datatype/src/codec/mysql/time/weekmode.rs index 0b97d0b8096..09ddef98e05 100644 --- a/components/tidb_query_datatype/src/codec/mysql/time/weekmode.rs +++ b/components/tidb_query_datatype/src/codec/mysql/time/weekmode.rs @@ -9,6 +9,7 @@ bitflags::bitflags! { } impl WeekMode { + #[must_use] pub fn to_normalized(self) -> WeekMode { let mut mode = self; if !mode.contains(WeekMode::BEHAVIOR_MONDAY_FIRST) { diff --git a/components/tidb_query_datatype/src/codec/row/v2/encoder_for_test.rs b/components/tidb_query_datatype/src/codec/row/v2/encoder_for_test.rs index d82351cd8b2..7c3ddcc16b2 100644 --- a/components/tidb_query_datatype/src/codec/row/v2/encoder_for_test.rs +++ b/components/tidb_query_datatype/src/codec/row/v2/encoder_for_test.rs @@ -62,6 +62,7 @@ impl Column { &self.ft } + #[must_use] pub fn with_tp(mut self, tp: FieldTypeTp) -> Self { self.ft.as_mut_accessor().set_tp(tp); self @@ -71,11 +72,13 @@ impl Column { self.ft.is_unsigned() } + #[must_use] pub fn with_unsigned(mut self) -> Self { self.ft.as_mut_accessor().set_flag(FieldTypeFlag::UNSIGNED); self } + #[must_use] pub fn with_decimal(mut self, decimal: isize) -> Self { self.ft.as_mut_accessor().set_decimal(decimal); self diff --git a/components/tidb_query_executors/src/runner.rs b/components/tidb_query_executors/src/runner.rs index 6e9df1011c7..4f46351f4b9 100644 --- a/components/tidb_query_executors/src/runner.rs +++ b/components/tidb_query_executors/src/runner.rs @@ -162,13 +162,13 @@ pub fn build_executors( .next() .ok_or_else(|| other_err!("No executors"))?; - let mut executor: Box>; let mut summary_slot_index = 0; // Limit executor use this flag to check if its src is table/index scan. // Performance enhancement for plan like: limit 1 -> table/index scan. let mut is_src_scan_executor = true; - match first_ed.get_tp() { + let mut executor: Box> = match first_ed.get_tp() + { ExecType::TypeTableScan => { EXECUTOR_COUNT_METRICS.batch_table_scan.inc(); @@ -177,7 +177,7 @@ pub fn build_executors( let primary_column_ids = descriptor.take_primary_column_ids(); let primary_prefix_column_ids = descriptor.take_primary_prefix_column_ids(); - executor = Box::new( + Box::new( BatchTableScanExecutor::new( storage, config.clone(), @@ -189,7 +189,7 @@ pub fn build_executors( primary_prefix_column_ids, )? .collect_summary(summary_slot_index), - ); + ) } ExecType::TypeIndexScan => { EXECUTOR_COUNT_METRICS.batch_index_scan.inc(); @@ -197,7 +197,7 @@ pub fn build_executors( let mut descriptor = first_ed.take_idx_scan(); let columns_info = descriptor.take_columns().into(); let primary_column_ids_len = descriptor.take_primary_column_ids().len(); - executor = Box::new( + Box::new( BatchIndexScanExecutor::new( storage, config.clone(), @@ -209,7 +209,7 @@ pub fn build_executors( is_scanned_range_aware, )? .collect_summary(summary_slot_index), - ); + ) } _ => { return Err(other_err!( @@ -217,12 +217,12 @@ pub fn build_executors( first_ed.get_tp() )); } - } + }; for mut ed in executor_descriptors { summary_slot_index += 1; - let new_executor: Box> = match ed.get_tp() { + executor = match ed.get_tp() { ExecType::TypeSelection => { EXECUTOR_COUNT_METRICS.batch_selection.inc(); @@ -331,7 +331,6 @@ pub fn build_executors( )); } }; - executor = new_executor; is_src_scan_executor = false; } diff --git a/components/tidb_query_executors/src/slow_hash_aggr_executor.rs b/components/tidb_query_executors/src/slow_hash_aggr_executor.rs index 6cdafb86644..13ecd652790 100644 --- a/components/tidb_query_executors/src/slow_hash_aggr_executor.rs +++ b/components/tidb_query_executors/src/slow_hash_aggr_executor.rs @@ -206,7 +206,7 @@ pub struct SlowHashAggregationImpl { /// element are stored in `group_key_offsets`. /// /// `GroupKeyRefUnsafe` contains a raw pointer to this buffer. - #[allow(clippy::box_vec)] + #[allow(clippy::box_collection)] group_key_buffer: Box>, /// The offsets of encoded keys in `group_key_buffer`. This `Vec` always has a leading `0` diff --git a/components/tidb_query_executors/src/top_n_executor.rs b/components/tidb_query_executors/src/top_n_executor.rs index a13585b0ed1..1c84d25b5e6 100644 --- a/components/tidb_query_executors/src/top_n_executor.rs +++ b/components/tidb_query_executors/src/top_n_executor.rs @@ -41,7 +41,7 @@ pub struct BatchTopNExecutor { /// /// This field is placed before `order_exprs` and `src` because it relies on data in /// those fields and we want this field to be dropped first. - #[allow(clippy::box_vec)] + #[allow(clippy::box_collection)] eval_columns_buffer_unsafe: Box>>, order_exprs: Box<[RpnExpression]>, diff --git a/components/tidb_query_expr/src/impl_cast.rs b/components/tidb_query_expr/src/impl_cast.rs index 4b9710efe09..ecba330cce8 100644 --- a/components/tidb_query_expr/src/impl_cast.rs +++ b/components/tidb_query_expr/src/impl_cast.rs @@ -516,12 +516,11 @@ fn cast_string_as_signed_real( match val { None => Ok(None), Some(val) => { - let r: f64; - if val.is_empty() { - r = 0.0; + let r = if val.is_empty() { + 0.0 } else { - r = val.convert(ctx)?; - } + val.convert(ctx)? + }; let r = produce_float_with_specified_tp(ctx, extra.ret_field_type, r)?; Ok(Real::new(r).ok()) } @@ -1558,6 +1557,7 @@ mod tests { assert!(r.is_none()); } + #[derive(Default)] struct CtxConfig { overflow_as_warning: bool, truncate_as_warning: bool, @@ -1566,18 +1566,6 @@ mod tests { in_update_or_delete_stmt: bool, } - impl Default for CtxConfig { - fn default() -> Self { - CtxConfig { - overflow_as_warning: false, - truncate_as_warning: false, - should_clip_to_zero: false, - in_insert_stmt: false, - in_update_or_delete_stmt: false, - } - } - } - impl From for EvalContext { fn from(config: CtxConfig) -> Self { let mut flag: Flag = Flag::empty(); @@ -6168,12 +6156,11 @@ mod tests { Ok(v) => match v { Some(dur) => { if expect_max { - let max_val_str: &str; - if dur.is_neg() { - max_val_str = "-838:59:59"; + let max_val_str = if dur.is_neg() { + "-838:59:59" } else { - max_val_str = "838:59:59"; - } + "838:59:59" + }; let max_expect = Duration::parse(&mut ctx, max_val_str, fsp); let log = format!( "func_name: {}, input: {}, output: {:?}, output_warn: {:?}, expect: {:?}", diff --git a/components/tidb_query_expr/src/impl_encryption.rs b/components/tidb_query_expr/src/impl_encryption.rs index a8930ea3b65..bac8140d294 100644 --- a/components/tidb_query_expr/src/impl_encryption.rs +++ b/components/tidb_query_expr/src/impl_encryption.rs @@ -304,7 +304,7 @@ mod tests { ]; for (s, exp) in cases { - let s = s.map(|inner| hex::decode(inner.as_bytes().to_vec()).unwrap()); + let s = s.map(|inner| hex::decode(inner.as_bytes()).unwrap()); let output = RpnFnScalarEvaluator::new() .push_param(s) .evaluate(ScalarFuncSig::UncompressedLength) @@ -383,7 +383,7 @@ mod tests { ), ]; for (arg, expect) in test_cases { - let expect = Some(hex::decode(expect.as_bytes().to_vec()).unwrap()); + let expect = Some(hex::decode(expect.as_bytes()).unwrap()); let output = RpnFnScalarEvaluator::new() .push_param(arg) diff --git a/components/tidb_query_expr/src/types/expr_builder.rs b/components/tidb_query_expr/src/types/expr_builder.rs index f2c2a3c44d1..564b5c67534 100644 --- a/components/tidb_query_expr/src/types/expr_builder.rs +++ b/components/tidb_query_expr/src/types/expr_builder.rs @@ -116,6 +116,7 @@ impl RpnExpressionBuilder { } /// Pushes a `FnCall` node. + #[must_use] pub fn push_fn_call_for_test( mut self, func_meta: RpnFnMeta, @@ -133,6 +134,7 @@ impl RpnExpressionBuilder { } #[cfg(test)] + #[must_use] pub fn push_fn_call_with_metadata( mut self, func_meta: RpnFnMeta, @@ -152,6 +154,7 @@ impl RpnExpressionBuilder { /// Pushes a `Constant` node. The field type will be auto inferred by choosing an arbitrary /// field type that matches the field type of the given value. + #[must_use] pub fn push_constant_for_test(mut self, value: impl Into) -> Self { let value = value.into(); let field_type = value @@ -165,6 +168,7 @@ impl RpnExpressionBuilder { /// Pushes a `Constant` node. #[cfg(test)] + #[must_use] pub fn push_constant_with_field_type( mut self, value: impl Into, @@ -179,6 +183,7 @@ impl RpnExpressionBuilder { } /// Pushes a `ColumnRef` node. + #[must_use] pub fn push_column_ref_for_test(mut self, offset: usize) -> Self { let node = RpnExpressionNode::ColumnRef { offset }; self.0.push(node); diff --git a/components/tidb_query_expr/src/types/test_util.rs b/components/tidb_query_expr/src/types/test_util.rs index 90884276053..0c2ecab2f12 100644 --- a/components/tidb_query_expr/src/types/test_util.rs +++ b/components/tidb_query_expr/src/types/test_util.rs @@ -35,11 +35,13 @@ impl RpnFnScalarEvaluator { /// Pushes a parameter as the value of an argument for evaluation. The field type will be auto /// inferred by choosing an arbitrary field type that matches the field type of the given /// value. + #[must_use] pub fn push_param(mut self, value: impl Into) -> Self { self.rpn_expr_builder = self.rpn_expr_builder.push_constant_for_test(value); self } + #[must_use] pub fn push_params(mut self, values: impl IntoIterator>) -> Self { for value in values { self.rpn_expr_builder = self.rpn_expr_builder.push_constant_for_test(value); @@ -48,6 +50,7 @@ impl RpnFnScalarEvaluator { } /// Pushes a parameter as the value of an argument for evaluation using a specified field type. + #[must_use] pub fn push_param_with_field_type( mut self, value: impl Into, @@ -63,6 +66,7 @@ impl RpnFnScalarEvaluator { /// /// If not set, the evaluation will use an inferred return field type by choosing an arbitrary /// field type that matches the field type of the generic type `T` when calling `evaluate()`. + #[must_use] pub fn return_field_type(mut self, field_type: impl Into) -> Self { self.return_field_type = Some(field_type.into()); self @@ -71,12 +75,14 @@ impl RpnFnScalarEvaluator { /// Sets the context to use during evaluation. /// /// If not set, a default `EvalContext` will be used. + #[must_use] pub fn context(mut self, context: impl Into) -> Self { self.context = Some(context.into()); self } /// Sets the metadata to use during evaluation. + #[must_use] pub fn metadata(mut self, metadata: Box) -> Self { self.metadata = Some(metadata); self diff --git a/components/tikv_kv/src/cursor.rs b/components/tikv_kv/src/cursor.rs index 41916ec144f..55cd35162d1 100644 --- a/components/tikv_kv/src/cursor.rs +++ b/components/tikv_kv/src/cursor.rs @@ -459,6 +459,7 @@ impl<'a, S: 'a + Snapshot> CursorBuilder<'a, S> { /// /// Defaults to `true`. #[inline] + #[must_use] pub fn fill_cache(mut self, fill_cache: bool) -> Self { self.fill_cache = fill_cache; self @@ -468,6 +469,7 @@ impl<'a, S: 'a + Snapshot> CursorBuilder<'a, S> { /// /// Defaults to `false`, it means use total order seek. #[inline] + #[must_use] pub fn prefix_seek(mut self, prefix_seek: bool) -> Self { self.prefix_seek = prefix_seek; self @@ -477,6 +479,7 @@ impl<'a, S: 'a + Snapshot> CursorBuilder<'a, S> { /// /// Defaults to `ScanMode::Forward`. #[inline] + #[must_use] pub fn scan_mode(mut self, scan_mode: ScanMode) -> Self { self.scan_mode = scan_mode; self @@ -487,6 +490,7 @@ impl<'a, S: 'a + Snapshot> CursorBuilder<'a, S> { /// /// Both default to `None`. #[inline] + #[must_use] pub fn range(mut self, lower: Option, upper: Option) -> Self { self.lower_bound = lower; self.upper_bound = upper; @@ -497,6 +501,7 @@ impl<'a, S: 'a + Snapshot> CursorBuilder<'a, S> { /// /// Default is empty. #[inline] + #[must_use] pub fn hint_min_ts(mut self, min_ts: Option) -> Self { self.hint_min_ts = min_ts; self @@ -506,18 +511,21 @@ impl<'a, S: 'a + Snapshot> CursorBuilder<'a, S> { /// /// Default is empty. #[inline] + #[must_use] pub fn hint_max_ts(mut self, max_ts: Option) -> Self { self.hint_max_ts = max_ts; self } #[inline] + #[must_use] pub fn key_only(mut self, key_only: bool) -> Self { self.key_only = key_only; self } #[inline] + #[must_use] pub fn max_skippable_internal_keys(mut self, count: u64) -> Self { self.max_skippable_internal_keys = count; self diff --git a/components/tikv_kv/src/lib.rs b/components/tikv_kv/src/lib.rs index 6ebb45f7b22..291e83ebc35 100644 --- a/components/tikv_kv/src/lib.rs +++ b/components/tikv_kv/src/lib.rs @@ -147,7 +147,7 @@ impl WriteData { } } -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Default)] pub struct SnapContext<'a> { pub pb_ctx: &'a Context, pub read_id: Option, @@ -157,17 +157,6 @@ pub struct SnapContext<'a> { pub key_ranges: Vec, } -impl<'a> Default for SnapContext<'a> { - fn default() -> Self { - SnapContext { - pb_ctx: Default::default(), - read_id: None, - start_ts: Default::default(), - key_ranges: Default::default(), - } - } -} - /// Engine defines the common behaviour for a storage engine type. pub trait Engine: Send + Clone + 'static { type Snap: Snapshot; diff --git a/components/tikv_kv/src/mock_engine.rs b/components/tikv_kv/src/mock_engine.rs index 555f1955bd2..ea343dfa9e9 100644 --- a/components/tikv_kv/src/mock_engine.rs +++ b/components/tikv_kv/src/mock_engine.rs @@ -36,6 +36,7 @@ impl ExpectedWrite { pub fn new() -> Self { Default::default() } + #[must_use] pub fn expect_modify(self, modify: Modify) -> Self { Self { modify: Some(modify), @@ -43,6 +44,7 @@ impl ExpectedWrite { use_committed_cb: self.use_committed_cb, } } + #[must_use] pub fn expect_proposed_cb(self) -> Self { Self { modify: self.modify, @@ -50,6 +52,7 @@ impl ExpectedWrite { use_committed_cb: self.use_committed_cb, } } + #[must_use] pub fn expect_no_proposed_cb(self) -> Self { Self { modify: self.modify, @@ -57,6 +60,7 @@ impl ExpectedWrite { use_committed_cb: self.use_committed_cb, } } + #[must_use] pub fn expect_committed_cb(self) -> Self { Self { modify: self.modify, @@ -64,6 +68,7 @@ impl ExpectedWrite { use_committed_cb: Some(true), } } + #[must_use] pub fn expect_no_committed_cb(self) -> Self { Self { modify: self.modify, @@ -196,6 +201,7 @@ impl MockEngineBuilder { } } + #[must_use] pub fn add_expected_write(mut self, write: ExpectedWrite) -> Self { match self.expected_modifies.as_mut() { Some(expected_modifies) => expected_modifies.push_back(write), diff --git a/components/tikv_util/src/lib.rs b/components/tikv_util/src/lib.rs index b1991663d28..b0d8dc7f739 100644 --- a/components/tikv_util/src/lib.rs +++ b/components/tikv_util/src/lib.rs @@ -395,6 +395,7 @@ impl MustConsumeVec { } } + #[must_use] pub fn take(&mut self) -> Self { MustConsumeVec { tag: self.tag, diff --git a/components/tikv_util/src/lru.rs b/components/tikv_util/src/lru.rs index b457e0f4c75..c4976261464 100644 --- a/components/tikv_util/src/lru.rs +++ b/components/tikv_util/src/lru.rs @@ -139,6 +139,7 @@ pub trait SizePolicy { fn on_reset(&mut self, val: usize); } +#[derive(Default)] pub struct CountTracker(usize); impl SizePolicy for CountTracker { @@ -159,12 +160,6 @@ impl SizePolicy for CountTracker { } } -impl Default for CountTracker { - fn default() -> Self { - Self(0) - } -} - pub struct LruCache where T: SizePolicy, diff --git a/components/tikv_util/src/worker/pool.rs b/components/tikv_util/src/worker/pool.rs index 01182382a22..66f1926bbb4 100644 --- a/components/tikv_util/src/worker/pool.rs +++ b/components/tikv_util/src/worker/pool.rs @@ -270,11 +270,13 @@ impl> Builder { } /// Pending tasks won't exceed `pending_capacity`. + #[must_use] pub fn pending_capacity(mut self, pending_capacity: usize) -> Self { self.pending_capacity = pending_capacity; self } + #[must_use] pub fn thread_count(mut self, thread_count: usize) -> Self { self.thread_count = thread_count; self diff --git a/components/tipb_helper/src/expr_def_builder.rs b/components/tipb_helper/src/expr_def_builder.rs index 020d8f3dd39..589ee1afbd6 100644 --- a/components/tipb_helper/src/expr_def_builder.rs +++ b/components/tipb_helper/src/expr_def_builder.rs @@ -79,6 +79,7 @@ impl ExprDefBuilder { Self(expr) } + #[must_use] pub fn push_child(mut self, child: impl Into) -> Self { self.0.mut_children().push(child.into()); self diff --git a/components/txn_types/src/lock.rs b/components/txn_types/src/lock.rs index d4955f3d229..67b70e2c9b1 100644 --- a/components/txn_types/src/lock.rs +++ b/components/txn_types/src/lock.rs @@ -132,12 +132,14 @@ impl Lock { } } + #[must_use] pub fn use_async_commit(mut self, secondaries: Vec>) -> Self { self.use_async_commit = true; self.secondaries = secondaries; self } + #[must_use] pub fn with_rollback_ts(mut self, rollback_ts: Vec) -> Self { self.rollback_ts = rollback_ts; self diff --git a/components/txn_types/src/timestamp.rs b/components/txn_types/src/timestamp.rs index 9f29516b901..1c2079a889f 100644 --- a/components/txn_types/src/timestamp.rs +++ b/components/txn_types/src/timestamp.rs @@ -33,11 +33,13 @@ impl TimeStamp { self.0 >> TSO_PHYSICAL_SHIFT_BITS } + #[must_use] pub fn next(self) -> TimeStamp { assert!(self.0 < u64::MAX); TimeStamp(self.0 + 1) } + #[must_use] pub fn prev(self) -> TimeStamp { assert!(self.0 > 0); TimeStamp(self.0 - 1) diff --git a/components/txn_types/src/types.rs b/components/txn_types/src/types.rs index bc00c5eb9a4..411866788d0 100644 --- a/components/txn_types/src/types.rs +++ b/components/txn_types/src/types.rs @@ -105,6 +105,7 @@ impl Key { /// Creates a new key by appending a `u64` timestamp to this key. #[inline] + #[must_use] pub fn append_ts(mut self, ts: TimeStamp) -> Key { self.0.encode_u64_desc(ts.into_inner()).unwrap(); self diff --git a/components/txn_types/src/write.rs b/components/txn_types/src/write.rs index 7e59157f4ee..844fc27df95 100644 --- a/components/txn_types/src/write.rs +++ b/components/txn_types/src/write.rs @@ -190,6 +190,7 @@ impl Write { } #[inline] + #[must_use] pub fn set_overlapped_rollback( mut self, has_overlapped_rollback: bool, diff --git a/rust-toolchain b/rust-toolchain index 08c09fbcd18..324e4c96b46 100644 --- a/rust-toolchain +++ b/rust-toolchain @@ -1 +1 @@ -nightly-2021-07-28 +nightly-2022-01-07 diff --git a/scripts/clippy b/scripts/clippy index d21c8dbf8bd..f0f46fccfa6 100755 --- a/scripts/clippy +++ b/scripts/clippy @@ -29,7 +29,7 @@ CLIPPY_LINTS=(-A clippy::module_inception \ -A clippy::unnecessary_sort_by \ -A clippy::unnecessary_wraps \ -A clippy::bool_assert_comparison \ - -A clippy::self_named_constructor \ + -A clippy::self_named_constructors \ -A clippy::enum_variant_names \ -W clippy::dbg_macro \ -W clippy::todo \ diff --git a/src/config.rs b/src/config.rs index 3fcb467c096..a5812fd3fbd 100644 --- a/src/config.rs +++ b/src/config.rs @@ -2964,12 +2964,12 @@ impl TiKvConfig { // as the shared cache size. let cache_cfg = &mut self.storage.block_cache; if cache_cfg.shared && cache_cfg.capacity.0.is_none() { - cache_cfg.capacity.0 = Some(ReadableSize { - 0: self.rocksdb.defaultcf.block_cache_size.0 + cache_cfg.capacity.0 = Some(ReadableSize( + self.rocksdb.defaultcf.block_cache_size.0 + self.rocksdb.writecf.block_cache_size.0 + self.rocksdb.lockcf.block_cache_size.0 + self.raftdb.defaultcf.block_cache_size.0, - }); + )); } if self.backup.sst_max_size.0 < default_coprocessor.region_max_size.0 / 10 { warn!( @@ -3240,7 +3240,7 @@ fn serde_to_online_config(name: String) -> String { "raftstore.apply_pool_size", "raft_store.apply_batch_system.pool_size", ), - _ => name.replace("raftstore", "raft_store").replace("-", "_"), + _ => name.replace("raftstore", "raft_store").replace('-', "_"), } } @@ -3363,9 +3363,9 @@ fn to_toml_encode(change: HashMap) -> CfgResult DagHandlerBuilder { } } + #[must_use] pub fn data_version(mut self, data_version: Option) -> Self { self.data_version = data_version; self diff --git a/src/coprocessor/statistics/analyze.rs b/src/coprocessor/statistics/analyze.rs index 9fe5be03b7f..d2a7e5943c4 100644 --- a/src/coprocessor/statistics/analyze.rs +++ b/src/coprocessor/statistics/analyze.rs @@ -650,23 +650,13 @@ impl RowSampleCollector for BernoulliRowSampleCollector { } } -#[derive(Clone)] +#[derive(Clone, Default)] struct ReservoirRowSampleCollector { base: BaseRowSampleCollector, samples: BinaryHeap>)>>, max_sample_size: usize, } -impl Default for ReservoirRowSampleCollector { - fn default() -> Self { - ReservoirRowSampleCollector { - base: Default::default(), - samples: BinaryHeap::new(), - max_sample_size: 0, - } - } -} - impl ReservoirRowSampleCollector { fn new( max_sample_size: usize, diff --git a/src/coprocessor/statistics/cmsketch.rs b/src/coprocessor/statistics/cmsketch.rs index 123f3019e95..c809b34c807 100644 --- a/src/coprocessor/statistics/cmsketch.rs +++ b/src/coprocessor/statistics/cmsketch.rs @@ -6,6 +6,7 @@ use murmur3::murmur3_x64_128; /// Refer:[Count-Min Sketch](https://en.wikipedia.org/wiki/Count-min_sketch) #[derive(Clone)] pub struct CmSketch { + #[allow(dead_code)] depth: usize, width: usize, count: u32, diff --git a/src/coprocessor_v2/config.rs b/src/coprocessor_v2/config.rs index 2574856f8c8..aeb92784e50 100644 --- a/src/coprocessor_v2/config.rs +++ b/src/coprocessor_v2/config.rs @@ -2,17 +2,9 @@ use std::path::PathBuf; -#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Default)] #[serde(default)] #[serde(rename_all = "kebab-case")] pub struct Config { pub coprocessor_plugin_directory: Option, } - -impl Default for Config { - fn default() -> Config { - Config { - coprocessor_plugin_directory: None, - } - } -} diff --git a/src/coprocessor_v2/endpoint.rs b/src/coprocessor_v2/endpoint.rs index a00ff39eab5..761facbde3f 100644 --- a/src/coprocessor_v2/endpoint.rs +++ b/src/coprocessor_v2/endpoint.rs @@ -11,6 +11,7 @@ use super::plugin_registry::PluginRegistry; use super::raw_storage_impl::RawStorageImpl; use crate::storage::{self, lock_manager::LockManager, Engine, Storage}; +#[allow(clippy::large_enum_variant)] enum CoprocessorError { RegionError(kvproto::errorpb::Error), Other(String), diff --git a/src/lib.rs b/src/lib.rs index 37d310011eb..b0cf7549efa 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -23,7 +23,6 @@ #![feature(proc_macro_hygiene)] #![feature(min_specialization)] #![feature(box_patterns)] -#![feature(shrink_to)] #![feature(drain_filter)] #![feature(negative_impls)] #![feature(deadline_api)] diff --git a/src/server/gc_worker/gc_manager.rs b/src/server/gc_worker/gc_manager.rs index 5450a5d843b..637368318e7 100644 --- a/src/server/gc_worker/gc_manager.rs +++ b/src/server/gc_worker/gc_manager.rs @@ -206,9 +206,7 @@ impl GcManagerHandle { .stop_signal_sender .send(()) .map_err(|e| box_err!("failed to send stop signal to gc worker thread: {:?}", e)); - if res.is_err() { - return res; - } + res?; self.join_handle .join() .map_err(|e| box_err!("failed to join gc worker thread: {:?}", e)) diff --git a/src/server/lock_manager/client.rs b/src/server/lock_manager/client.rs index bc068a4d929..5b9dda7e006 100644 --- a/src/server/lock_manager/client.rs +++ b/src/server/lock_manager/client.rs @@ -30,7 +30,6 @@ pub fn env() -> Arc { #[derive(Clone)] pub struct Client { - addr: String, client: DeadlockClient, sender: Option>, } @@ -43,7 +42,6 @@ impl Client { let channel = security_mgr.connect(cb, addr); let client = DeadlockClient::new(channel); Self { - addr: addr.to_owned(), client, sender: None, } diff --git a/src/server/lock_manager/waiter_manager.rs b/src/server/lock_manager/waiter_manager.rs index 9bfa443b7f2..ce4f48fcc38 100644 --- a/src/server/lock_manager/waiter_manager.rs +++ b/src/server/lock_manager/waiter_manager.rs @@ -95,6 +95,7 @@ impl Future for Delay { pub type Callback = Box) + Send>; +#[allow(clippy::large_enum_variant)] pub enum Task { WaitFor { // which txn waits for the lock diff --git a/src/server/service/diagnostics/sys.rs b/src/server/service/diagnostics/sys.rs index 399101a9a06..f33394ffb43 100644 --- a/src/server/service/diagnostics/sys.rs +++ b/src/server/service/diagnostics/sys.rs @@ -485,7 +485,7 @@ fn get_sysctl_list() -> HashMap { let content = std::fs::read_to_string(entry.path()).ok()?; let path = entry.path().to_str()?; - let name = path.trim_start_matches(DIR).replace("/", "."); + let name = path.trim_start_matches(DIR).replace('/', "."); Some((name, content.trim().to_string())) }) .collect() diff --git a/src/storage/mod.rs b/src/storage/mod.rs index 34a06b9b24a..bc6796cae73 100644 --- a/src/storage/mod.rs +++ b/src/storage/mod.rs @@ -6495,6 +6495,7 @@ mod tests { test_pessimistic_lock_impl(true); } + #[allow(clippy::large_enum_variant)] pub enum Msg { WaitFor { start_ts: TimeStamp, diff --git a/src/storage/mvcc/reader/point_getter.rs b/src/storage/mvcc/reader/point_getter.rs index a651b1b256c..2cf3727e197 100644 --- a/src/storage/mvcc/reader/point_getter.rs +++ b/src/storage/mvcc/reader/point_getter.rs @@ -43,6 +43,7 @@ impl PointGetterBuilder { /// /// Defaults to `true`. #[inline] + #[must_use] pub fn multi(mut self, multi: bool) -> Self { self.multi = multi; self @@ -52,6 +53,7 @@ impl PointGetterBuilder { /// /// Defaults to `true`. #[inline] + #[must_use] pub fn fill_cache(mut self, fill_cache: bool) -> Self { self.fill_cache = fill_cache; self @@ -64,6 +66,7 @@ impl PointGetterBuilder { /// /// Defaults to `false`. #[inline] + #[must_use] pub fn omit_value(mut self, omit_value: bool) -> Self { self.omit_value = omit_value; self @@ -73,6 +76,7 @@ impl PointGetterBuilder { /// /// Defaults to `IsolationLevel::Si`. #[inline] + #[must_use] pub fn isolation_level(mut self, isolation_level: IsolationLevel) -> Self { self.isolation_level = isolation_level; self @@ -82,6 +86,7 @@ impl PointGetterBuilder { /// /// Defaults to none. #[inline] + #[must_use] pub fn bypass_locks(mut self, locks: TsSet) -> Self { self.bypass_locks = locks; self @@ -91,6 +96,7 @@ impl PointGetterBuilder { /// /// Defaults to none. #[inline] + #[must_use] pub fn access_locks(mut self, locks: TsSet) -> Self { self.access_locks = locks; self @@ -101,6 +107,7 @@ impl PointGetterBuilder { /// /// Default is false. #[inline] + #[must_use] pub fn check_has_newer_ts_data(mut self, enabled: bool) -> Self { self.check_has_newer_ts_data = enabled; self diff --git a/src/storage/mvcc/reader/scanner/backward.rs b/src/storage/mvcc/reader/scanner/backward.rs index 6f385985e4b..95a19421ef9 100644 --- a/src/storage/mvcc/reader/scanner/backward.rs +++ b/src/storage/mvcc/reader/scanner/backward.rs @@ -415,7 +415,7 @@ impl BackwardKvScanner { // Value is in the default CF. self.ensure_default_cursor()?; let value = super::near_reverse_load_data_by_write( - &mut self.default_cursor.as_mut().unwrap(), + self.default_cursor.as_mut().unwrap(), user_key, write.start_ts, &mut self.statistics, diff --git a/src/storage/mvcc/reader/scanner/forward.rs b/src/storage/mvcc/reader/scanner/forward.rs index a13b39303a7..3c836ce6526 100644 --- a/src/storage/mvcc/reader/scanner/forward.rs +++ b/src/storage/mvcc/reader/scanner/forward.rs @@ -2304,7 +2304,7 @@ mod delta_entry_tests { test_data .iter() .filter(|(key, ..)| *key >= from_key && (to_key.is_empty() || *key < to_key)) - .map(|(key, lock, writes)| { + .flat_map(|(key, lock, writes)| { let mut entries_of_key = vec![]; if let Some((ts, lock_type, value)) = lock { @@ -2351,7 +2351,6 @@ mod delta_entry_tests { entries_of_key }) - .flatten() .collect::>() }; diff --git a/src/storage/mvcc/reader/scanner/mod.rs b/src/storage/mvcc/reader/scanner/mod.rs index e72b2860c1f..794c68327a9 100644 --- a/src/storage/mvcc/reader/scanner/mod.rs +++ b/src/storage/mvcc/reader/scanner/mod.rs @@ -34,6 +34,7 @@ impl ScannerBuilder { /// /// Defaults to `true`. #[inline] + #[must_use] pub fn fill_cache(mut self, fill_cache: bool) -> Self { self.0.fill_cache = fill_cache; self @@ -46,6 +47,7 @@ impl ScannerBuilder { /// /// Defaults to `false`. #[inline] + #[must_use] pub fn omit_value(mut self, omit_value: bool) -> Self { self.0.omit_value = omit_value; self @@ -55,6 +57,7 @@ impl ScannerBuilder { /// /// Defaults to `IsolationLevel::Si`. #[inline] + #[must_use] pub fn isolation_level(mut self, isolation_level: IsolationLevel) -> Self { self.0.isolation_level = isolation_level; self @@ -64,6 +67,7 @@ impl ScannerBuilder { /// /// Default is 'false'. #[inline] + #[must_use] pub fn desc(mut self, desc: bool) -> Self { self.0.desc = desc; self @@ -74,6 +78,7 @@ impl ScannerBuilder { /// /// Default is `(None, None)`. #[inline] + #[must_use] pub fn range(mut self, lower_bound: Option, upper_bound: Option) -> Self { self.0.lower_bound = lower_bound; self.0.upper_bound = upper_bound; @@ -85,6 +90,7 @@ impl ScannerBuilder { /// /// Default is empty. #[inline] + #[must_use] pub fn bypass_locks(mut self, locks: TsSet) -> Self { self.0.bypass_locks = locks; self @@ -95,6 +101,7 @@ impl ScannerBuilder { /// /// Default is empty. #[inline] + #[must_use] pub fn access_locks(mut self, locks: TsSet) -> Self { self.0.access_locks = locks; self @@ -106,6 +113,7 @@ impl ScannerBuilder { /// /// NOTE: user should be careful to use it with `ExtraOp::ReadOldValue`. #[inline] + #[must_use] pub fn hint_min_ts(mut self, min_ts: Option) -> Self { self.0.hint_min_ts = min_ts; self @@ -117,6 +125,7 @@ impl ScannerBuilder { /// /// NOTE: user should be careful to use it with `ExtraOp::ReadOldValue`. #[inline] + #[must_use] pub fn hint_max_ts(mut self, max_ts: Option) -> Self { self.0.hint_max_ts = max_ts; self @@ -127,6 +136,7 @@ impl ScannerBuilder { /// /// Default is false. #[inline] + #[must_use] pub fn check_has_newer_ts_data(mut self, enabled: bool) -> Self { self.0.check_has_newer_ts_data = enabled; self diff --git a/src/storage/txn/commands/acquire_pessimistic_lock.rs b/src/storage/txn/commands/acquire_pessimistic_lock.rs index 1ab2fd01705..777cd77a577 100644 --- a/src/storage/txn/commands/acquire_pessimistic_lock.rs +++ b/src/storage/txn/commands/acquire_pessimistic_lock.rs @@ -74,16 +74,12 @@ fn extract_lock_info_from_result(res: &StorageResult) -> &LockInfo { } impl WriteCommand for AcquirePessimisticLock { - fn process_write( - mut self, - snapshot: S, - mut context: WriteContext<'_, L>, - ) -> Result { + fn process_write(mut self, snapshot: S, context: WriteContext<'_, L>) -> Result { let (start_ts, ctx, keys) = (self.start_ts, self.ctx, self.keys); let mut txn = MvccTxn::new(start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(start_ts, snapshot, &ctx), - &mut context.statistics, + context.statistics, ); let rows = keys.len(); diff --git a/src/storage/txn/commands/check_secondary_locks.rs b/src/storage/txn/commands/check_secondary_locks.rs index e5ebf59de00..901bb4144d7 100644 --- a/src/storage/txn/commands/check_secondary_locks.rs +++ b/src/storage/txn/commands/check_secondary_locks.rs @@ -52,7 +52,7 @@ enum SecondaryLockStatus { } impl WriteCommand for CheckSecondaryLocks { - fn process_write(self, snapshot: S, mut context: WriteContext<'_, L>) -> Result { + fn process_write(self, snapshot: S, context: WriteContext<'_, L>) -> Result { // It is not allowed for commit to overwrite a protected rollback. So we update max_ts // to prevent this case from happening. context.concurrency_manager.update_max_ts(self.start_ts); @@ -60,7 +60,7 @@ impl WriteCommand for CheckSecondaryLocks { let mut txn = MvccTxn::new(self.start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.start_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); let mut released_locks = ReleasedLocks::new(self.start_ts, TimeStamp::zero()); let mut result = SecondaryLocksStatus::Locked(Vec::new()); diff --git a/src/storage/txn/commands/check_txn_status.rs b/src/storage/txn/commands/check_txn_status.rs index cecc0544ffe..ee27fec67d6 100644 --- a/src/storage/txn/commands/check_txn_status.rs +++ b/src/storage/txn/commands/check_txn_status.rs @@ -64,7 +64,7 @@ impl WriteCommand for CheckTxnStatus { /// situation, `self.start_ts` is T2's `start_ts`, `caller_start_ts` is T1's `start_ts`, and /// the `current_ts` is literally the timestamp when this function is invoked; it may not be /// accurate. - fn process_write(self, snapshot: S, mut context: WriteContext<'_, L>) -> Result { + fn process_write(self, snapshot: S, context: WriteContext<'_, L>) -> Result { let mut new_max_ts = self.lock_ts; if !self.current_ts.is_max() && self.current_ts > new_max_ts { new_max_ts = self.current_ts; @@ -77,7 +77,7 @@ impl WriteCommand for CheckTxnStatus { let mut txn = MvccTxn::new(self.lock_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.lock_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); fail_point!("check_txn_status", |err| Err( diff --git a/src/storage/txn/commands/cleanup.rs b/src/storage/txn/commands/cleanup.rs index 87e9ae7e9e5..0b01f4ff311 100644 --- a/src/storage/txn/commands/cleanup.rs +++ b/src/storage/txn/commands/cleanup.rs @@ -39,7 +39,7 @@ impl CommandExt for Cleanup { } impl WriteCommand for Cleanup { - fn process_write(self, snapshot: S, mut context: WriteContext<'_, L>) -> Result { + fn process_write(self, snapshot: S, context: WriteContext<'_, L>) -> Result { // It is not allowed for commit to overwrite a protected rollback. So we update max_ts // to prevent this case from happening. context.concurrency_manager.update_max_ts(self.start_ts); @@ -47,7 +47,7 @@ impl WriteCommand for Cleanup { let mut txn = MvccTxn::new(self.start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.start_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); let mut released_locks = ReleasedLocks::new(self.start_ts, TimeStamp::zero()); diff --git a/src/storage/txn/commands/commit.rs b/src/storage/txn/commands/commit.rs index 5857d9f9ef4..cd9774a1751 100644 --- a/src/storage/txn/commands/commit.rs +++ b/src/storage/txn/commands/commit.rs @@ -39,7 +39,7 @@ impl CommandExt for Commit { } impl WriteCommand for Commit { - fn process_write(self, snapshot: S, mut context: WriteContext<'_, L>) -> Result { + fn process_write(self, snapshot: S, context: WriteContext<'_, L>) -> Result { if self.commit_ts <= self.lock_ts { return Err(Error::from(ErrorInner::InvalidTxnTso { start_ts: self.lock_ts, @@ -49,7 +49,7 @@ impl WriteCommand for Commit { let mut txn = MvccTxn::new(self.lock_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.lock_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); let rows = self.keys.len(); diff --git a/src/storage/txn/commands/pessimistic_rollback.rs b/src/storage/txn/commands/pessimistic_rollback.rs index 381562c0305..4d8113d53a5 100644 --- a/src/storage/txn/commands/pessimistic_rollback.rs +++ b/src/storage/txn/commands/pessimistic_rollback.rs @@ -39,15 +39,11 @@ impl CommandExt for PessimisticRollback { impl WriteCommand for PessimisticRollback { /// Delete any pessimistic lock with small for_update_ts belongs to this transaction. - fn process_write( - mut self, - snapshot: S, - mut context: WriteContext<'_, L>, - ) -> Result { + fn process_write(mut self, snapshot: S, context: WriteContext<'_, L>) -> Result { let mut txn = MvccTxn::new(self.start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.start_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); let ctx = mem::take(&mut self.ctx); diff --git a/src/storage/txn/commands/prewrite.rs b/src/storage/txn/commands/prewrite.rs index dbdf583c68d..46bff4d0915 100644 --- a/src/storage/txn/commands/prewrite.rs +++ b/src/storage/txn/commands/prewrite.rs @@ -389,7 +389,7 @@ impl Prewriter { let mut txn = MvccTxn::new(self.start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.start_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); // Set extra op here for getting the write record when check write conflict in prewrite. diff --git a/src/storage/txn/commands/resolve_lock.rs b/src/storage/txn/commands/resolve_lock.rs index fc395b39b6f..0c750225362 100644 --- a/src/storage/txn/commands/resolve_lock.rs +++ b/src/storage/txn/commands/resolve_lock.rs @@ -63,17 +63,13 @@ impl CommandExt for ResolveLock { } impl WriteCommand for ResolveLock { - fn process_write( - mut self, - snapshot: S, - mut context: WriteContext<'_, L>, - ) -> Result { + fn process_write(mut self, snapshot: S, context: WriteContext<'_, L>) -> Result { let (ctx, txn_status, key_locks) = (self.ctx, self.txn_status, self.key_locks); let mut txn = MvccTxn::new(TimeStamp::zero(), context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(TimeStamp::zero(), snapshot, &ctx), - &mut context.statistics, + context.statistics, ); let mut scan_key = self.scan_key.take(); diff --git a/src/storage/txn/commands/resolve_lock_lite.rs b/src/storage/txn/commands/resolve_lock_lite.rs index 54698e87e5f..ccca17a556b 100644 --- a/src/storage/txn/commands/resolve_lock_lite.rs +++ b/src/storage/txn/commands/resolve_lock_lite.rs @@ -37,11 +37,11 @@ impl CommandExt for ResolveLockLite { } impl WriteCommand for ResolveLockLite { - fn process_write(self, snapshot: S, mut context: WriteContext<'_, L>) -> Result { + fn process_write(self, snapshot: S, context: WriteContext<'_, L>) -> Result { let mut txn = MvccTxn::new(self.start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.start_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); let rows = self.resolve_keys.len(); diff --git a/src/storage/txn/commands/rollback.rs b/src/storage/txn/commands/rollback.rs index b8c69bedd1f..c05781c1734 100644 --- a/src/storage/txn/commands/rollback.rs +++ b/src/storage/txn/commands/rollback.rs @@ -35,11 +35,11 @@ impl CommandExt for Rollback { } impl WriteCommand for Rollback { - fn process_write(self, snapshot: S, mut context: WriteContext<'_, L>) -> Result { + fn process_write(self, snapshot: S, context: WriteContext<'_, L>) -> Result { let mut txn = MvccTxn::new(self.start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.start_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); let rows = self.keys.len(); diff --git a/src/storage/txn/commands/txn_heart_beat.rs b/src/storage/txn/commands/txn_heart_beat.rs index 273d886753a..9b28af1a180 100644 --- a/src/storage/txn/commands/txn_heart_beat.rs +++ b/src/storage/txn/commands/txn_heart_beat.rs @@ -43,12 +43,12 @@ impl CommandExt for TxnHeartBeat { } impl WriteCommand for TxnHeartBeat { - fn process_write(self, snapshot: S, mut context: WriteContext<'_, L>) -> Result { + fn process_write(self, snapshot: S, context: WriteContext<'_, L>) -> Result { // TxnHeartBeat never remove locks. No need to wake up waiters. let mut txn = MvccTxn::new(self.start_ts, context.concurrency_manager); let mut reader = ReaderWithStats::new( SnapshotReader::new_with_ctx(self.start_ts, snapshot, &self.ctx), - &mut context.statistics, + context.statistics, ); fail_point!("txn_heart_beat", |err| Err( crate::storage::mvcc::Error::from(crate::storage::mvcc::txn::make_txn_error( diff --git a/src/storage/txn/flow_controller.rs b/src/storage/txn/flow_controller.rs index e63de54fa61..dbb109b5d61 100644 --- a/src/storage/txn/flow_controller.rs +++ b/src/storage/txn/flow_controller.rs @@ -91,7 +91,6 @@ impl Drop for FlowController { if let Err(e) = h.unwrap().join() { error!("join flow controller failed"; "err" => ?e); - return; } } } diff --git a/src/storage/txn/store.rs b/src/storage/txn/store.rs index 1cd94a822ec..cfa910b944d 100644 --- a/src/storage/txn/store.rs +++ b/src/storage/txn/store.rs @@ -70,9 +70,7 @@ pub trait Scanner: Send { } Ok(None) => break, Err( - e - @ - Error(box ErrorInner::Mvcc(MvccError(box MvccErrorInner::KeyIsLocked { + e @ Error(box ErrorInner::Mvcc(MvccError(box MvccErrorInner::KeyIsLocked { .. }))), ) => { @@ -175,7 +173,7 @@ impl TxnEntry { let v = WriteRef::parse(&write.1) .map_err(MvccError::from)? .to_owned(); - let v = v.short_value.unwrap_or_else(Vec::default); + let v = v.short_value.unwrap_or_default(); Ok((k, v)) } } diff --git a/tests/integrations/raftstore/test_replication_mode.rs b/tests/integrations/raftstore/test_replication_mode.rs index aa8ad427881..75f043647d3 100644 --- a/tests/integrations/raftstore/test_replication_mode.rs +++ b/tests/integrations/raftstore/test_replication_mode.rs @@ -449,7 +449,7 @@ fn test_assign_commit_groups_with_migrate_region() { // Split 1 region into 2 regions. let region = cluster.get_region(b""); - cluster.must_split(®ion, &b"k".to_vec()); + cluster.must_split(®ion, b"k"); // Put a key value pair. cluster.must_put(b"a1", b"v0"); cluster.must_put(b"k1", b"v0"); From af4be9c09ddcb1d40ea5cd8fde69390a1501f36a Mon Sep 17 00:00:00 2001 From: Aarush Bhat Date: Thu, 20 Jan 2022 11:51:46 +0530 Subject: [PATCH 08/16] readme: change images based on github theme (#11795) Signed-off-by: sloorush Co-authored-by: Ti Chi Robot --- README.md | 3 ++- images/cncf-white.png | Bin 0 -> 4072 bytes images/tikv_stack.png | Bin 22315 -> 31133 bytes 3 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 images/cncf-white.png diff --git a/README.md b/README.md index 5e503acf283..018c09e0499 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,8 @@ The design of TiKV ('Ti' stands for titanium) is inspired by some great distribu If you're interested in contributing to TiKV, or want to build it from source, see [CONTRIBUTING.md](./CONTRIBUTING.md). -![cncf_logo](images/cncf.png) +![cncf_logo](images/cncf.png#gh-light-mode-only) +![cncf_logo](images/cncf-white.png#gh-dark-mode-only) TiKV is a graduated project of the [Cloud Native Computing Foundation](https://cncf.io/) (CNCF). If you are an organization that wants to help shape the evolution of technologies that are container-packaged, dynamically-scheduled and microservices-oriented, consider joining the CNCF. For details about who's involved and how TiKV plays a role, read the CNCF [announcement](https://www.cncf.io/announcements/2020/09/02/cloud-native-computing-foundation-announces-tikv-graduation/). diff --git a/images/cncf-white.png b/images/cncf-white.png new file mode 100644 index 0000000000000000000000000000000000000000..88ab8ecf8d12c274f43ea625169e79ddf3db43e7 GIT binary patch literal 4072 zcmZu!Wmppo7aanlVT6txAn=M(qd`GB2QorXkdn?Za>Nj6l`=@FfuzKcX41Tb^gv=r zN;qV6DfRLG|IYK=I_J6f{ywkZk8~JlIcWg^0E3>crU?K*#&q4wPy?^?hqcz#>q6tB z`_vBrpkw_n$N*W{?AIijpNWn-plXDhcs)?Ks2QpO05!>U=k}BU025kIQ_T!TMzU6^ z7F1-eR->ZPmDdF!5Yh1&H)#gkLdm5*g$D6PkYM7iL3h=&+KhG=fE23O*83~y9Vt-f zVWcJnp^=$o;4oHK7h$x*qAFEI+~+bfob@{KHyU-jcwdoQGaooBo<6vy!jSU+fbBHc zH`BNN)puiM4@_!{R0xCN;y}sZ+S!WeYv}DJb@Ni{Fzz+QpqBS?a)V{Bk3cyVUN&-D z4rXbHd|giqDy+}CzodMYBmXx?9!f8EQc-8?7F0OlZFk$XMauSctZwgI9SBK z_Q%3v?lTn-Q$pA&vLaG9<2DFr2(ijcMJa|ne{uA|x8OIy!a9B9m#|weD3zI2-xTcj z@em!mLYW>cJgq%VCdYr+HL3Ud!_F8@sD3lj@*PS02t=V?8ueHZEs63hs`X#{e0)Z^ z|5GVnROEPpKpR^AS8Wx8=&%BM0!ZRVMxLcVSZ{Kbk9k-AOou|0`Rf2N7CnF*0~s=( z@=%~yqK9k*s3xN8h+8}2QR7&$LBB+t+g|iVpWF_&l3iZ1K06H}bvOS}9M2JJe78xx z_9})*)PP~;H#I5a)qs_tyY>pt5+*U$Jp1Sjkc?NG{elZmwHReP7|Qb83qpCykg77a zt-O8`M5Uibs{A98N5kK z?MtVQ`)G=ZCC-LJ3}{fb%7U#0x>U~YZk-&P)&09JjE+2}q_yUp;b8EIB|;d|=Rr-? zhM4}XEwDfH)AzW2vsC2M9EbcNddfqOt3#d{mMs{Ig0Y#_=YfjqPXy?A-L3= ztAphR|7trU%7rNM(f!YE+P}@zL%e&~!mm`b3{-gyu6ie27pKw(mvA0FRbXTfweY_u z+TinQYs$*03D=YHj$0m8_v8tRI%fMiVo}L}&LL&F6zvMw#eo1n%Joc#hDyCYvc+!U zYq(BLZFJ=67v}|G!B2O8LyU+gIv>CAY1ag2X90{!&lDXNoGjK%O3?Xv(@<_P4r=X6 z@ny~Bun>8R39jQnRrSW_3zYP4)&Og1TU3-=v?->gwgW!jWM9)KxeCj&58H)2zoBU| zpQj4YezKEpb}GR~&oZnMIngbB%=^H8EaI0Qmi|7Sq;ypCq=*^-*RzN5#@zT^Hi?p}!QF7dRfle?~W!xQs7|``j1jHpc(&Y<> zcD?1}SeA;HzO{ddm7Y03JG!r8s6aGA5~FE{nq<#|M#KfyB>6ZVZQNR16%9|vuvMJw zCn>n%m?_79!RqgLlK*LWeh&ktkAr4+la0Bpd`R?=q5F=YvHrrOR2b$y(o>Hg7_&7K_AD|fjmP#9lT&l~{8!4T^_$**FgLac* zc<+x)k)#Uz@#8a*52nyonbf~5j1-1*J{_GxhPY=487Clf@pk;=kv?8~sVd`Y%2 z$b&o|$eRO;wl>JOjF_c95}k1LR#FVlQ`Ef_O zybb-cd#(U6cJWG(k8?9KKG2P~d1-pOw3*D1j&>c*Kpwfpi^+;jTp}h$m_BJBmG{9l zh#+~CJ0TBry-Fi0vuDG^%4`? zmS^(V?~kyxYuGtsNl4%}6=bfbzyntuej*SuH94-I)@OP4^OuQ+hX4vp7?%RS!}*tH z_uFkGggcm3N)@+q2!zN;h+nBhjUhMAp~~`Dayprb#er>~95gA>TaYI13?Y zujE-nk<>Jw2OLdb)J!{9a2Zo)EM<;5H^lK;UA3Fv(ClWPlUmPHYhJ?HXF2^QmNN5-na=VcWv1r|cO{nJ=0 zn@KZ|48|Wm?r?P_DeAe4xd^D!>zsyjMxc$akie#K8nROQ_TwH8Vdz8lLRNT|P|q^n z6`JcWa&gMmwalii{4&5ia9ok1w8AEFphn3`oxMplCuFt(7|gn0oaW-CIj7V+Q;Qrz z^Z7^6WpY}RUbPSbg!w!MRD)JN6=gr#mC};j+Y%e4L`_}=$7eFnz3%bo1!<~U29Hsr8m^HwLj3Cm{a<+e;*s;h(xQ=r|`tOBx=?XR|z zaJvtEJ`I%U17F~SFK#hXB;Ix=+C`=uQul`H$1L1S!)=%nTHa2?Lt#6v^;5r zf-5b|GO<9CWUSdWQG$Ykhuvy(6Fd`%QBdtH8XB7JpG&o3dJ%$xUvt=3b=v|C!=VaE z2X_Pov#Ei&QHQC6uP}?O|McvCo^|~!;X_dTEef=WD&5}VRmX7*_8ySSp)WLWwseeTS5jS($vvbi5fq@NZ(y?AtNaqe@Ul3ugJfN zkoXa8STi+hpEJp`C0KcfNmC%z>S>T+9)5>S+&J*W~p8aIL}a->+)c*Pmdxx`BE z-ee@<$>?2n@p)@n|Fk9Z&UP~>h*St8*}i#^d@e2IP&vb}Yz~2~BTDw*F=)uXBUlqJ>zym)1BzbCl8;zDB9oZH@h6=t+=rd*FP04P2INa_Y?XMY8WojIT8Mjw;Q(h z$KdXu#}4beaxRz!N54wiD1*8B4F0P~s>2Xn%H9fZm776uD7!hMxsuB&J)!#Je1Lwr zF6;DGc9;foZ_uge4zJsp@0QyTJ%Ovs0?H#reLj;$$SodznxZ%!x?2J^3&jqP`~)`O z^yTf{Gv1G)tL46iF6tx`S_fW?&6m5MX=#FY%pnU}oneg3>_?nkrp$ch7x(8PFIRSU z+zCEW*|Tklsb>i*X;>$?Qr=4>?=ow{27lVOE3qk~im!CkxMtBtS1A`+tQr_ZMhJgF zB~pt0s?q2Swx-wO>7nie5?Z=jx_p<+LzLEeeNE8}bL6jkZ<<5E=8%(stft4j5-Pow zN5y`cZiSHS&rJxD^J+>iY|yA6t^{Y=UP0pA7Z*b;HoT}Pj5!*K4SFR$u?|~nOBdXH zpWD|2_`^mK?}nuvjm-!y4j5T`DJ#+Nb6bY(qMC>x^PU0kYP#`g#?8+bU)FVL#Q|z5 z`veTH-aI`U7p<6sZ7!*GRL=e|1~t4Y8`Uwl+KWOqb8dI#4#Ew43&4~@v_trU4eyYW zK!nuiUyA)g8a;>@y}5O3cRG5H{=vTYMjC_Y3c3{Ii1XZFUabgtH+%2rPYSUdW$ztV zNm9qw6Vl)=_Ym(%z>GvtHFYwT+L^(lK`g?rRA&R=nFO6 zPLvNXbse~+X{=-~HsbTdU-A7)Vy4FDB+hk2{$nn>QQRp%>FjXMx!3~A!0F2I$fN&`)8%w)R z9y@IGwK$eyT>gpvK9+V6`Kvi1$+wR-S)3dqHDdva%k!AtFh7FJOd%X&!7q{0I@xiD z{*ppcHjHAdi^D1yu6JG-Hf=(1-&H5nL!w&^s6+5g8T34t;unqDeG|nw(|&t7^=2%n z#K>T{&E8+{NZJJ%7%?H1<(>WB&kX}o9p%~=v9r>_dy(R7q$Z!VTZegX8i_KcauNG6 zVnLJ;ZYP42Q0al}VbD&2{dXJRut&(m0K!f1)yClo;jJwL&Hxk=)WgskNA%VfboP^E z7O^E44^YE&d$@N!4;LF_(4hQmtIAI0m>{?O*Zu{*lf zs@#Gho+<_saq|r#A&=h5Q7`$O+m6}VHmhLqca`>fFMh1jkUapnYjM=UTyJ7=N{?}` z7#>tsDpt52{oA8lBEz|ByXk>fH3P*SWL{Y+`CV|Af3P@Ii6ea*!N^hZj+Jh;h_Dfb z?W`Jw>W*TNi*aPoxmHFHL(|NyQ9ImZrm2~gsYa(C8IBZh>OJuanU7i?kUmW^!`}m) zkIcfda+j`#PCLUlD(x~5-98T`><4i||GI?*!<%bb&%9XS+lJ4rJLZe>bXr8dGu3?T zNz1L9au4$FGG3Zu4eG?(m}azEPz(l~%mRfxu_m5T(_~IhL{5f(1Mxsi{qumbA0Lx_ z(@I)jW8pZiHHGYi(uR-CUx&&G!$uOy!J;uia4z5apAI_KyN0L-A3L92-pHkOdiLO( z^WP3HYD2b$l|Lz0cn#L#^qSMEzY2|u?|+3`$uRjXLzB#BKpdX3@s_wd(jFh7;#XyX zpSs`LEx&=Gw3e$YWlr_qDGPl`tg62WF#!&d>-g8w{OVO>HL}gZv$v4m?#3z`E-3s> z$MQUGuj|^?Fi4KCy3rsYTl})OQ=%hrTC;GvN_8zXpJ2QoIzK|3*T)ll2!#YDq0B%T z8Ed+|6G94J)kak%Cs9 zff=g1tE;}M+TY&0B9s&)Q9j{+f`WoVk(Lrufr5gWgMxy#Li_;v#z4NX2lDHqqm;G_ z6ciHX`w0z|nT-qi652&YQUt1citrHf0nSobUKk3hE)MC%7#<4B6Cf=ntmXlIy8ba+ zevbg*Vpt_aY)E0KJJ!`~v)+Ns%#cFNO&pOdgh9K20i*srtR*8dKl1is zFm*bS!wZ^PJI&_pIX_^_=$Ja~zVAHXp7z{081np$LH>S3Hn)n1y&tT9j52>h4w^7r zIZO=5F^C>)^{6ejX09A@muHr*Ec(K^%N`E$)`y@c9YmNuWq`ltJ++vBT?ZP7N* z>*BQzcSU>a`!4^tz|*=>15fcpN^kK*YQ<`BmR#ajX>w?2=w^RRDmj`m)e?=j{jn6; zb!v{N{u<1ClE4hM@kk=cPzIXVyMD%mpHgMIR8k@$Oo-eF!q=Ce;l19#VCSoOX>N}# z$&cPIEoPIssyl;G%vQ4&@m!ZX(npPlU#ssNt^(#0Nrhpg697$}4<<0uch;Yzler?? zHa)H$E=?b=<9M$Q)&+NKdWc7IHtk0#ZqMx=56;7N;{8NlJqzJ?I7RkI7)b?Pk3tIN zlQgdO#_9X|`c``$3xxtd)>0%4k*cD?j|$2euvXa8Z`XTSvV*-fX6n6(y8e38p(W>+ z5|X`*y!%3Fw5*aOtJy`=EVbsi;6^Z#74lvB}c<4&vzzk{3`pG&?HnLqj zu7%bM<>j^mBrNH76p%nz3!vl_6TkrmA2wJ{kNpvkGFvELIJ)O7UreQQSWF`yl#%Ux zSY)IVDjT_zw=th?MydB-QlNd4fVa}>>7M@^WJ8azh+GUte2=m#ggVxA0q5PHJ}o~o z7#{8G*1G%^>C`JIux#cNM8su(3VJ!db)U?@99I!&jJ&S%7I|Fm%+?yiVriKs6r}u5 zl}1Rkt--8SD@H4SEMhe46@m<+5mgZpkzMfFI;CZ`-qkahw&iYrB6#JWeIi#p8#e~SDEK$s)MeiG~nEWxRMCSHrK^@#y^WS0bj5#l^S*j+T z%AiY$w0NFcY`2MYUqq*uv&#dwz=mxXFUO4}<@k5W95fWOBMS)`n#DdGglN23&2p@9 z%1{GuZR^wt`b{71zi!_Dn&^*so;UTIt`RmB9>jLFHz)F#YjEhvYSPNc=Wwc$=B?(C zJU?`I7Ms>U1AhF`ob47xIQ-jC!sqj#(FmaD~nSZ!aN6$J7wt18!B^q%6< zNnMH+lMg`>rO$kRFIv?Guj=4j8mG^m4aP1`3w>R@=DPv8B4V!y;NP3&n@SSV4|3h< z4Qe`_K=l#ciz1HerL)u|OD37R&y?yp?JbZO|E;Coe}B59`c?Q(hz@C&(A6$8!y%xP zigH{%5n4y6%X9tH0%)E_I0#Oq(Bsf?X?q-AgE^z-m7V7a1pRn@MpAG2Y~lL6GDmIO z0(QPC<@36cf?p8Z736rk2;QB<`5KW}lDb0QJkZXp#|k_DJ1!d_dBkoKgR=71_n}H>GuYv1;?_n1V_BP5XLGg zH)TiK49%{e-n!-TZHtcRj`^Ku8Ok0tC{`NXLfgCLONbL0JU`(9+%nMc|0WH0Px{ z9o_BO)c#~ZyRx~$w2Cz5PZ*DBMdPWXB@~jQjh^(6gYi)3Rh7artWJ4?M#L@Hg)R z{xYqC*i!*o7?adnDsmekQn25WJwP0(@R5D%FK9ARWMW|sr#fXcsbuWtaBeUOvUn<2 zs?F;wkU#K~;*p~{UU&ck5t_vcD*#Trq*m7kiYe@5 z9kQtdd99&Odb_QTN58C_Fhlv0XB;aPSlW>bt{qwm+vsfto}%J zCEzG$b){bX4>}V6?XVeF=vqQ-pnnk0tsCpee|R~lG-;idB<|9kObiZ>=xKGOP&<2Z zlkJ2zUDhBN@5}Qcr%z?XbT@{?9%PLOv;#Je9PXPI=vvYCY&IU3WseVCO|m(FjX4Ur~@enI;TI{%Z*RV>C=G_}=OmNzIo_9?V$i)*WLI(kZ)J${)~SYj214ovNudHC#? zv1^;eHSHxdYHX2$to~q^{i9Td&c4922`yZ2lB7R5BqWo-mi96$kqbqV9aKln$|U!* zLD#@~w`r#ppMRAhRx9U<&+eKn{2q>qZni7=?bt+oZym{HTJjq<%8J(A3tD$?&Z;u0 zHDBk3W0daseP1jOfo&-|%U#dAu7_`j9*3K^M7lz8`JSM1%6dnz{fg5`{n1|B7LV)d z%~6(#YKe4-L2ygH!1MYUN!dwb2TM8?wPOq&1k*BIA55otFn7~9sMSzxgy~qBIo*<& zxzI0^(%<4YMLQZM2REM0tEL3y7e9%M4wE*XRo+>^rYwY|o_90%Q%@+}uh&|a9d*Pp zTvlaq?oNhH&V!u@NSDQVm4s?@xouEsuO-x7jRIq-(1`gaO&s*P)IMP|mhs?ipMy5} z9+AeI&zF`BOdH&JWa~9mO94@dd-OJecifjn)~*|=GcyGBP9O;3QD{lCaJNkMpc8(7X#=OmX1HT~mT@O2{ ztxF++v4d+z^uloT=QppkUMdrv@j18$9=Si)NK3MuAOa@IEEOvf6GqP7WUe&rdZ=ZR zmn&RnW48kG?b6W9c1IFan#zxnNb%$_Jz-!O--a$Jue{Tk3{!tojz=()_Quc!ERt2M zb%?s)-K=~qQ?5y9w;xOY<-C0|*Z#e6s^9m=2#?~Ty~CruCYI72*ev=}HT3Q5fkIv_ z{9iAE?gLM5E5I#bL5Z(v2WkSb>zS&*t-|UNhz1 zk13j8SAwB{n__pJ!h0t5_S1C-ul^mnsf66-*ZY$ybtd9Y@IBCf<1Pr$-!c5d69GB` zDj|)yjAm*tUufvdQaYLM?q8A?GsvxXKBao1LlQE){0Y#WrfMV>tLRZ@F9Y}PBI6Q{ zDBwRy8lZPco*GSxUSr@@cM54DRhG%Ho1ywWwX4H*;S&3T0aMv}*;YBRX+D-M?yDVB z@xGEV3=yJYh=)KO-%TEe*?4T7#_bF~YR(o)Q&jI5r;!L3+;NWYVgjiBrV%ilh#eK4 z8oL?m5Rs9S4P)OJD94TivC%rgf2%6eij4Y#cIZCL`R&dY8HPf-RSy##owO&Hw+env zeSi*vkeD@5oS<}O_fY9*xUcu)uADag_{GOG%l78e1|G5`(O#4F+MFXQwDyG|d8S!B zqb>)x=gCj4(elA-^W%G)-J`vzBg_7B%Gr8*aXd~Rq2xGG#Umi62qi~*J`oERjgpYw z;KX~{njl;xlVWz#dTGM`pwW-Y|Fpsc0wU@z_=-ZHTw!WT z1!OVoQT;9U@#94H;I?A$2b7RXc+jJcQGS13`z=?lhI*xk)98f`1I_EF&7bc3H?)t>+Z1*6?{FgvC99&6oe+E(@YFm20buVl`rgfDUXdM`?#L@S z;^uUgY7O-K!q)9W19rnyGGPwqA$FG(T^17f0jK^=T)LLn2`C6Y(2Z zX)1@iK*sC+{T_2{mPE}5WAlhMG{TX-F&?Zo9oJhQu!grd^;NlshHb>NcM!$Vx2RsX z`rT$$E+0N*xq>~P`(ebAz4Qd!_3C$AOs?y>6ciB+<-|hu8uC8b%Y@CGD|n z!wouqS2Jr@Cn6s&>FY~$-7bvxjKwk%?1HAL%JRK;dkp}mK=mW~UbM6IvY@L_g1b^3 z8@~OA>2UMyr}ab#@U9E$_imBy_Bne*W@CG&70gQ9&o=GCO549b zPtV8_Ea5Qg8#GtGF=OigvS#$;KBU)KeA1;=YYh47%`ODLatN?c34DNHP}?|%w2B1H#hRLZaZ;qjWU4f+c4q{4;w?5QD@ zI(&*9YjWV%gLU5WS1+ZS*k%&AolygMV{hBUw4_M&6TKca125WN5OgHD76<`jki%`q zrf(;m)wIz2o*3Zo3m-wioU1hns%2SNC|J#!s$!-!EGkBRF3qxqtN^cmH&Q8nP%@mK zx(rmpV)BeTm}LZ`h1<(vwEAysI4&y%|B4Ya?r(X;^2m}S(2`sQafBuatwL7Hv^waZN%Mx{6rnqsKpn20}Io4?atvpAZB@p!2Q$35~|j6zJ|`)pyv39yBE z%@r{K!Ur*u_&xk;j|MskQk?iys8>)?NJPhfmCmic>T*BNMaBb&SxjZqx}U8n-xnE)ojKs&j~ct#wXExdS1q1w>mODswxd6h7GN> z>ARWU9ZHe?FG4NG9kEJp?D%m=etZ131ba90OVpY=hfMT?)na8v<^+bEeyd}jTOMdQ zu435uY8Vc!$z~}PLZq#9xYs<-KZb$dY5<0s>0tRBqaNj7FFA*WrezWB52$ptR?kQCE*jSw(Gcxtr+a%{^sS!$xaYs?Q zbPkVe>pe*y>;A|ou%*HXKVGetgn}@CM+o;mkI%Edofx6g3kD{TeSa1bRAoj$q7|#p zu0vmnpt-KuUaqNC>Q(H>j=5ZJ{AUlqclBPk)^P(kdI3^QeW+MT&)4{Q6kD^fmT(&n znVz!USR?lYaYv&?;Oln26Wyr0uqms~|rP zoXKvZwh|7ugr0VKxG?@OYae#m#|4joAnDlyq4HK`wpk>&8qEBu&=5wsj)#R-yK3`9 zSr&T^yq#aR?8$7n{9ocbftB4n|K`!u)?+fPBC)f_h7Y}o+`74{;wFkPj()i=OB>A^ z)|*ANN6*Di1C8U|Ns-DR@J&Z28mB%%f~$-0s*>aC?i6^$Hq+{4>WX(CcVaX~WH=}D z_uxCP_{YG@6O3ht60VN?G>0u@_#nU)@_Z!$j=pL1)7>e-*%qd$5h2A_rrMaJJ!dHk zQcU@cU?XH`*ZwdR^OHu;#1cnVvg=Ra+sYp{n|e8Rw`!y)$i=ieO7u~quW^E}N|=H$ zbPN*p)6pTg*hNO+Yd!w{z*szJ?D{-2VlJZS(GN&S%HCG~i%@+T0z+Oo>_cC)Hthw5 zPQRFj<6HmuT5Gwc;1po9rhxHp+eJ7x@Uh{Y-^Ir-A1hSiC}Kum_@$sjLSbvkRS^ED zA?O`$hat@6i4z1c<5m~q2NR5{mbS(}>LGE{pL0|(LWVP1OV|KrD^B6}QS*P3T7^;= zv}&dC$b?)h}fhi!+&O>&U>|vq7W0B7?gj9&;dOTa@_x{qebPD z(_;u};NP~!=AU6h!?I;>n6D{7U}QzJYVwC=1e=xtbXiKzs504Gp6?ZL|Y;zGi<|AN6+Z{50{*cKNvjua4&xBIe7)eX@=(8O` zxmi#Ntd-H^w?SNMDyM;Pho67`P+A%%5GW{4HvRPrV+c0~fytngW|w!wB$kj{*x+I@h=_e88WW5TLnEmMES_+mY9p2=;2!<0u4BmM*Y*XHK5o2{EJ}g! z&5OcQ(;JwDZ@*X+k5Z2O&PQNeW#Iv~j5Xka z&{(d8_s`TPb#Ma5mzn&QOe>9jT`|BaR7^B$;NOqED~pz1mS{I55<&rPhLDLtnY}}+ z*W!eDo5Ac*#9>3N2caT7IgnB14J^fjin%ULu%d{__cSaeoe|crce-Mov=FgK|yV+$#c)G|+yE zc1tHt&}#A)&`aA+&h5T4DT_4@M+WUmYcy~&3ljY@&kcp8M-)`-D)G9 z&C5|mO;`tU3Ekg)TK)DPM~!AmN;58s#|NWHk_)aw5tx$1d>;8sAS7LxR0dks1ak3D z+D&dba+w`^I&LQzhv3YnV{)wWg#^lL%hCA0JgGCiI`>)8Dk_<8(ItTp{ONn2)^w{RVFL{m>DRqkV?c*V{jiz7Bdc3m z!085@vH6Wjtx$<-#v-v?%)%)FKOGF4+w=M)%iH>*JNb*vVj`Z9+eDzHXlDPCebwh_ z&}Dyaf7NHTb<#VLw=~~&rn%=5WO{Iy_RUJADdz=zPjUFSE$6ETS@$meO?&)(HeaLe zhRtK@mzjO`Z29p-$|w)a9_WOZ57xfC?#E1X`&G%kQUF-Av&}S)TRLmJ_So4j8yvFh zKOSz!8=_Rc4SEFA*$Erx1KYl{6{1LS&HUu|!i$YwxA?QQ;@(y(qfYs554{HOYzc$@ z3(@erWGa&`)w%mAmFn+w#qE+{@7;cdszx@QwO1%4iG-q8>JvjoS$_3di|BQP``o+f zO*HeV0gQU*755{#%$h}ARyKJ*SiJ0s48W-Od2{M4R=E^}w+QJVy0z`%;Ydjo4>tct z9UMA=0?LpcH*-6kk$})Pg)x=`9Nx%qmmWVA(g6+;a;$c2++ zif3!(Ar-RE8rIP#2dYM-L>N>1zfkt_{5Mk0&YM!cu4aU-Jw_q68IBy9&Q|DCNMD%n zsV^oA`ukmCK}PqvSn4n0m7eey&*yfCc#?g(#_+WsnH|>3MikgpJgGBM#p`Z9apORl zfc=9do+EDrFppfAhlb>*+hRw5e=ZnbY{x5`!bu>*cjm3E5#N_!o6PSloE9LYj z)P<7YWX0g(0m2Dg%BtTez&Jze8 zrz_3nzNS}?Gia8Pr`0VBD^$obFrk2(q4wjU>1e4;i?K9SEAsIvwwx zS=ylcuezN9*KPi#@3wxAUhS@3txZOy+l%NUlYx$b9D0L8)3+qvp7qo&!OEl@UCxo* z9Nxn3jIs~7q7a{pYg{o*|J1JgUko7 zD5twNj|U@dK<@e@dXZmf%iBZ6LS0Hcx!K%@K3GoYra;j9SouQ5-b|mUN~Ie2Xi80h z*;IlgOoK(-6_V=&RK@|4rGG+{rQ&Ft>hNw!Kf>-!gel!~Gzr|ykONH9JC_1wDBs~T ztW0+OKXSL2XP#VOV7>DJE164~K93ZWe&@y7v-w1_w#NC+oh66s*lto38JY4%`2SRd zxd&E zg0Y2L6a?QS5zYl2t6%%Y$2B2y?C%h^UCFdB86=sF8m6Qf$dG@;q|~TF0YRksve`n_ zIK?#9q$f^N^KwVZum3`QAq1wiveCO#*lwfgm%`<=GjiTyUmAT);%+}~xC$0&oVXrM z8m1%_m-+Z4ZfJSPxzY5loe?cI2Au_Pi~w5 z-DnS85N&Il*#Ft0w6XFztbIE7G;84j4E|>v($ir7NX|Z#&=sbwIxOD0@wfSm0hE2o z2hl!gCaqIcU3#fW+!7G(&!e!oc-@cA$%atlSOHTxreQZJ5NX2gp`|uvRZzp9?{(u4 zBpBYY(Yh|FDMu1@I`w)j{e@rvC-N4;LsD`(Bb05j;^hiBsd@SHsk{?Nzn)rDCfgIq z)va%Nsa6rt&Q^%yMqdi-Pj}}&vK1-){d@(SBmnFNPtCU7LwR1VfS`q!7tfcZ&V?x` zIzRAj7R6s*In~4hY)0=tGr!Ko?Ncz)IDYBm2K~CzGM#zgTMqPJ?dh;JXc570^3tsQ zE%~;6sN2y?-fSdSLf&p44bP7NzCJOY`w*%k95jQxd!&FEuQ#F)#(<2?WqY=W-|peZo>l z00lRC8r?KH44g!}P@|e%fWDti#M3rB5rINk=U1k4{P|m%`B&K!?ga^d+qFd4-S>ky zDs{+DH_kD)*^PWy{DQsQ6CW#Dfcpx*x^WTdeAw6bBY|CC6<_~|e0d)_ScHerv%h>h z=D*_W{g!KehQnf@W2W|{{9{_jCbY$8@x-UQrP3`Uk9VicX*XOi4z3ops7%J*EmFwZ z*oLU9?x0om6kn@Ld(8-Bm>vp*`?96r{J#(FwR6Ixmx;|3&Np5Ow7kAorErCERR%TP zAmPO(?W+3E5RvuWd!!`|AwZiqFW<7?g+)-fQG?XwEC#V9iPZ7irYk{HvOOdX3q}H= z5<$BK@1IMVU2t0ZTxQ_AmjCu#mS;@Yi%5#?fSv_jdG z8e3kWLGOsS`=)W6Fr?zkH9-;qbn}2nyEy^UhI@!5O0DH+3$2M9>R4hw9(QYyrChSn$+NpdN!RPxkr5LNp?IA{)(h@9-w3U5M`Nrb zzA{=N^@I)~6NEy-&6rV-Itc9$WID!Ek!0z+Ct($)iXq1TwI&)aM6d^^$*e!`HbF2? zYj;8l6=$W?mhRU6BW^6w?@Bn$F>dhC)U4a#JMZ(LPAckj0RpYZK$oX%gQj*r;wi%|7q7OnffRmUOkGfWtAr8ni3xYE@0r;`PD`Y&m)6I1)WDzAqH zGwKa>M!>=(x8wZ^L3|^5STpVzX#UPH*k?&)nfvl?Rv2cE5B&DXZBa}s(QL0*K(&E@ zM!3YWXg{MS-#TJOrW@%cB-cG8*F=FhR?(%ssc~i;vbA+>3^qTDWNcQANZ-9=^E?A& zaM*`@TJ*jlDYk#!ACIVay!&y>>$XPFG*@WjPNJzTC&Wkr(+aUDsn+}zDA=8L#3$;* z*oaE|Oc~+m-9=TkfT}+oTcvxMqqYFZeSaDk|9b7!F)Eu`MVe1VQ0<-|p$Y;smw61) z!)p7P&F3XXG*P|kBxcUQD!?wg*T9cm^5%_-TRn+1ncFSFc!B@unxgulUpC63-kVz)n>H{o_%j0jdf#>!BdwU$$gnxsgCTEKaatBSTBL;hjeO| zI1}63hDa_)9f(Yha{bwuH7Ukr==IXi?|&Q*q5H#cT-=CL@G04v(z_t(qeh85C$t`zpWz6U;~a z1YQJbb?e$s*#i8>HOi}pPH!#?j@Zl?v}ucS0E^oKBeln-;XiR zD}z2rnhU%_^`QV}*|4jU_H>Ggg&X+iatJiA=ot z+#d4rkArmLi}Ri&XF>z;F7QM+ehlxUG+zSi%)1!r%liX?5R=5!rDc+OE%uz3VFPbT z49Wh~-MK$0iK`t>^#@`{$Da*+1e(n1X-?>oZF3qL7~-io2n`SlAj;7V+7h%!<_IT31D-1|K_saMq~5~qm<+h2-7ahe+0fy9uO7d7_PB~3vUwz*KC##x5HlNe zBHp7RdD}kocav#J7R%q?_lK8b|84E>J=ds+bHjn_9FajI5ERW^U@Acsxv3;0#r^Xf zOyu=0ig&SnXjf+aU}jGL>b{E8mjUie7RWV@>;LUD1#wiB1+$s51oR|Pm&TvADK`h5 z(A4K7EbQq@d|@xfMd)fQaLJ3?1LYE#ZHMu118xxI5+VjNpa@6ETiy1Bu%j(;qA!St zV+j17+3e#kPjd~K)BY?&4~^qG9tLs){&NO$(Nl*w%)qpnd}eATyuR=#lSy0 z7Hn#~l=_#>p)L3=iIX;(rLbftYXz{-ct;M( zS=IfPy<7s0N=WJR+Q^Ji5z~pRx}JSXoOz6v_I4$&BOGpstrr>IFARw1n_XRFsgv55 z!MCthKgpldduuN$%%ATUo;mY%I~+62;EEA7G#RohhHC6h4~C~b-1@ZcqkDn}ZFu00 zel69=#PSJ$PG<~HU4O|tWhc!S;rN2S3+hDy{f;vXlhd}%3yq*a2-n>V7(SvzeU(D9 zu0tWe7|dXQZVtgJpI5W_!s7n>PLPKP;8?rR>WZM7@xe$LFf`kWZT?nm*Bn4F?Y~xg zn)C8ld+NT1Kq>xD?{^%@zI-@fJo+H^gj~{iZPMwY!_To0 zEMzA8to%CFYPEvuF#sOh#FU6oGeoY|{rJ&s4DqPF*@^~YMAN{0T06v4qh}S0VbUoi zqv_u?c$xGMBvf#+TnuvhQC6y5Q}!L>r z(&<`HYq$xRXuZH#QJFU&57sO9p~ONikjI7LU+eK*LYuc^;R7gtjb^#+Y5TA8+DTg$Ksi=nx1NPz7q&sqEPb zvo#ARyU7tg*4^My5GKUa2(A|=uk0{y5isRwc{B0 zDCeNcr|{(1IWuXtg?&}T$OllTKX{`OlEFuQQEe%b%h4E9t1LG6x+@FSCTiN!@t383hwC;U$-rcPFrtY(b z-iXD~`LEXw*8T|Oxq_Zo6@9*&;T`_*2OSGnLok2r%lXjXDMB+%yR{z=`o5`G*h-dI zpZSo`UuOVjT8+tu-f@DsfYmzz93~B!*u z%Tcq}3?27FqYH>i zqLHX+A7Gz%*O;^Ft{Mnpx8yDiVIkP>(Jv(%zzVvyGf)DMEael3r&sRQc-&1;=QgOr zT@D%^RqXO~s5V=3Ahi;jl)OI&2vLVJCjPO3bOWo$a}`WipQqg2K?SM$Md3mV=?v$}xohDQwmS@x684 zlTu}8@J39KPhn0pn^|SBS*Gss#(okHPJ7B-WW@YRMuzjdQ?a;LNW1MiJ-%(8iRfz@ zlYz$>7+@{@e5AkjI#H~U;p5Eu_3PNP3lcMKzM)d|6^Cqf2g7VYip0762#Ipn$6vCY6-FYHUpn-4oz2q5KY#y zqe?o7s<`VEVjgJ*LhKQk1;F1bU0$xSB*SqaoKd@6Lcyt=#f`h46G2s%FHlr@NH|Q8 zHsy|Up~h%~Wt;OO6%9ic*$+ake=73P6Iihd`K@Fo49;eVJ6>w}fltcGwxaO)0F)`#lws%=AMHH+cUTym$VV zU%*a8XW#&NF)|<>@ISGaITEaA6(iZbXKllG#rVN=(P9%nZe(H^@?#iMJxpDYHi0e( zBcbKJ8e6#Okjuq`ie}7UvUw+?T&T4_LX4Pb9`tOBEeq7!Ov6z)iU9Lz8dn{SwE08T z$cl@X^E(?_ZD;sMYFoW_b?-^1B9(RL9mc8Vq3at3-|NWJde6r@$0bU+WTh+YQA&rV zva+FBtOLc+ksZ8>-TpfL9B-j-SwG%)s8FH4sOa3=l8?eBMsONgp2?L3a1Nah-0A{O zp7pn@qewWbccqC)&y#6rXp_Ev)@0A=5TWW+A7q?1Z@l`)T)+Q-*Yu_~qMMLaGcuVL!j|KpD{|GJr)P7!8Q?d2 zt9AyiYmLb0p%&K#1eb7|hUdMxi%3j8F1tNW^ZJjqhh{A(m^`gc?+wCLpORw4_gw5m z?R?bLJWrGkwvYiS+z!~*qIkC7s{TvKay~zUb*=r~;j>*nk=H0@uiwL%^Pv$#(`XH5 zwv#UR;=aJ&?5Gy9;f)vf`|66f;WSasqDL-q4}@V#1e0m*0#jY>me$+fge*?qq?_|P zXpVW?a|g#S#>VM{+)LoYW5+{I-;#{k$7So&K9B9{BNQD(P6AF(k9MpqkcL?N-Tp1f z01n0qnFL>s6Hc^RBq#G!HDcVj$rAX?0n!gDFqd$hcYl4Uq#g05;f|*JA>+Y~31QBb zao2leB>M2GCzpC{FNpF=x_@c&F97>MEfo{}C7a7fhd7)+l`qs~OjhTES~xPOj6x^% zXv0vY)+VaOr!dmm<@sPk8l~OP zd?KrCyOiPe43nw){x{9Olh0X{(wQ`C4~IR|Ikhx;EHU@sm&2$;zCn5nfW`e=C-boX zi#Uu5bsBQUTm`ETdW=xMLKOb?nfK)c<;N~B*$zV;=@Rp;=;=6CLI11YYp7uPAcJYf z2t$RF3w99c7RX()v`3tL5T?Het*vq5M+)uc7&E6|5@A^`@dV!tpcjEzKM8L2PK8Aj zUmE?brvWwY4Xla!>LpJs=t({ZC`z(%KbKuCjG)ZL7ISC3Z~rR)u-SKV+i|~Z_RsF4 z)Jq~pcS6g3Ctdz0lO!i+ooY@~DB^gV$0XA9c<+oNQ6HB2GL?sftdQL8pX+ zSxMe-mJ2m$JcoU7O&a~z$*~{Fus}4~s$%td*K~>m=x*n+ zUB+zD$tJG;5bkD`_4*@sqUHrv!JGxuABORI@b$n%B7ZB;ecT-v*CohGIz@nVtRWDo zgfR`vWl1{U4Lk8Cp=m}q$%TXk13;5GtlU2V3mQ%dwrL9?i7jYTI@ofoSGsH7Ty95{ z$F~?zi0F3^Ygl;t#eO_wFt6OGQoiV3eV>3gi+$6N`I?F`RfIBV_o!y6srSY3@#gr| zdiZMLpXU+xMN9qe)YmuQUBM6hWsUO1nym|V@}ClqeY<3AgTRRm<*m%SjSqzi#vPSC z=Y8g%#)_^r=M|okn4imtSlupgft{f)Wkk0mI;3DFf;sK1o*B1QOS3ge)rIrrKL1tU zY_BuOI3C;Pm)witlpW{w&8+D8Tk=S%i;a3u1av6(>2@sbhb3RzD7I9kHDJ}U-nD1= zvQg8ssUBEskQpe{HlBz=_fujk`U>5wfvxK)a&4g=7-PXa2eS`%^h&{Ts?p+bzN!M> zzehFI1Qu|5hJL)8VWmysF33jgq8-KT`pQPIqz27G?0$JKIDc&}m8#~J*YlOax`)g) z$XU$8*Ci{Q`6J?j0H+5k8{F*HcpIi)kE^7eD0G=&lRojXUtwo5*ROW_KLN~p;J)+H zX~F16$9fqFBZcRyIqs|bO@G;O`xv#u)lvh@B9$)Zqx(0jzQ+S_acSF4H;1M{yQ6rR zKf92fX-Cf{GyQ&Lnt$W)4k?zlrEOk9PR^gcD>X)!Zu`F$ZNG@typTHT;vxG)i@yu^ z7K#dae_z7z?C>Sh-M@4>y~B^8lLf^pj?^xKR61>woAn?U=rCHC?#g^3 zo_ibx(MQl@p|;oYy9&Rz{b{ZwoHA+r_tgAERMJ)#yd=mbvHU3My4aTlS-;MMQ*>>6 zHhpheKACx|(-gyZD6%DzRIZY*HzK>j?9}M^$ z4Q8I&E5B)2rsy9kWF(e6-#l6X-fzr&{mkySgFy6%yoV3<7U}(NB13+ z>sWtBY|TGmeWagDv7(QORXbei+yL7Nw33Y%7upLohC!ky^)QAtG^++M{1O$ zx)CRo3ba#sonrdo04Tli%A4soUZ$doFAXU+q(Bn}=)c?Z6hUor(+c!Euk84mDtn0S zB9zxFxd%}e1g`Dal~B4NX1BBrY$=J>*o=nCWfM`F42+unK>OQ@U7uS z`)Z2RmwqvTb{zai+H`v=pfr0>?}L4DYNX~Yg*iIaNz?j`yrx5C%PV06zmXbM6C|r@ zkO)%f?^&@BWwTbbzaH9nY4f|QUGV5k=~A|-B8cJ1bzF~wCL#T=Zl~MJ@zeGat%5@) zJWgk2jCZRPd)Z`T?m)43Eo?TnVAV~l)GXd^Y5+{gs3#U51G`+$F0)+S-~6;5ozgr` zvobR-InfTr(2f}I`=*QZn9DhwpJ5D*TTe6+K$;Yx8aSdBT*n4Sbz-59Khm>m7Q@qp zbAnCVy=C;5oXYK)^f?rcxqM48ZAKW~USi5E#4#cX1JNxL(u^HzB`Zd*i$)joX!EWf zo64GYUkc%SJO~xUgPm**f9|UBpV|0lRYa*C((Zcb!pfpVx4YiWE|G|=iL#A z;f162xeu#|&!3m7axk%cTT~%fcUPJwT=C6ux;a9z>T#^<)TfENnXmt6v2zO66lWZ- zAW2<8_f{RVx)0vL@{|CfIc)tBQCQSlZb7mpXg5)i~*Lb@5@ruOyr2!|8Wqy zf$iquo%sy}(rM-1RAS=^<4`#@T8hks<<&39S_j_rSO?;{;g0wnDY*qqGB znqK%km9QoDNtQrhnVclF;SZu-u3N&t%BZI=(oK`W0pDonV_S_oc$zOoAE%;L1CibK zxs3h>L#S4#gyzZC$_FnNO;XR`B;N6I~ASBJY7ky ztM^-_7BZD4+Ew>+q?5^d0q-#Y zcFLj{Su^`T#eHR5)ZN;(fP|uyx1_WP(j^E(3Wy34O1Ch8bV+x&w183)(hWmO%Fv)R z4BZ{lG4Sre``qW8-}8QWpHJ_H`5?%D_J8Mft!u5ddt+X*=}RQjoJkaau2NjDDrCFt zP}P<5lKOgx7>Qm>?Ni2nA}(boQ!AU>`h7=D?DNS|w)kJf=yGI&GM{=iGUOzRQyPEG z?W^Qo1w0rMxfg0&^yV-jXArI)Y#T7xx4qOi?{)RGj^}vzgw|E4!pK> z6B21XOg!#(=5yA()4|*7Y8cWHlF`_)%I3QC1w-0!w$=hKwFJRO-AX(-M2g{8_myGiZbB`aS;zjEC_CRR=Fjc# zs5L9-e>|QgV2cU!C1q6@m^oIBW{mTWam^sSNAfzYB5k*p2ro= z)r8I*#zV|b^$Ra$Xw7`9Z2A)Kn5Ti*xE(VU|NQN&{OQ%`<8Shxl-O7+R8_i#B676F zi>_?swW*Z87q9{j4}++1p`q2=0xt_L{SmLlvUF{ULri8IBWzWcd}8ADWU*Rdv+zLK zqx2E0TpE|(MG|>e2LrIX=7q<3i(Paaj3>qZR2mCUm?FF9HrT&BiAhN0*IXc5;ioKX zQ@as{N6ECuq@0!TP-ZWr;L0*|^9)(E*b%QxE~F_I9{rJPAG>>b38G|m@-2BIU#iIk zPKNg3ld*6?-5n2UKKAiEJg2>C%T~d$5!kFqbuk_ymdBfKITE{uDrj)B=5uVv;=ItW zAl&M%kz+avUs>OyJ#%_(Iu#SCH9P6>w^YA zWkr5C66LJmIo(q13yv((#%5x8Yj!pPmFl z8}7NCE@=FCI+TZ>j-<`QrBy)k@8NatqjO^A6Dr}d|lni=HQ zqgpu3WNKs;PJ;V6q1vxG;YtrI#bngkws1=2NDdI2_L`U=l#I*6EAjRIya|*JfCOs{ zTu(z=YGMlfnt7wJPv4r=kHKd&7TWmkz-1p|9U?&h64cYP#9^_w%>N?FAnI_oCz2~! zXpW8{qC1lP`{s5urNp(104+*OIpesNJ+9x(Vp^^$N+$#u2+*iK^V2o2z67j^CHO&RZSkkal z{7+7i>)i#)#o`NxdpgIH1={*la`1gt#F9%fK+F04_iPh%5A7kThDcHTvvbvBL$Pqm?ED)%h1z(k*mx0|7~l z=fV9@gVU9iG?i#~(bJi@w=ij2tfy}d%`&yZPz!b*6TJx8Xz!JPy^I6JUkm0CTjv{q2@fkwyNB>mtz|GCR ze~Xq{atTN zoieLx1{802AB-F)93Eua#w>?sYvF(I-0H{WmM%pK%R}uL{Y>{6t3>-<7g*Y{pKZdQ zskD4Hp~KfV6WRbDGXqW|3)JkK2f^vB(2gfsHx1?b9GK@%67NA?{|zajkgaO5_0%>isE$nubvkkzy0r-|lnMetgkN>oA!MmW6Y(?(+?=Q00l1lU= zEL=TbBcX_88m7!`?eRwF?DqFa>J?Ty( z{ZYE7ug1=``AD_i+NjEk`s@h1`4oe}!)<5Lx(D{&!m}d|XEX~OGu#hd)hFiw%oWZ3 zlJ0Q9vT^d{wTI_!yrwQklqdj8mzgzZ_En4up81x(X^_N2EI}Nv1{`+dp<-3!+y}mb z^UOk5BMjeLkl{E#PiV+DDlL%?^342H#ppM0BAvc}A$Ew@8XpMar^(O{!00m+O^#MS z_l%y@;LMY-c0Zn*;m)vmeanNbuzEfdMt`LT0*dL%tm?rL9+Is7u3n;8_PC2&%+1m5 z>G+!7nb-ofP6j@4nyvTe5QnpnFtRbR+RA*UqLzK#vE$sEMzXZqwn(^k?j$tn=0o{( z%wKyC7Y8gVkv02P)h%C)?|<&o{)&gCE0-XqeFy!8(S~%89wFIH z6lmh>vyeFg19xBRQ(c`GS7=tN^v#@UTlm)X>wfws_4`$72*Z!T9&Se>0bgClZS4B- z>08Z3zb4qEvsAiY4ydwR({xJyFx2xGjlgIg&a;M^o4PjLa&myCoQsg^Ln{3+D;*qiyvZXJ< z7Q;(N?hU_+H0jIXl|=G(Mi=!{Kj3H`MFY(eJB(~uA5iYn`)tOY3x`IvReKfmj@5{|=GNR(^2EO!xj7<-PNW1~;@ zE?ICpBKZ=#mJ@FgivxA0xEmVT8g-xF82bzWz8+bM(wuYXMDf0idC~sHKMT5?4dI67 zKH3p_xFMgSTs1uGkkj$vxKQg2yP5iSTcCX23|e-D@irvg#~(!ena2M5(QCu1Ll*S> zTZ(9y+)MmBQ}O&|*m;O89=j7)wot5>SxHy!an>J|9xv|BDeMKTR>1d%PvV$AD_hvb z7f&lT^LkJ}G4y3s-_Mv>)@amzBUr_5Rd`8SZDUk5Gxy@8eUd8s5+1HdaDBPd66l?n-{N{WJ~653L}aCGL*Z zv1osKuJq7A;}G&H19V3Y=2~OkR$hYx=fpF_r`g%!(Z!zLnMG8(d}K#544bUn+`yBm zD9e1oPW@i}{r8I7$FK7YM~W}@vfF)4=;5SL2UJM`wE`gONNmW29CA57nav@`i;@$4 zEtmdtXV@<`Xzz0Buzksfr*W2Zd~udzHglQYE6Dvj5veum+f$yImZ``osv3{|qT4vu zrlg-dhhJh&ZMk^T%Ty*;T?%!;m6>BS0n@87kmg4fw!d0U)I zQ2t)gJkCGFIzwYYU;9eo_%fcmd3_TXmT1}CpjtI2Mv_JM^d#w5^(SW6mIY5F65syZ z!NPUKbEihX93YBWuwn5DzN2364CZRAN#jU|07mC@zjZ696GQF?UD&U=4JkcKp#-V% zVRtqKj)ZBX8GTSe7zl)#z9g33zT2MXS;`~*s6f21rOeKmI?nwWfso0Fin?fVK3ny3 zIB5jgui})h_m!4P&rDU^KGTYqBy;De`Zbmsj=oghxM(YR#9S9Z$<1xxWGM^@7pwjC zr5UwCm~&`onN^Es6#H+9nt0p)s5S3*vk6G#X3pt1=X0!?Ak$|Im*wJC=@4n%#*H7w zbG1I?W{zyzx_f)^6DUvQ77V*>&8WXN%deFegWnb}6$WN;l|}as4urQgigIqZjK25w zNuk_O*|yhAcBm;xRI(<#e{%qbd3OR5f9uoOZA5gJr0%)4p)=u6y7zDgAK$J-T!&6? zd+OYQ+hVg}78u5X-+HIM;>jM~;d#ZuO>)K5GBZ}aRXveX+KUcr^cB^O?VLEuw&L$_ z+Cb>neakYO9?0<>7+fjJr=cuHX5WiX?Zf}2bv1iyS|r~~j7(i;=u#q1D^YnZ&SS(V zz`@!~h=RAmNZ=OvY1V`YUy@L^MW9LM%c%NwnD}sa*Ksy|gM)eN7rEs4sVU7|S?)VI z8X7L_Gf;#>$z4Y$*i;pZqC&$kY->gQIR*A;tLZ0I@9OOFD!4%|b|RTC&<+nNqx+b! zZpuVI^|s(5bl7!K>87&8t@$9M5zlH;hu7?tLH-(=ZD;uUp5?HpLj#zvmUZR@{c=f6 z-a`23P8YncGLU3-!-Sz+jh`%`X#jY|x9@f7<>l6MSG9^u%6&7|#S*FgJTlo}Y{)u7 z;w6@x#KKT%gI&Nu@l%~)PYsB_d8#mcO@DLIK3M0(YCV{Qe?927XVrf6-F4j|!;S2E z;a-5}9c*dNsjo(xMJor*e~M`NQRpx-1o11L& z8|!?;sew6*&K3*A`(b+0^(WnqS?|N;m5B?KTZtw}_yxK&j0Rtj!NTrq%ypA!^CQIS3^^X^R1bXE7O)5XV&-iCc-4qm@c9X@vR-k_HlG&bAvG~v<=LI zXgsk;2*L-8be#OP`|TLR;q2&OW>uF;ASWc*c^|4 zqzWZ)O!}UaT>J6Tby}+X1-7v)i@}ESZ&MlzOr{#K;J3%(Gr3Kq#?Ay4atWutTOv7c zM`X&$hnUm*w$mnWAzGXIRRr90lA4>zF@^*%(5+)5(khs1`F5V>>iAmz-n3Y^udOqC zA$$58*O7oG1BxDc_t5^9n)TD;CdJBdw2>#V9cI7Efnj?iy5Ws~Y>&Q@+>_)#!MowmgUJ7DYCAxT7LbxZd8vSn)-I z^22w>Z?Cp(eU%=OdGioIMl(&a$y0AEv>f)qOTCv}xZ?gxg}ecxjfu_lSlp<_bCjG& zd(~6BZWwyVuN?2wT{V=RmjJCXXaOn3Cau_I=5y!DrvwlqXg=dear!G}0w_I85(7<5 zo--CJbiGo){I3i1Z(RjncUaGq5czTZUv78&dWM@_@@p?ccj?Wq%V#94P|4&+9wjdn z16gb9&eb9M@DZn@0FT{o8Xbt8!6j~cC{LvxN$N3!8GbH1ACBK4ERynbe4IgcQI4RA z+vVZx*rkkyHQeZe(JG&t9op0%Fd1pOw4YnMvD#7_I8n%$yCt7&Gl1^QDsluTsY(F3 zxA4u5rLymjs0&ZUN8TIzohh*vjbOzlpM;+!Er=cLyTmEahErDr50Tu;@F3UyIloG? zU(U~xx1w%yKoXbrTpgSEzVv9=_rT$WeA!dJ$-|5drHab@gTG*KdX`||tirBmCbZvG zU?{-8NyXIQl}0wsakKU8TciT34EvH!YbrgWrWoIL{Z)4O{n5O|Q{k}Wr|3otRmtxdJfp%)%%`3Z zzbC8Qkxx|(C88APyUSMpv$s{#zVDZ^<$zla)Nx&FrDsgazOl!fY_TOJqK)K%h39IE zx5Y~is}^_bk=5e;m2OZKN@mr0b3FH-pp zu*L^qMJMQZ&{DbP>aDamO4EXH?5sA?RED=RMRU7i_s1>VSRVUWNLLOU?ZH*MD;~BF z6wOg?CVOTLuJ=(W4S2VL;DDGO-x|-XvFb8ee60$iPt43 zq*TicG9S(9q*$D5^~`kZOyUx>H)pu~FoKo+QISX3sTHr1iEwM{;K&i`F*WUc7PsJ1 z_~A5KKx^hw_UClpeU(HAnxQgIrs8iu;wXW)4SA-CNP9Lk&4nmX*?d&oQdLUU7}YzI znENCpKhkidagLLZ)4)tfHLV`Kxm*)pZ|cE|Iv&2`_TLg8PB|UUc8&jJPJ85{vyw%7 zhjdJ~Jw1TY6$de}GvW^J&=hLS{4CVRoLD**w3Qsx6ACzIHro$b-f?bWCy?=Q7=tf9 z1(^KG2_X0^v;!s&3Yz+{^x4E1F=~#0y@pxMP;J+GdEI!dT9mZd9WS3mNKE_)SU5)- z0I?}#r=C7bCjJuupaLtrXS_W^vfj z&<*aJ@<;e!;&!Ep`?hG0X3J9p+kh}$vysi32vJ};+@`7g!w)AVe8EsCX>~rJzx0czc(!Re(46p*C7i{ zn0XXE-szj|`d>Pq0T3Pol;B&V<-}m8->WTV0iKa}(KN&AK23vsutecjWmYGG^WZ)rU z3m68)w+7F_fc0VSGDXDm0IL6Leobf`ZSS3D83udOkmqL2BPz z5xj|T+Jax2J}k|p$qS9SWO%L?k5VvaCaU!LbMnMHlhmH~mYD7{ zD=9Qr@ErWUFJx8cumzoK@n=qW4yS15MLz%@7F|Hl{bOouy0hMT_s1K2I=gv*cP2qP zU)4{L<7eILU|ulZnf}gPE#h%ez2yLP3&f{p=V-hrQ!3I@2R2u^WnI7gS-E{wZ?@F} zFO0yth*At3?hzc9@MHOyN8v7b#&ef~Lh+ZiQI##?=v|OE-(+*Sr@2l&Lv91Gl>}gi zs1IPVp*~S4Mr4N0(?=zwgbY+2#0IpqX@GSea5wM6;$mlvX*v&9b>B12P`&YLE}a^p z)5ogjIXF#<>0c$*0xqkf3i&=clX{M>F+4moIMjQ9jmyL!ehUZq52IZHpAf$nfl>-k zkzasNw+#4Fm%e1C+>lfc6WlIVh526%?yhlh&v%2H?u%B3Nd+pt4#H83n&QK1t_41h zd2+Pa_*Ajlbq=``>m4+9VT10~qBzQYJES zHa=q7N3sbD-|;RyZ}_-fc#@TmZ8!Ch5-J{m&4|u%SA!EIU_LKhfe)B68uOWy!8P@v39-v8GbVY4LYhQ zifqB<$Z4k&16E^E)%SP|{td)}xLnc*X>yL2Jtmp;yO}yT>fmN}A-S4`pG16p1SH2G@i&oG0DBQnAeSZ!oI~-r# z6^HWO`O(uk3@SjHS`~mJJ3si@M7|~dTbeJHk7=ylY^-8i$o;l^Kezi8Z-rKse2;M! zKoI%_)uTwms14kvrpGskZl!1KC998Z#aj-=`W(N2X@cKLz}BT}o`HpOdf7Y`(E*Qe z3v~}E?0xEJ%i6iA#lEk{b}!0I!EWO{#qIBx`AIW8N*#9GB-8|J7GN^83;cAee&6fK z8TqJuAHf|l=C_s1q~XLK@?s7}VQm6DUQ|mGxig5QgI)lBEZN`Z)^8eP!P_O%BS}fZ z1Sr&82BQG=m0Wb!Wg9chV6D~>(?1kh(4~uPz}Nm7J14u3s@(ipV}H|vyAw4;*?qVf z|L})xjV{2~#M^8ys$+3O;(bjj;PmMU3v^6&A#$LiLCvnQPhH^2SZ-i+O))+iLw;O6 zy{?Y{KY0|3G`^UddDBDszztQ}xUPKVaU2;B=9Ea_2L*WJ z-|!Q~gHqn6iA~LgQae(>FmC2W9mizCE)`B19a#w5m2F*a%MJu=qe$yq+XY`c-SP`p zt$#J0`het{`jYce`Fc^kuP*Mocg*Or`?cdI3Lka$ExBjvO^3~!^s&2Ke}T;}azba^ z>so=qvf;NQIe_V9merogmsf>!&gxVewgu47Z3oWUUzQ=uEOt1Eh>xtQ5hx7(QxQ^e z%3)<2WBhpGG`kAmeA3|=sJDTsSNeoX{g$tZXj4yuv%uSpst;>#_TtJ(EMEmwbXvqS_Ds?- zOh0%`0i%rfPA!b>3UAN#A||-G!p{F<*~I&3_tr+H zZRl7Nh0UVS_5-rBnJfD3Kt#J7;^E-3W&CCqwj@rqJrhM|99!?;D7meKoXPmw8>#XT zeiWHt=|pky?*bx<2n`s^X-}wRgoE^p!14X2(ZhX@_h!0trq1Tb4memLS7G#&+aQj z+n2+?orm2`mAi^l?lJR{vamFY>Yso4IBRc?aMj3*>t}4kH72yOcK8)j1&>G%3>~?^ z-k9m!Oh`mc#wZ?z(J;)wEZzQZ+ygi{{hL&I?5#)cjXC~7GNZ^GV&ohn>t^X~{itM- zlR>H6x0PO7r|T*#tGhci6x#W3B5!BXzRt^hQ}|4WtsFTcURC<=A*X?upDmFq&I!ln zxf14@FqS7HSf|(3QQ&YQW)#-bDN7c|`#9!RegMN_@}n&1Msy0ZwdJqF4eMj^t#U;W znr7J~r{{gZm~Rg?u45B_NGWDL<9DldCZN&3NvQUKh9W{*065^9@m;Z^9{ndmysoh( zW}(ue>={{emM@Mev1|JdIU588)29m@%5bh6W46P430(ffV(oEWsxdi;W9mzex_b8= z4!l%~WWEP%f|6K*f+bCR{<6u7L*pB;4R(N$ML(3InQlTc#0PGem0N{NLybUGvuMzu zans-59iw(QwxKm%rZ+_VO_38G8$ln@ zP?{?MOX;9xU+p%26DipQ9Z-~NF~@bX?SI-<9p!i7QA@`#e7yDflZyQ`M(wDA_lFtz z_|6_Kwl)eBcfI<3cu>86;-AT$$dK`-ND|LvDhKMfQHoRInBAq}<@R>?X>rP+R9Ae* zV_a3LSo(JMT#coWEW6zE^tkD>)1CYQTxfyxQ8A;*n}&2t+@8p1n6vASTd)jGtlV?j z$Uabc*wjXTWy=UV2cf}`PJHNMsM@!W?;rJUki7MVn8HT;H^{frl9;$r+t>K}vgoJs zyw4=5=47iVz`j3pmrh2{RG`r~Cqe3P`JOUOf#aeWe?(wJ3v{GfMPE5F6h9KTaoXC6 z!eO~D_O>LS$D(H6N9x2Fu%D`dN@P+Z96KdQzmGXI?Xy|p8a&zP6)MyJ%G^2>UzAof zp(~aBgenoAr0^6UYXFd{M`dF`t@!!07k9+xIG-Vsiia;_#e8s~M070I8i%IOe>&?_ z^SRVv9($B85QSr+D${=?3c*4h1t2$xKmX(fNcxqi9rI7Np$AgEAk-1D{g?ElKdKB4 z{@tPZXVoNT?(>v3t`feFsR9BJb|6`?HPU+OFyb}h z#Ck}>W6;YR6=6ddD48V>ZC;#_qBK> zu8Z>BPaU@rRq{bSVSw9k-5!8QRPKq5c=DPINP-_;1JAsk`TxXddVgn(?4N1>yx*TO?JNYMBYGCqBBjQeT6>v^3X_aWOsxpS zV<8?kpAggf=i%gr6o|1VYR!eMiAt811dqdERaiIa6$tf$MN1~U3vAATr?@-oW;+|~ z9q8J+lX*I$>5f&PT(SjM>m0hOsU`UN`Q><`jV?Ic?w1WsOD8V|ezY>5c}@IU$_MCR z=f^wJq;KiNu0V3*U*BKtzGjy#LVUFFTynWZ1UM{_;TFB(XFQVrGcJZn5P^v3@lrsa zur}7SK$ZH`GKYMs&H8Uo~L;@$(LO)Iwh!~tph}Xq7RSgLPUV%3Kb8cc>n!0 zwLAtNip0MDLzGYlL36E4GP(PJksI z2KrV5qdmCi<29#;YhT}$@!ydlP_Za>`{94TG!UX(*g}*f{BY7t6>*n!{v-Yh=q{QJ zIFj4;kdv+=BiWd}YA>iF+t}DW-CeSXpb~c3Uf8|zDS8S=gt0O9x_yb-bK#jX+I;GF zMPh8_x|S86#_E!(YU@>c0W&fWfIyME#>m1YDLa7)h@isS^p1v2Kci(6M!iQ z5nk(Y?>JAQBmsMdjTUOAsI+k7qdMy!Y7#hn!NT9yH}d~CVFV?O`=>MY!ob<`1awrT z;Xi-$_fa$Mm`FpnKl>kHafKt@#ynWGwhqR9CzXetw!^n&}lNVi4ay?aL_+F&f| z;B@x{@s}|0aZ}y_=E%oG)x*WQy5N>FqHU2Ba#5ijprc#@yHNDww~rp{Hq!sXjG46cKv^OYfNSu zzVn)oDRekHHEL8@7Q%5wd%y6Jl7_|h{eIE)GFZ7Mx6|${W6Ca&3+In|H5yk&RKg@Y=gmw8G8GpL{q}&I z9${EtJb||zTxWulF|E z&U&2Ab_h=Cf!icoB$`BkV!t2SD>mSsU05U$-~zyu^|F+A;>2aLiQLg#2G8Sy)$1Gz zw@QUw4kG7T1M{UMQJ~)F952&$;5ogF3jqBP$UU_r=BWyk7`1RRIUa+s-Al5&l!8M? z6W~13KUWpKFXXVP7N^+igtE~#9G(S6;_Cn!Iec41JLtdCzO*@Bp1;BQ?QDicr655T z*pWNjrpT_ToYgVxDrg#A$hNYB@3o#+Sg?GVL znqeWOEwEsOLW_548)-I}5G`bkVN(DcHIJ{PQDAa;(3?yuYJTtIvK+KH+>UYiIVGGt z(Yz;_O`}3#^qt4hik#5)Y_sp&rO0+fqi((m8*MtN)3Z;;0X@q=sjcL|1Ju9FJXu>f zDUnBz!}zTunRxcxc{wJ0^ElXf_<(})G!_awpl1cA{g^M`q%l$~iOh!%y?}4Bs>@JK;3l6&ludxHL zpFAG5NP%U$VbemS_eeifuYfbf$}1di_va{u&Xnr-xZ(5OxFc-rCx_dZjIaW_6$)-@6KV6sqUalfcEy^gzIN z@t$t^8_odVzs%*~HRV<_4KM_lDboh#8P5~=4+qF6johwmDDYsQG}4uKp!Xp49LL8ugN9FviY$Y_w3~2nW34^uEf^ncA7qrq+?3Pv8ibX)mL`dcP_G-2EYwOy33+%!aM&rJu zb-}P~4_%>D;s<^k&%F>fUqt%UZU##QJx#XAvSe+b60NTymH6=MC&Wqbyk+bzxMrZ? zWIojTxExw;66o2e-fvl~WAx8epbBw;)FdcqqVpZKDAwtoAKGkmKE#=JTOE zWtKw>rtYRm_1rgEINIt5mXPNf`r>StLtlHO18tx73r5Hm0^3Qegtx-{ z!zlRWrapRU7iWxK@%gykG&C{}sJW;8VH5IxHtput8mQ%wGM+y~-(Go-z~~WQ-PV5~ zYHoZ6PAvlLl9T2YOoL!@3*GXT$(PW5Kh}O$mVrIN0}ahsw=`Ld?2m_MqDjwwn7mKY zkffx3b?boKWa!82pR5|e9E zDg%l(rwsEHP$|4U0%))74xw5DUkaNp2Ep{bJ8kq zL;c(c2KIlQ-lB^1kpCk{rpIMp3eaJQ0~fQgAV}f ze*XS8lXNw`stZ^(wo63D))xg35c?JiI}#;g)?MEk1)OI8Tz)0V^?x%8`Cs1w{QvDC bNNOd-;Juu0XoTQ4G}L0RxOZ^ literal 22315 zcmbTd2UJsC(?5!$6qOF5R1xVl(xi)2AvEbVf>J`25_(gRrhq(BLJ25HLhnUF6QxOq z&>^5=XrV~xHTU52zTbEM>#lY0S~rUoPEO95J!kgJ?B8$pB+fujlj<7FH4+jMs>fPt zPe@2EV@OCYK`F?9Gq$>)+DJ%D36IrOj00vi5L9pzlj#GR{*0~Lx1L5_(#)oXadZxy zK2-lUEIuL*G6Y_rZ7=O7r`q8A!a?^W9)0_IN)cN1hmL9i{5Y^593=Vu57ql?*EvUe z0vO+;_k+)~>Y}BxLLHa-Wt|4+mruhcVGy6ebGhLCmPtz@Y%917p#|vi|LgUT($Lnn z3SC_s#2sySE4p*K_n+^aBgIa-&w9I9vi_+q5@-^2s4=Rq^t6~haK*3XlopqVl_Tr- zVF$-%#>TCAxX|O>zz203zf!Os`&rOLiD@=}-(Hr_V+N9+u5}`riM5^I#BY47d|+i^ z;Z0f=r1AEjFCt8L_|}uQ(EiW*t$}lKsAiwd&UDY# zKMNHhTj<7yKpAXMOv8_Phxt76@O6>UUx8ayxoy7`&reIv7syQ&@M6RzEX~pEK(pOu z(=sHea|tul@?vpno6lV#q<@Mkc)ew)Yw4#*_+wZ|y{W0`nZVuw9rpwuvh->?i9O4O zinJI*Rl^@30asS>r$0!~j%7|odF&ggu@Rv&s9<}0;X1s1#D=`Qwdt|OLH+646n1#EpfyD-^ae11D)@kRV0^p| za!WFxC$qPBjb?xTAi`f?-f4zBUUY%Df|=jJORJB@44p1fZ4L7DwFs3Hdr#sWWvE=k zgm_LHQc}~2clMNZwJq4ouXadY%oi7Or@aRqcD8h5#^Dco27Me3RyfP1;Q^bqwaQL} z*&K~>Hsd{ymdH=ScPo`s)6JUGOAF0C2#s_T5%4bHp~X9VP1zne_YBV-%iIX@dAKOb znGI_%ARf=tgg>cO3iI2@njcs&-~Fkn$+?G&f|;1?8S%1zqEjdT??d?ukq7k*+bmBW z;E9t+y8$aP^r>L@A4t)W)z_eN8=8P_wuA4)qQb(E1qyr|zx+e+vs-}b_Hu8L?W+D~ zVY==c6W@)*&L^u=+rDmfrI0Fl{#4|ocI7ztb8r~{hD#r~OgB1TG^nM^h(Po48>ko;?CrjKwv zX)5d<5`7%TX&bm`#x2k6o2|YV+hR$yw6NIwL>MxR_^(_#j7WlNW|+?(J53J^zH!?r z4D92POPn5RGqW&h5r7SJ=U=`AO0a8j?ll|xk;|p(zxPDFn{`5AuIblt*6d`-rLpQC z%Wk?AN4Vo>=G*bBU0kZ7#U)$BR;%2nM^XcwSn(Ib3qmC&ZK*$CIAGMu`HZ1jhoqL~ zXI+RNZSe;ep&|0wKSRbjoh!$ir7j`u@pF8QH`@lmjZ$UL_~4;EJInps4YythJ6=QPZ~_@=1J zrZQW6Fs97^FdU0JP&iG26zOEUo{jS5ctTLkk16+mREghs(*Q3oEjv>SZ95O2WWD}9 zL9`V6XWa9wq!_)_(u%|7sxw^%CXMi%eiR5VBh>6?`mu1OgEA&q8=gO^uSw7LBF&`zZAE3NuE z%{_EF!>VNeJ1xGr_{1Rk!b%G8D;0TNigXxA+oDoZ1w3><^NJ&G zRbnt}^NkbEz{=>8RpjlBsgcE>qBjBw;gW-qCQ2#%9`!z=M zT!11goxAYNnVGVB(ggJdV<1=;DtT~CkNMRvv`7TgIa0b5I5{hstACPuFf9FUwWAhc zsvk+slx!MhCE@FwRo+o6xLf^vRGv606i^@@-Dw0)Dk`fWLZW<6!5Ah}V^jB38pAgi zW@gJ!t023sz3$>i<4>C`-{AWq2O2sl=|7|2SN5(Rv(UJ(#~L3syCt*Wm)=H0nkFvn zXOAvuZvLgH8ypw@3%Yj2#MIPN@5uXJJRL3OZ?Zr$3kZp|Ij!;J&W>6cHC^p?aOUNb zfAbEUXFJOjSrP2xvAH99bSfX(*V~$8e_<9NIl?y;sY^??4m>;yoE|F(g@X0lo@=Wb z1!ZJC@#~?}^q1{)vL<94OcQ_DB7Wux%%g9re!61o}tjpfmx!Kc<_+1UtvKooo0`6+vn5S+TiJnG}Q%v)q zYEn~{-DBrLefjh2E^QZU!$#Z3+X;TIw|~<7Hf6*PzPN|+&C8^GeBvzLFla*et2;M* zJgFWZ&>~QnYC2jG9_fwh&SU!@t+dqDaY8#c=XdNp&$sOCEhCrjrSZItw71NaQb#ex zYhSPOv`r63()ZKX&*(q0cr}v~8C7vc#jc-ccGrUa`atZb z&<*fy`Vl`~?z7Yf4XT-c%VHH>I-{o*qj=H@C-63 zmyC(ksrY+M%lGKmcn_tp%gN7@Je3o{BTl;VE}gY^v3?NwH&3`aYtOseb-&H)+01Si zYW};<#IBzH`P#5W96Gqpg-lKS8ISoP4UAZ8el!+O?W#<*Qe$tDH-VVJWFnLyvv~7z z=$an;Ec>lnZ_}C<_N?eT+83vG-mLYNdbDmd5wp-_LWW8IeVo7#04TD0eYk+Zfn(yJ z1-ks1rLyZAas?s$lhYc=J#RsoyCquHPefDde=aEsp+kqh>B9taA{SsGC&3db$9Bt@ zq++<~px;7kC^6X^#&nJTKkhEq@OB0k&{t!~jG^z07q_Y^R&ZeDsW)`RfoD%0TFkXc zukKk~o>~IyJl6@Bb@t`8x4dlqYCb2DP=jS#141=8ti_#)DRm1~zVLwY-W??9?@I5H zv71pNc4)|T?vG;@+G?zEubp4w!c&#A{NWoZuBE1Qq&lpbb>XF=G|lL!md}d0Qim8& zJ+U-oRfs8X6}n%(FzM+WO)YOQI=E!VJ#RWZD{$^htu2*&OROWyPIGQ8nXOGJ{pV*9v_}@ z3~69`4>x^2ZR6QltHdd>c_sW|p3=}~LiD~?UEY8t-$ysykrkoxw#KttJ6l0_q*vL+ z&Y*o0fF~>eMX&P^x+g2^SpP$TKOPLzrzCTnnA{E337i8gTqa=W3YY= zX>{|ig42(3r9K$WuF6=yH=yC*rrw_xDmPNgJSw>4{0Z7mhxsceV;hHKswfLP^R!yz zN2Yd-RCn8M^fl3AW?Joj+zJ1ZN65!E<0(-&BX-IKVV|KKUUZluqLL}CNqTMLKF&-id`6?V z?ciJP;lf5rkQY5BS7}Ep=C)w z{7to%aafI7nbg44M_Hs<>uwBno_XUSVnCuyC2EG4>=ecxVD-Z$Y%NBuc4C$>>A`@> zS$`kvIZ8iQ$+H(_%xu7)_Q@^u%LLLbt!7~5o%=+=`hLo|?Ck1Jb4t<3*vc<=0jtbv zUY1o`)i5hI@grz7c!}-|rR^KzK9W`lNRreXkGmKtpg7E{7Moy^el{dXt&= zGj6099R0Wl{#=vVpj1CRt-=)nnvGgnj*OS3zI@EJ`F7oby2%%b6+P8qcMD`})f_ zK!{}J^bE4OnGNJKiS{_(z%KuqgaHYq+zmEYw1sqc?6v@sp)RGX`OYH|b3N3pHa)PB zo@OSDmA$Q)t@UZILy3=a;hfS-_~5l6Uo9KX0r9?WxVtfj%#{$y&Xk(ldQMJd0^I|S z79|$-S3n4(G3fM*y=<%SME(TS0Du>ncq3%~alCv@q!k%03|ZUVb>n7#9$oaqy+d8b zSM_NY879C9md}lpAS|!H!yROHO*3IcxX$NR-142L4!=kbK+`brWe7MX6=RavhDpW9 zF16BP-X=xT4c^48x+Ak4d63=F!VSqzLdeM&w|u^<(hELm(|orghPl_5>1AW}jLzl6 zwcRK8d^}G)JD|<$6HRHQl6%v)GAUy7;2zR4EB&G@tATOsm|@|5^^Sius z;#R*SRa3-UAz4c@)nU3q34wYYzHhcC$3%Nq?`jr`NGu4TTDXy#SLJe7J9ZfoA!90; zSwjp3ryCt`rO@Nk4ItYd-oGowvA+=l%}yC!?t3aZxxHN&0nc#@5z2brvGb^A;8%!WqH}-nJ><9E)vq2Y827{;oyqCE z>?IAb=Z)>;;1(3ir60#ykAmJ9DPa^}17Qv^Xgk^nChqvhjyuoNa9GvJ;Ywt{O;>@j zr6wio84^BNX0IH^j!2JgMo1~v`)UrNpS^b_9{LZ#jASJox3-?mEXKgnv?v@%)m|_VRRm3 zXy6*1Bp?2#hm*E!q?X+DG(_YPOP*ls9KTjgT*Lqs^ppIzrK8s$$rMW1tU7!(&V9M< z$TL@_YH2~((eGX}a@)&eYH3EaBk%*Qi?Kh^IY>wdqk3~~nW`0_n+R2YlttWX)yrxW z>ax*g2d7ZxZi|#DP;0xoQG(0qVCWtPZr@ zZLqx4<=?E%v&-K}0Su@WTkfTWxqj%w2NL0QsqA979SaqztcSNfW3bRmFtfGYe$s0e zZuaafS@9r@51~xHasPQ%ee(E4aYE8iu4C6PM`s;gln)rOn9j9tpr!ijVWTGdr{3aB zsmQG8vu76zD}eOcIgvYfQPt>ko!K5gY#Hu|o(@Cb>Wa9JVFw|^lN-t$l}{wOO4?0R zQ^&<-gYM%ZYoRwtS^ST_c$nE3ohb8IpO0M%vL6qZ~+&31tZYFa^-C3lU#WoO(oJlxexX&fJ4;^ML(@+4PJ*N5F| z04=xp?XlvYjwN?Yq5dY2Z8< zx6dKmXsmt*kf*Nd-H}tKJQdo}6OOI^;SuEvz9RKV?pfD9@Tl8snKS;Pxv)jv*}#tb zHM3h`&Ne%Ll1}eeh1%{*&-g^xGLF%YIygLLYUM50KE1j(?&E=tcqb}Kt+)mc1j>w_ zi1Ul{M@g`P z9y}4YH^(xKm6LIARPV^WLIW#$#mGBSMhyd?kb9m?NS6{3$jSw$5x4mCl=g-$Dx7LQ zJ-1MP43yT$6#&Xc_({J0={Dd|NAZWM@s*R4$VZ3>1Zfr2JJ`_1_o;)F94&4{eu@vwMt z0HK!K?MVmVZyQg2oAnwxCE-{7#hnp3A0#tnQ%%P4YilPh^5?%Fdx`|5+Y-oyY}L9sBWY2Lxu8hnl^>$c6s~j*D^DS+BWYGXp}hF|}zauEBaxPsTsja?G+(RPr zq9a|I6kXZoYU_gS(NLjDp&+EUI@V$i=0th-XOUU|kS--&v!vK{UW>O+*_q=!S$9$C zs7P0$?ZJVndL`45NYoI=$&t|fWSb9h0-8QB_u`8t4e>1EW#oA_{UC8A3*4G5x6C7u zl`)k=I`HrjK!#$W9YDnhl)#1L=B}i943wY;DoQHCd3kDyIrKAI8!bcUSzU?B(8bkd z+rz=5bla-NjZ9h2IbS^c{6GU1Cld6hyY(?i-z$N8nb)oQq%g5q(-Rk$4jrZN4G7AX z(5TB7cDyT-cz$@O4Af{_KMH5eWr9tcljUM?^e4Ok_5XPTC>(+k;2L0{{?*~jHiEC~ z=B*#xe-0x?EXq>QQJxi4N#un~tf6{-C<&#Rn>YYhKs_)FDX%kl=q!1Mi~6kN_P1@2#MLx<_DAlHoglzm5j9CIzniEBmh; zlK$}sn-`|d$bOs)&@bWeU0|KdjILYYIb&pPv|CAzY80Cf-F^Y@;d;pgM~uFlH%iW6o1 z2YWZRdaAmEg&@a~lK>-`N9d$&?aEuCLOB7Kk}rGTMkD*nnP}b~EQD?^fzI{%iFnfW#7&(lFNJ;Ip>;Z4X_o{O3Z;YKv4^g zNA*o1oGAI2zKTYH-T7;2d*IE!&9r>a`LlNoL+9iturC*NRmGK$_aZwq7y~`I&E~oq)?wC&hDDxAcLTONoM~J+R#Ukra9| z=7c=7?+9y1_$cTllxO@MZLgjhsP14A9Qm^&Iu*{ua@X^I&-p(<`LJ$7X_)8~3YDj! ze@gFAKa%#ElKhBooy~V)<@3IO>1xx&-f!aCW-EFq`UZBqX@yX7O4MP02~_n$Ufv`J zxrf5~UNxP$7Pk3ioXeacfTXc8sfM^|rq$<9_!v^OvDX$u`7{nK-Y$&UDt05gYEfD0 zH1l{xPj=LEVr=4={Svl>{@HQ(uRBOA+6X6VgN0UrZVXg>nd)%=z5A*TdQIApNYZ%EmtGjwGCk0jkl@V1>*5>A- zyESaa-_guCXq9%tcZHd=!$z8u|)OM$+NS0@{#%TmQ1|k>yvu^{VhnmC5{m-=w!N1$MK9DwbP99Ixyx z4LRaG)L$|xnsD2*!Zz|@^iQ>=4D)}-L*s+Njr7c;n*=+DZ)epA!)FEvm`p~_x#CW zA5OUpvoQGN##o>b8EC^+)k+CN3B?Pga@j~6+(M`a?==Z(s;Df*L(9wpjT+x4{T9md zyTi$$v}8_1_lXfwH<_&x@4@R@Hfb^v8PYE2o#+r}u>wJk~ z*I{R}@jS@c6yE4O_b|8+G`TRaY$96=wF(-|clD4*St4QeY~QywLyLScvSdDJo~sQuLpp(`+&E8IoW z+Q?_*oI*fa4&iCFCVt=2mVI$x*kQcPl4fNoY;QTcB4-I=1*59%d#>@M8l`6WvRjye z$uI8b@s0eY-)-V+vdko`LrCAeidB_s%^R5TUwg|p`ule;YE@3V^A6G-NGr%-srB#g zbTs3O7vK%q;~xZFphA03wNW zOv1{lzqL=Nzd#&RDouKgN9Cq;4>+NK#aCStwjp<7V+L+|1s)L-y0RxT_3@v>ac`-8 zL>!b-B5wNwah+(l(+112Bej0>7~B;3aQtz~Fy&@Gbx{KE8XA^H&%lV;#9TF_So42{ zdxS58Q~UMaTiUo|&JKZT42RcKQ~GuB3|AwPd5Z04j=wJ5jv1)DYgZCHpQ!AW+XXZ6 z*PiM_$FYBqziLc7Z%!vRz#=}U~cR}m4Q9z-7 zeb$?i?Oo~M`?FleK$y>`x{BUVbIk^GZ)Bvz(<@FERZx992c$d<6>9I?7dqv3N3Ndz zb4jo|yH&$_ll~Q0qDPFT!vyucx|;MWPLe8r!=Du7PEMwM<0@+6No0}6#}a-c^T?-H zGgzx?BPp1wOuubPqv`FB%W-m??oLTi^_X)RsmsN?;}5Sm-Q!mWxgQvDuPiyt6vF!l zq_z^_e&k7(BcHm=l?=bRFXxZ@&uaB~(myAZbwr##J4mG%_J>v0R87!4TKF8==paS}vL6>c zkZ-0~xOGiK>$$7EKja9r^wX8qo%N;&-vB9qH<9f(7+Nd@OcM3 zJJjC27JuI*@C8J@5WVx+#<07RI(8q%{m-$D4J1xgYNmZ>xSoZg07f6V*-D7J0zXCV zNna~l7;hvO5l(;53*>aqxj-Z)Ro_O)ChO^Q+dlNpuawG#@v2t`QN8KCX}ypayw}_- zO8aa+#o-}is9@&)!s zZmRv?akP>DWdKw|X=wqB4t&jZkvo9*3&;-a)LtY~;2pr@|6=7>s37EjPW&H`{(n*Y z|AEc_#rXfqyukRGXa+#LhXDTm55}hg;6762;a@`XKjZxmHvd2PgmxU2%W&0-MPOj( zT6y+GDTjJeFla*IlskM?@25^&y@+)M|IE9|zijb>2>*Y)BKQf`y%X}%-z{XszT!3H z@o8!SHZ?UKCNz_p%3Byf&|LydceJ&xe9uN-8L9h>XJhh?;f>0D0G zWt-5wR+_~;0&{-}+mW}JTq->Y<-^Ye*|Ebs4$;?}suqq#6K@v;)urI)3baJ{{Q|%@ zb14A6W8nXbk0gQwyv1k(HzbKbGgOzz(*aF#JU1dg#J6!r5m$O1h2ix&B!_FrKdAHs zvBn~|{y8mJG=@||GSoDO{|8`|6J`3XN#`0ODFVVn~xK~j=cY4C=7+@Y& zR_eJx)wtPnjQD8Wh|(!f++HK#4Q`pz*s=UJJP=*#iclm7VW?ZF>nnh5&^OcTDnZ% zAz2gn^zS)9_pL8u0l@J^NXixi(*{leypb8yLbN;at&B{Z8j{D&F|>3lwAE zojkHqww0&fyK?uvw(1KY-@k90Fb`Sy>dyE6FuMuanq&r49e!hI$gc!K$bItfJMeUF zzEO%`jwcfF6?L9^$5H+UJG7n^ z?W8^g*7wxTd#2%bmU^!@)A`=+`4yFHZ)S(&9sCGH1$Dn=+hkvm%e#l_dQU_`Bsw}TZ_4JGe8?#@ApuVkMdistUFK0V6DYM#+njYiTqJr&~{qh zHdofc6UNwSkn%gG4IPZ4$gek1&D+?tINI3nDP&iR(s1=T76|t|B2A1WK@3r zCEnc1qJ-Y;bty&BlgqniskKa69?$Z-)bX|0oP3SL*_mi{CUl~rTflo6$J3v->JE*$ zUq>uDG!o+Wg(BdI9b*AN7gXMNbp3u)%Vs=$746D7OWQYTiE^#LHElI!Z9!NYt8;-# z6{9O%)Dr$|=<4po>C?bbi88Z-WYw#F!nyp}IC}}-{A_J6!^1nv-5Y-{NME{A@kA)| zc>E~O3~$@K_DDHCwCB&qdA@_`Z?mYP5oILPw@s&lXDbg->kSQUNlzq8NldR;bq{-= zX*g8(>*H1iiClCLBSYJNj22%5*!ZbBazn9vOE78euw>I{x(feXP`~(K9Wq`g4b#Pa z4@;XewJzAhS zgam|TCn!nfoO_zObqxO{Rqr3WN7W=bA zCii|fE`r8Z;tH~cuK7WB(17o6BaV{oyn@Rv6yeQIMZUk`6$)02NNwZgz?87L0dGsvciPMCv(`BWX?^fx& z;XsA!MQ1hma=UG?`?AIg8lQeScD1x82A=jJRiJvVqIfh(tqFR2X6f`Bjq`%uSF^9? z_ybnysjOf^)yAyQ)q7xYxYy||hyuA!E)$|2AQZ0C&2ft}lLZoptf<|`C!5n;sh?7i zT{Z0#SUig`|0O)qzlYJ?^vJ94!@ki>%$mY@)8>9qHNc{R=nF&V#vV|5o;)9+*6=eb z`kEtmQCu2ZNiEub#Z;ch9v?AfZDA!)^6>>LX+~7N;0xRex|>30hr)D!zp8-ARb`vW z3&b7=Tn*)*?~Y#*V*eJJjlbsMx)1U+=toat&q`4g?OHX#8+4e)$2;(lVI^B1)zIMh zn5asj&o-E#I15V1!7ZRsE%_Pi05wB@Mm!W}E##nPNO+RNB1WgRQyH0ut)diUtylPr zJh9bZdSySMyHT0D$9nC_m2Qp>gNr4VgfS=N4OCr;5$k(3L(t1Vh!?jy&F>&yod=71 zCrOhLS_^F$2QhLbe1vc6af_oHI&0Z`ek<|%aSAk1osT!vTl`#A=6XTS3xwH`>;v7L zO@`Uz_+8u^l+cr>Sa;0i$%`T?CJ9QEF1 zSPFs#Cp%6vRe}kVIjT>$`yp1WvuK;1OBT}$onVl$T1p{X)>B5^iRyj-@ipM@{gw!-gX zmt=u1v{`YjPdYz}=)8P_-xz?I-AM9sL|0~tt4x=*Ryj9e{_>ji6NFp>(w+K3LGJQr z+KkSPyTh3ds>mVfyQLYD_uRx}*Pez)Q&HWy z!GO^w4$H-Po^48XbK;gAXDt-((_u2v`pdL$Q(VQL3+N>try)W z7S?_``?7@|L;u6Fg6|>AA}w%1-ovuQbVbrq}zZd-} z+nahX{AKnUd4xr(p}!;D-{tFG2yNYDT#2_oIkM=LnK7~l!p8x`kGCDmEnK`Ul_(W; ztYGzIoF)L>zTUH!`aB#4`IEtLDw%vY3=!}N+0{3(u46lQG{clJxQEH4Xwl0CU?&%O z?CJ+7F9i-OXj$YO7gd`TY4yn-EiW7w37^P!kUwbPU=V_FA3jRE%YezR*@ldD)Us0) zu5L5d)qC;^+k@DPingnz)ZeADpinA;k;OWZHA{>c|Amoy2QJQvImS^ElS%8(gpmwh zv>54bqt@%KsnuElfK}54&#J_;1QbdpShv0w&$E{+d1?TRpt~@<&M6q#E|Ul(6%qpN z^xHhRsK%RbV*$79(dHsLpq6`j_UKy)zysVF{fnEqK~fwyw~Dqa5A_pbfa8Zw0Hg(2 z9|8eS>?G-+XT+#WkyKsj^aTv@4L(46CpnUgoY3uj0gx~DhJ1F6DFAFt_7J=XAYuRu z1H9As<_7a$Y)*?gB)N#LTSkki)o{_`qHS#haK6NcPfuN%W8+tQiwU~)&&|U`#&0!e z)1W5yDrUd#4OB>lKcAXC5=@c8efKAu9Czy^xZ=K7j~`Kx@w|8Ne4;_{+&jZh0uOlS zn($BhAFL!FLT(m$0vk^xJYdtv4|i@px=!Ru;cC0sIf~~%c9HhzfiVUhlm3Td=jWuS z$*-Ho5F`|`U&w!`dN?(+`kZQN-gI6h zSy6m4a>2wdhh2^sjp{M?wTloXf$=>z^S`jP_AN3Mrf^LP;z;uA(+NZ@^uyQ7|4%0N zmHSyF*;pC7+@}Ro;Wp_2gEl?_7cz!s<$&3i83B=ZJFHa3>40UZHi;}aZlbA`QKQ0j zVnJxJftkhjeTREQ=~Bpi?4)VGvvgYz%l4Ty3wM;BH_8kWq2&&ugdu?6(3d zS)kxwDtHIEed)rr#4l8Ovw!_>7{OP7+EE1DQ94P_&PCzEPg!UHP!T~N&G9TQb~Tk| zVD7tg*-=t9NBGXnk*ud%!iyxXsKBtU3PoN0*X|Fo+V9{V5#T)Wx}WT-1(N)4_$_R0te|& zZI|$#S)@+D`$SIoai>uJqcRYWlP5pP&-ei zji%3kD+M0{j8T)le=LB+K<>9Hm^@uEXvpE502>>>dMc(-dP%)>=#R9ucC%7+F!)CTC&)Hy0ml90%}LJ(1MVqUI0Uam&!V?8s4Rd+32VwuLDsNp|ufjyPGY#_>$%84wF>gYjuKw(?x zl&sG0retyTZM_G63;$#d06zvXeOA;{F?%O>Z^ST+G-xaq+YB(ZDK71T#dWEG51eJ_ zJxq5nGrN~T6Mto{`={f@;JSb&n)NNs`5g56`aWz%@0UvVWEJD;wKx6HX_LjoKSo;X z)nfX$vKY2G8!p8HENRnI1hVsceV=fEvtT$Q6c7%|vWC_x&q%L}5pqX+Hk-~axAiyT zhQdrUk|{jcr+=rnQRDP+hB!TZosARD^V^!0oUexuHY1^LEWplw#_tvkGP|{q zy-V3vTI`k`et3!f3gH`bSeKZYrOe`rYO_$HV)NPee|qE3N<)py87a*I8@Jzg*K(@P zt{fOsyIbg`=M}hbWrx_uI5dro|5iH%=rxH<$X^({{_Lr%*y*8V8<)viR04lVl`(fY zOzix3kY~Rg_nCtwUpscVqKo+}{QxTa#RpE48haVLQH6UH_wsm< z&qC}zENZdg%@K&ELG`b;%b9rNe?G@T6-`YC^Xoc|^!C+jI%=3IT&(wOeoZ3Hr=m?0 zDiS}=fu*WljNM&gvX(xQK}Lk)+|LrjtY*v4ze!Ah$~9oSUNF&)6xKM3#Ezd6y7S z-Y9}oUqWwgVm^+3n|{egJqso(GpI1_I_>_B*vN?L!tA9Q)Z_F=Y$$(a#uN4jkrO4Q zg$yOsYXT=58*iLuN5<>vzs^APp}t@%zd6aIMJvt2yB2L92_@d5Kg(b<$Wl)TBm^&G zJl~X+S0Jad{Iu`>MM=tLR|#M)rqB2>PEwWe)U#dYR6#cG4#6oi(s`DStUrfrB+FHm zd@4a^k!Ocz=_mHn3C=ncLV4~&<$2^#+lfc|<)>A`rUh2e)TYeP{pXI~y9mI8LSo3!kgxRkBmbiNG1miw~Km(kD`bz zo6`=3l_Ms1%x>J@tg!g)G=Z-jzhp2=U}40}XcVz-dagnfy-@)9@(+P&#~go2tm<)a zk`^^u+e5FRMTJDo} z#go9{R)8@h2~P4K1%+nxi%bF5ToCR2Aup@EU}peV%H@^GBF3-9v~7}6tvZXr^{@38-qcBJ8Qig_OcqY6aTq@;-{8cPLQsuql#<2~L_wr1X+{jIF`UmKa3Np^MnuHrVUtHv^&LuY=cGt4oH3$plmFfXa z(e=RdU48)A{8^$dBTVTXFMB--Am!+ zd`1Z{lvrp8ts%&LM|wOKUscJ$w0z%R79el*MvJ(wj8Ie3a?K96}*FgwKjhON1 zt;-o*!oQCb@jhCye8g;+$<(s1bymQnA;kxvQnif-F?*(AqHwR+@=BKE? zJPkm^=5K++Uy51eH;I1%Gl^Q3Z3b}F)&(160b$R8y*zfx3w}mo5I~Emx*Y)rT%j@m zAcCFYE(&iiAk_AugIlj7xtm6>hQlAL?0c=cpgxut01Tg25GT8H7a6<)5T}|z zk`4{9jzZfC#j31;RoaviDzQ+PkYl)Zi1q*-N5F|-!95$E($>J-a7=` zmaRG@hi)3H*wy>RlN;Dy>G@YXL`|abUk>?;7g?b*&9Z#5o8m6sjm@ksh}&3YcNds- zODmJP80hwoXO{JyaiU3*_Ko93 zH)L2M*FNH&st*nYoZW#`e3MO&S8 z6*tVG?6H0oaiL3Lx4OvrV#9Kq;UQ)0YJ$d%3duZ+%YNWY=SMXRK|(lrHcD>Ti@eOsnV>!ZZKKgdQDAhQz*Y^ctKc4 zDW0fDF{J`cJx0a>A1zuZjD53$?;u$dzPNj(B~Zn#?!APyH;b5w`)h)mz_7uZ=~MkR zM*f25(1*3EUhl{6@W-nE58FMYr@aon0{uz>r3AC85zo*iy~_a!;!3q4=S(bLLm0Cp)BV z2nFHOqGMM(9T!Y%Hm)omn{UrOZ`B{HqWRAzXvPUZ_(4>(C9FyhxT6ckDX@UE}!>a#TPd6WA|i0d`|eVSGSjGMT=Fr%r){VqcJ7UCMkKCan|! zeoAW`BLCk^|#Cf;I)C@A)S^>XI%P_OMD{|?HQtl7t!E$d_&23dy|dvv5R zbnKHsVn#yNY6xw}HkDmPLXpw29U)oI2w^D7o=6TN^W3AS^PKZ~{a(-S_t)<~FUI$K z-S>T6*L~fe>wOOv-LohZ&ZKPbFSgi1y}ORL*`2iVuGXg3?hMDIMGawt>U>BOF4n2Pllg8vo^(Zs@m}bA~ zRr1wRZ$DU$=Hgz0g>(ooo2MY-r1|bBBGHMv6G5JCdWrm@iU`C2Rb+NqIfN+Vu|Ujm2y(PF=J2Tccl!8ALw67y91WqtbBun_I-o&DJ|MJ+&DFwa31ehn=rp>kTam zCq7H7Kj(wu-B~KAmMI1$fQFU;Epc)8o$=*2Dz1}_R7!kBd^OuzwUNfclK@&4W68X1 zz~5DWJUn1SEvk0&ZC1UioQABV%VTm?%5r1#wHL{n-In2FiKOz#Z2`u{$_1XPtnTHe<#qZg10=%PLsU) zi(0c={9Rq(&SV#Qf+nm2GTvy9$acAg=d@42f;-DC0cH(Zq@ai7kdt0v?7yno6Xct& zNvoWsk z{B9YROHEPG=+X^!rVfWo#GBtrBLxi(_N-b+8#GJ2(d)DwImfPY{Oeh)p8!=!vaVd3 zVnewl4X+-tlmt3F=h*y+&8P~_tgXs@7=BbaC~%O7;tK+Y+^v=Rg)I+$+6~peJM>Yj zWD%>C8@eY8oK3-vKLP6FjC6c0rUo>*21(&g#&!2TsmU&Ww$#GLP!clsTI5P^j}?Au zh|9PVLG+Y8|09-#-?7f5zOYReN!Komi9Al}mBBWli8xOTC1N>UdG@9@zJY#!*D03OXV&b* z@1B6=q(2EOTE)eV6oo2xEOd)b6kSs$%F|z;+P}1l%`(9>>)p`~lPQX!YNLMe_vGWn zA0p4baLGtMRaX^uR83@gz9mz<9HY-n5&uax_JTB2mzll0b!2frk502oyn7am{M0&{ z0aA-48(?T=T@i(T$_kU0rQIa__yNtdVE;4mw8VjR0-VGHk!4)cybUJ4N zqrj{?o`O=Zch;C#X+!+J*?{TMG5ER{)0${!#XI9xWgaX3`rfk?!bJ%L?hmi)KUHCf zVG{E_mXTNrGFRNsM939)e@S{L=RpSmoik_CXbbTfk(2s3Ac*KwF+l0hhae;erCFxr zLTLPwG~BtpoB_(Sk|!YTrRK^#9$Aw9djjSm9ztNH#HEGo}f9y7}7`S&U{l>`zh>2%Mu_0j8kNXE&tmf zCKD7v&bKpEHZymrd2ckV2+D%dCB)TsP`~ogZq&M!==TYO-$e5+Nds?9A=rZ*!|p5K zyJT8rR7Ke?GXN~LBnU$7r*yw=;+bT@oWpC+77L^<=M`sKLO0Sfy!jko{MeO04E}T$ z-VQtdLI3sZ)tpRciRhF)2ynW8Y*3(=05Y^ZwjZ=igboAe6^&as!zZ-Oy5kbY=B&_< zRhwYHi4X_i*`nG2_4Y-fpFIg}jss(GS!RFX%e>K!yYDO)`TxFL%qN*8nUJ(NxEL^^ z(mN&Sejf8ZY+G#5a$LJ;GIQ9%e)MC9xy*ZfgY&W26OF{%Eb~+5Avo+A9M%dDX-HFh z-1O9-y2=u@A;iPzyXGU7`B4?6{1+NS>*hg}?8JebCTbad^z($|_pyiMfk#1Q;KtM| zy}8k?ve!;@0M%ifDu;LH&FX`_mjRp!I&)Ep*;dG9|BID-X9N&;xHI@3h@>=)e|s}C zM40{y{fM1YtRROGl^`h9Gp-Q?j?NUI>vpg6LJ-%GOs{WC9ilCYi~#G;7yQp!4l*u` zV*q3_#1jT?2K_?)_gp|AB6b!X+6fg89`GvW%j|CPm`dDH8AwKNnnJKBRHKX2Z}{K(?VOY#K+N1dJG{jiudWk+n=yii=4{X_$g@(p^P!ZApx<2n zbo^7Pm~d-DQiNe%xL6uIXOq@6!19K|4q9-m(BX9!&WdCeY+B_@aPX=uD~fs2sl^j% zn3;Z%m$5oS^*{FZ#AL(tiRz~HH(xMWdrwq5W^PWIuZ+xhf*fDhb7HdmWvipGlP(LP zptg16z~;A43MzFvAL?`EN6_k1S%-ZhYWh=O&UT|{;kkTV6H??p_!PUg@l zBsjRYIb>CYl`5rYRQfpkcIXj()-rtb9%|gS(#m@bYQWbAgL`&Jc$`dsw1#a+gpjk3 zc(A!tv3OHaFy$k1DKTB;z=~;yqV8t5E#Z1gsZLupPbar`_#wkG*!%h+4({t%Igfs_ z4X>Ktp+Op#BF@RY2T0S0Q(XPFDmwXxt-UvEz3G@>x0y}7DnhQjkVPTE>RkkQnrauF-1STuQSt++KL z8}B=vG4J}ZG!QjNOma5-oulj*aQ>MrFIa9u(X_7XY<6p#zIZ#z{aOH?Kc*p=TrQz9 zOm8hZsjjoyR1&R~P3W+^=1`w5>V}jU@6?|6mT>FQ`gjl@q2IPneKR^J$TF?f#PRi6 zf^R`sOW`5`QC~rvcBH|MJB6b3i9Ua%R;b%t?Jv_mlHL7Tc!f?%M=6jDUWwwj-ijg+ z*4^Mr*?u024;qN(i`n>S^#Joo3Vt0Y3_hdbiWd(K9S^tu-NPRzBPZyDfTx+*skdsX zCTopk%!ndLGy_+wkR%($WJv1sdSi0xL&Uv_^?;mOqgYCA#gwj~$3&g-Mqm!Vi>4yU zz}z?F@%%SqG5j8vHT9H?OS0!iyDUPS8C~2e4FWXnWaXU8qOUn5Vo3AdwfMR2%mlct zs9KE@$$;mS*z;}X59rG3R{qHzG+dr}oPyzxFBl9*Z9}M9iJn%;BqHKlyH{GuFVvS_p^sGrIH01$tYUM((=8zD-9}$7?bV3@9nh& zxQbLlOG?|z*KnA(yzJRM-g+b8mexpnAgTJE)Oo2OA)39IF||3w&R``JpEqf&te@Y8 zq|Im|w};NO&}M8RdOcGb&=npQ93#&)Ll?ycizXTZw3}QU!aq40KIV2|@Fc)%|0uKI z3q%?^&xIt#y6gQKe}_{0G9d7BSpM>RMUNhZW;Z=kdS?A%j^AU->OHuzZ>H~Dw^9x@ z!3M3jN%X)VB@fZCm`UmSpu0KEvr3;mY%k>>vSRN?kud9%}xDz64pfDN96RL zZC4_SjRhwsbl!!K#L7sIJ&_VGsoJ;whH}~$y2#9(6Yo1I(0htC8SS}XG-t!S1gspp zJ7)LczxKBOaVzap(9q@r45`p}CW%iKgrcPk-Zkuy^*;&%6PT>t6=t0VLB4!4Y~JzA z`UO0ltcz*&EsbfFJh&$PqcRXOi8GFq`rInXkZpw+Fc7j50Mh?YrQp8zX2nc%5(|(F z9y5T_g#%9bA&IBx{SE8$={>>e(`V_o8mWB<=RwJmg79Em=+@Qoyy=>K>*X^)jr(72 g9Q;51*}aj$j(>xlnBeJ)WWMs4xeeyA>6zGn0S3sAB>(^b From 79170c4fb6a5f9815336772a8e195abc7e1afafa Mon Sep 17 00:00:00 2001 From: NingLin-P Date: Thu, 20 Jan 2022 18:55:46 +0800 Subject: [PATCH 09/16] raftstore: don't remove other peer's read delegate (#11882) * don't remove read delegate besides peer_destroy Signed-off-by: linning * add test case Signed-off-by: linning * make clippy happy Signed-off-by: linning * address comment Signed-off-by: linning Co-authored-by: Ti Chi Robot --- components/cdc/src/endpoint.rs | 1 + components/raftstore/src/store/fsm/peer.rs | 8 ++- components/raftstore/src/store/worker/read.rs | 15 +++++- tests/failpoints/cases/test_merge.rs | 49 +++++++++++++++++++ 4 files changed, 69 insertions(+), 4 deletions(-) diff --git a/components/cdc/src/endpoint.rs b/components/cdc/src/endpoint.rs index 425f7d0c5bf..5854ec56de3 100644 --- a/components/cdc/src/endpoint.rs +++ b/components/cdc/src/endpoint.rs @@ -1673,6 +1673,7 @@ mod tests { 0, "".to_owned(), )), + pending_remove: false, }; store_meta.lock().unwrap().readers.insert(1, read_delegate); let (task_sched, task_rx) = dummy_scheduler(); diff --git a/components/raftstore/src/store/fsm/peer.rs b/components/raftstore/src/store/fsm/peer.rs index 378f81a8ac2..b8a1110cdc7 100644 --- a/components/raftstore/src/store/fsm/peer.rs +++ b/components/raftstore/src/store/fsm/peer.rs @@ -3452,7 +3452,9 @@ where .insert(enc_end_key(®ion), region.get_id()); assert!(meta.regions.remove(&source.get_id()).is_some()); meta.set_region(&self.ctx.coprocessor_host, region, &mut self.fsm.peer); - meta.readers.remove(&source.get_id()); + if let Some(d) = meta.readers.get_mut(&source.get_id()) { + d.mark_pending_remove(); + } // After the region commit merged, the region's key range is extended and the region's `safe_ts` // should reset to `min(source_safe_ts, target_safe_ts)` @@ -3690,7 +3692,9 @@ where let prev = meta.region_ranges.remove(&enc_end_key(r)); assert_eq!(prev, Some(r.get_id())); assert!(meta.regions.remove(&r.get_id()).is_some()); - meta.readers.remove(&r.get_id()); + if let Some(d) = meta.readers.get_mut(&r.get_id()) { + d.mark_pending_remove(); + } } // Remove the data from `atomic_snap_regions` and `destroyed_region_for_snap` // which are added before applying snapshot diff --git a/components/raftstore/src/store/worker/read.rs b/components/raftstore/src/store/worker/read.rs index e59c11a6755..c2d600df7a6 100644 --- a/components/raftstore/src/store/worker/read.rs +++ b/components/raftstore/src/store/worker/read.rs @@ -152,6 +152,7 @@ pub struct ReadDelegate { pub txn_extra_op: Arc>, pub txn_ext: Arc, pub read_progress: Arc, + pub pending_remove: bool, // `track_ver` used to keep the local `ReadDelegate` in `LocalReader` // up-to-date with the global `ReadDelegate` stored at `StoreMeta` @@ -230,6 +231,7 @@ impl ReadDelegate { txn_extra_op: peer.txn_extra_op.clone(), txn_ext: peer.txn_ext.clone(), read_progress: peer.read_progress.clone(), + pending_remove: false, track_ver: TrackVer::new(), } } @@ -238,6 +240,11 @@ impl ReadDelegate { self.last_valid_ts = monotonic_raw_now(); } + pub fn mark_pending_remove(&mut self) { + self.pending_remove = true; + self.track_ver.inc(); + } + pub fn update(&mut self, progress: Progress) { self.fresh_valid_ts(); self.track_ver.inc(); @@ -449,7 +456,7 @@ where // violated, which is required by `LocalReadRouter: Send`, use `Arc` will introduce extra cost but // make the logic clear fn get_delegate(&mut self, region_id: u64) -> Option> { - match self.delegates.get(®ion_id) { + let rd = match self.delegates.get(®ion_id) { // The local `ReadDelegate` is up to date Some(d) if !d.track_ver.any_new() => Some(Arc::clone(d)), _ => { @@ -475,7 +482,9 @@ where None => None, } } - } + }; + // Return `None` if the read delegate is pending remove + rd.filter(|r| !r.pending_remove) } fn pre_propose_raft_command( @@ -968,6 +977,7 @@ mod tests { txn_extra_op: Arc::new(AtomicCell::new(TxnExtraOp::default())), txn_ext: Arc::new(TxnExt::default()), read_progress: read_progress.clone(), + pending_remove: false, track_ver: TrackVer::new(), }; meta.readers.insert(1, read_delegate); @@ -1210,6 +1220,7 @@ mod tests { txn_ext: Arc::new(TxnExt::default()), track_ver: TrackVer::new(), read_progress: Arc::new(RegionReadProgress::new(®ion, 0, 0, "".to_owned())), + pending_remove: false, }; meta.readers.insert(1, read_delegate); } diff --git a/tests/failpoints/cases/test_merge.rs b/tests/failpoints/cases/test_merge.rs index 0bc098592d6..39ee45782ae 100644 --- a/tests/failpoints/cases/test_merge.rs +++ b/tests/failpoints/cases/test_merge.rs @@ -1302,3 +1302,52 @@ fn test_merge_election_and_restart() { // If logs from different term are process correctly, store 2 should have latest updates. must_get_equal(&cluster.get_engine(2), b"k14", b"v14"); } + +/// Testing that the source peer's read delegate should not be removed by the target peer +/// and only removed when the peer is destroyed +#[test] +fn test_source_peer_read_delegate_after_apply() { + let mut cluster = new_node_cluster(0, 3); + configure_for_merge(&mut cluster); + + let pd_client = Arc::clone(&cluster.pd_client); + pd_client.disable_default_operator(); + + cluster.run(); + + cluster.must_split(&cluster.get_region(b""), b"k2"); + let target = cluster.get_region(b"k1"); + let source = cluster.get_region(b"k3"); + + cluster.must_transfer_leader(target.get_id(), find_peer(&target, 1).unwrap().to_owned()); + + let on_destroy_peer_fp = "destroy_peer"; + fail::cfg(on_destroy_peer_fp, "pause").unwrap(); + + // Merge finish means the leader of the target region have call `on_ready_commit_merge` + pd_client.must_merge(source.get_id(), target.get_id()); + + // The source peer's `ReadDelegate` should not be removed yet and mark as `pending_remove` + assert!( + cluster.store_metas[&1] + .lock() + .unwrap() + .readers + .get(&source.get_id()) + .unwrap() + .pending_remove + ); + + fail::remove(on_destroy_peer_fp); + // Wait for source peer is destroyed + sleep_ms(100); + + assert!( + cluster.store_metas[&1] + .lock() + .unwrap() + .readers + .get(&source.get_id()) + .is_none() + ); +} From 292dcf253ea8cc136211157e8b5137e69b8115ea Mon Sep 17 00:00:00 2001 From: Zwb Date: Thu, 20 Jan 2022 19:35:45 +0800 Subject: [PATCH 10/16] logger: use eprintln! if the logger is not initialized (#11869) ref #11651 Signed-off-by: Wenbo Zhang Co-authored-by: Ti Chi Robot --- src/config.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/config.rs b/src/config.rs index a5812fd3fbd..cb020c9d5c6 100644 --- a/src/config.rs +++ b/src/config.rs @@ -2829,17 +2829,17 @@ impl TiKvConfig { let default_tikv_cfg = TiKvConfig::default(); let default_log_cfg = LogConfig::default(); if self.log_level != default_tikv_cfg.log_level { - println!("deprecated configuration, log-level has been moved to log.level"); + eprintln!("deprecated configuration, log-level has been moved to log.level"); if self.log.level == default_log_cfg.level { - println!("override log.level with log-level, {:?}", self.log_level); + eprintln!("override log.level with log-level, {:?}", self.log_level); self.log.level = self.log_level; } self.log_level = default_tikv_cfg.log_level; } if self.log_file != default_tikv_cfg.log_file { - println!("deprecated configuration, log-file has been moved to log.file.filename"); + eprintln!("deprecated configuration, log-file has been moved to log.file.filename"); if self.log.file.filename == default_log_cfg.file.filename { - println!( + eprintln!( "override log.file.filename with log-file, {:?}", self.log_file ); @@ -2848,25 +2848,25 @@ impl TiKvConfig { self.log_file = default_tikv_cfg.log_file; } if self.log_format != default_tikv_cfg.log_format { - println!("deprecated configuration, log-format has been moved to log.format"); + eprintln!("deprecated configuration, log-format has been moved to log.format"); if self.log.format == default_log_cfg.format { - println!("override log.format with log-format, {:?}", self.log_format); + eprintln!("override log.format with log-format, {:?}", self.log_format); self.log.format = self.log_format; } self.log_format = default_tikv_cfg.log_format; } if self.log_rotation_timespan.as_secs() > 0 { - println!( + eprintln!( "deprecated configuration, log-rotation-timespan is no longer used and ignored." ); } if self.log_rotation_size != default_tikv_cfg.log_rotation_size { - println!( + eprintln!( "deprecated configuration, \ log-ratation-size has been moved to log.file.max-size" ); if self.log.file.max_size == default_log_cfg.file.max_size { - println!( + eprintln!( "override log.file.max_size with log-rotation-size, {:?}", self.log_rotation_size ); From 66ee8eb3a1268c1bd34c44c91d25260b213e4da2 Mon Sep 17 00:00:00 2001 From: Yifan Xu <30385241+xuyifangreeneyes@users.noreply.github.com> Date: Thu, 20 Jan 2022 20:15:46 +0800 Subject: [PATCH 11/16] copr: fix the calculation of total column size in analyze (#11884) Signed-off-by: xuyifan Co-authored-by: Ti Chi Robot --- src/coprocessor/statistics/analyze.rs | 8 ++++---- tests/integrations/coprocessor/test_analyze.rs | 6 ++++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/coprocessor/statistics/analyze.rs b/src/coprocessor/statistics/analyze.rs index d2a7e5943c4..49832e17822 100644 --- a/src/coprocessor/statistics/analyze.rs +++ b/src/coprocessor/statistics/analyze.rs @@ -499,7 +499,7 @@ impl BaseRowSampleCollector { continue; } has_null = false; - self.total_sizes[col_len + i] += columns_val[*j as usize].len() as i64 + self.total_sizes[col_len + i] += columns_val[*j as usize].len() as i64 - 1 } // We only maintain the null count for single column case. if has_null && offsets.len() == 1 { @@ -535,7 +535,7 @@ impl BaseRowSampleCollector { } else { self.fm_sketches[i].insert(&columns_val[i]); } - self.total_sizes[i] += columns_val[i].len() as i64; + self.total_sizes[i] += columns_val[i].len() as i64 - 1; } } @@ -1043,7 +1043,7 @@ impl SampleCollector { if let Some(c) = self.cm_sketch.as_mut() { c.insert(&data); } - self.total_size += data.len() as u64; + self.total_size += data.len() as u64 - 1; if self.samples.len() < self.max_sample_size { self.samples.push(data); return; @@ -1188,7 +1188,7 @@ mod tests { assert_eq!(sample.null_count, 1); assert_eq!(sample.count, 3); assert_eq!(sample.cm_sketch.unwrap().count(), 3); - assert_eq!(sample.total_size, 6) + assert_eq!(sample.total_size, 3) } #[test] diff --git a/tests/integrations/coprocessor/test_analyze.rs b/tests/integrations/coprocessor/test_analyze.rs index 068e6e1ec8c..89da5fade70 100644 --- a/tests/integrations/coprocessor/test_analyze.rs +++ b/tests/integrations/coprocessor/test_analyze.rs @@ -165,6 +165,8 @@ fn test_analyze_column() { assert_eq!(rows.len(), 4); let sum: u32 = rows.first().unwrap().get_counters().iter().sum(); assert_eq!(sum, 3); + assert_eq!(collectors[0].get_total_size(), 21); + assert_eq!(collectors[1].get_total_size(), 4); } #[test] @@ -297,7 +299,7 @@ fn test_analyze_sampling_reservoir() { assert_eq!(collector.get_null_counts(), vec![0, 1, 0, 1]); assert_eq!(collector.get_count(), 9); assert_eq!(collector.get_fm_sketch().len(), 4); - assert_eq!(collector.get_total_size(), vec![81, 64, 18, 64]); + assert_eq!(collector.get_total_size(), vec![72, 56, 9, 56]); } #[test] @@ -328,7 +330,7 @@ fn test_analyze_sampling_bernoulli() { assert_eq!(collector.get_null_counts(), vec![0, 1, 0, 1]); assert_eq!(collector.get_count(), 9); assert_eq!(collector.get_fm_sketch().len(), 4); - assert_eq!(collector.get_total_size(), vec![81, 64, 18, 64]); + assert_eq!(collector.get_total_size(), vec![72, 56, 9, 56]); } #[test] From e8718be24177a7218c317ade7d8055a3a1c4d6b5 Mon Sep 17 00:00:00 2001 From: Xinye Tao Date: Fri, 21 Jan 2022 13:56:39 +0800 Subject: [PATCH 12/16] engine: update raft-engine for data consistency fix (#11885) Fix https://github.com/tikv/raft-engine/issues/142. When encountering this bug, TiKV will panic with message "applied index > max(commit index, recorded commit index)" after restart. * update raft-engine Signed-off-by: tabokie * update one more commit Signed-off-by: tabokie --- Cargo.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index cb4182c2963..15aacbb82ed 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3638,7 +3638,7 @@ dependencies = [ [[package]] name = "raft-engine" version = "0.1.0" -source = "git+https://github.com/tikv/raft-engine?branch=master#7842aad23f36ffe836b47762f85fc752b5d7ad48" +source = "git+https://github.com/tikv/raft-engine?branch=master#cc0fa30612c7f8e849436681a468996d1624682a" dependencies = [ "byteorder", "crc32fast", From 434fb6e218bda164a79d7ebb7e290de67d39e0dd Mon Sep 17 00:00:00 2001 From: Xinye Tao Date: Mon, 24 Jan 2022 18:36:46 +0800 Subject: [PATCH 13/16] check docker build during clippy, ref #11312 (#11819) Signed-off-by: tabokie --- Makefile | 1 + scripts/check-docker-build | 19 +++++++++++++++++++ scripts/check-redact-log | 12 ++++++------ 3 files changed, 26 insertions(+), 6 deletions(-) create mode 100755 scripts/check-docker-build diff --git a/Makefile b/Makefile index 1f330ce3ec5..355e7e598e6 100644 --- a/Makefile +++ b/Makefile @@ -283,6 +283,7 @@ pre-clippy: unset-override clippy: pre-clippy @./scripts/check-redact-log + @./scripts/check-docker-build @./scripts/clippy-all pre-audit: diff --git a/scripts/check-docker-build b/scripts/check-docker-build new file mode 100755 index 00000000000..26a53cc1ef6 --- /dev/null +++ b/scripts/check-docker-build @@ -0,0 +1,19 @@ +#!/usr/bin/env bash +# This script checks if all cargo targets have path specifications. +set -euo pipefail + +for i in $(find . -type f -name 'Cargo.toml'); do + # These folders are excluded from docker build. + if echo $i | grep -q "./fuzz/\|./profiler/"; then + continue + fi + for target in "test" "bench" "bin" "example"; do + matches=$(sed -n "/\[\[$target\]\]/,/^$/ p" $i) + if [ $(echo "$matches" | grep -c "[[$target]]") != $(echo "$matches" | grep -c "^path =") ]; then + echo "Path has not been specified for a $target target in $i, this will break docker build." + exit 1 + fi + done +done + +echo "Docker build check passed." diff --git a/scripts/check-redact-log b/scripts/check-redact-log index ec35f558bf1..880de323700 100755 --- a/scripts/check-redact-log +++ b/scripts/check-redact-log @@ -3,19 +3,19 @@ set -euo pipefail function error_msg() { - echo "To print user data into info logs or error messages, use log_wrappers::Value() instead of hex::encode_upper. The former will respect \`security.redact-info-log\` config and filter out user data from info log if needed. Otherwise, use \`log_wrappers::hex_encode_upper\` to get around the lint error. See https://github.com/tikv/tikv/pull/9250 for more information." >&2 + echo "To print user data into info logs or error messages, use log_wrappers::Value() instead of hex::encode_upper. The former will respect \`security.redact-info-log\` config and filter out user data from info log if needed. Otherwise, use \`log_wrappers::hex_encode_upper\` to get around the lint error. See https://github.com/tikv/tikv/pull/9250 for more information." >&2 } if [[ "$(uname)" == "Darwin" ]] ; then if grep -r -n --color=always --include '*.rs' --exclude hex.rs --exclude-dir tikv-ctl --exclude-dir target 'encode_upper' . | grep -v log_wrappers ; then - error_msg - exit 1 + error_msg + exit 1 fi else if grep -r -n -P '(? Date: Tue, 25 Jan 2022 13:20:12 +0800 Subject: [PATCH 14/16] gc_worker: fix incorrect scheduled_tasks counting (#11904) * gc_worker: fix incorrect scheduled_tasks counting close #11903 Signed-off-by: Yilin Chen * remove check_is_busy totally Signed-off-by: Yilin Chen * do not use wildcard match in error handling Signed-off-by: Yilin Chen --- components/tikv_util/src/worker/pool.rs | 8 ++ src/server/gc_worker/gc_worker.rs | 179 ++++++++++++++++-------- 2 files changed, 131 insertions(+), 56 deletions(-) diff --git a/components/tikv_util/src/worker/pool.rs b/components/tikv_util/src/worker/pool.rs index 66f1926bbb4..655081ceff0 100644 --- a/components/tikv_util/src/worker/pool.rs +++ b/components/tikv_util/src/worker/pool.rs @@ -115,6 +115,14 @@ impl Scheduler { if self.counter.load(Ordering::Acquire) >= self.pending_capacity { return Err(ScheduleError::Full(task)); } + self.schedule_force(task) + } + + /// Schedules a task to run. + /// + /// Different from the `schedule` function, the task will still be scheduled + /// if pending task number exceeds capacity. + pub fn schedule_force(&self, task: T) -> Result<(), ScheduleError> { self.counter.fetch_add(1, Ordering::SeqCst); self.metrics_pending_task_count.inc(); if let Err(e) = self.sender.unbounded_send(Msg::Task(task)) { diff --git a/src/server/gc_worker/gc_worker.rs b/src/server/gc_worker/gc_worker.rs index 0a6e6bc4365..be4c8dfda04 100644 --- a/src/server/gc_worker/gc_worker.rs +++ b/src/server/gc_worker/gc_worker.rs @@ -668,7 +668,20 @@ where /// When we failed to schedule a `GcTask` to `GcRunner`, use this to handle the `ScheduleError`. fn handle_gc_task_schedule_error(e: ScheduleError>) -> Result<()> { error!("failed to schedule gc task"; "err" => %e); - Err(box_err!("failed to schedule gc task: {:?}", e)) + let res = Err(box_err!("failed to schedule gc task: {:?}", e)); + match e.into_inner() { + GcTask::Gc { callback, .. } | GcTask::UnsafeDestroyRange { callback, .. } => { + callback(Err(Error::from(ErrorInner::GcWorkerTooBusy))) + } + GcTask::PhysicalScanLock { callback, .. } => { + callback(Err(Error::from(ErrorInner::GcWorkerTooBusy))) + } + // Attention: If you are adding a new GcTask, do not forget to call the callback if it has a callback. + GcTask::GcKeys { .. } | GcTask::OrphanVersions { .. } => {} + #[cfg(any(test, feature = "testexport"))] + GcTask::Validate(_) => {} + } + res } /// Schedules a `GcTask` to the `GcRunner`. @@ -723,9 +736,6 @@ where config_manager: GcWorkerConfigManager, - /// How many requests are scheduled from outside and unfinished. - scheduled_tasks: Arc, - /// How many strong references. The worker will be stopped /// once there are no more references. refs: Arc, @@ -752,7 +762,6 @@ where raft_store_router: self.raft_store_router.clone(), flow_info_sender: self.flow_info_sender.clone(), config_manager: self.config_manager.clone(), - scheduled_tasks: self.scheduled_tasks.clone(), refs: self.refs.clone(), worker: self.worker.clone(), worker_scheduler: self.worker_scheduler.clone(), @@ -803,7 +812,6 @@ where raft_store_router, flow_info_sender: Some(flow_info_sender), config_manager: GcWorkerConfigManager(Arc::new(VersionTrack::new(cfg))), - scheduled_tasks: Arc::new(AtomicUsize::new(0)), refs: Arc::new(AtomicUsize::new(1)), worker: Arc::new(Mutex::new(worker)), worker_scheduler, @@ -888,37 +896,19 @@ where self.worker_scheduler.clone() } - /// Check whether GCWorker is busy. If busy, callback will be invoked with an error that - /// indicates GCWorker is busy; otherwise, return a new callback that invokes the original - /// callback as well as decrease the scheduled task counter. - fn check_is_busy(&self, callback: Callback) -> Option> { - if self.scheduled_tasks.fetch_add(1, Ordering::SeqCst) >= GC_MAX_EXECUTING_TASKS { - self.scheduled_tasks.fetch_sub(1, Ordering::SeqCst); - callback(Err(Error::from(ErrorInner::GcWorkerTooBusy))); - return None; - } - let scheduled_tasks = Arc::clone(&self.scheduled_tasks); - Some(Box::new(move |r| { - scheduled_tasks.fetch_sub(1, Ordering::SeqCst); - callback(r); - })) - } - /// Only for tests. pub fn gc(&self, safe_point: TimeStamp, callback: Callback<()>) -> Result<()> { - self.check_is_busy(callback).map_or(Ok(()), |callback| { - let start_key = vec![]; - let end_key = vec![]; - self.worker_scheduler - .schedule(GcTask::Gc { - region_id: 0, - start_key, - end_key, - safe_point, - callback, - }) - .or_else(handle_gc_task_schedule_error) - }) + let start_key = vec![]; + let end_key = vec![]; + self.worker_scheduler + .schedule(GcTask::Gc { + region_id: 0, + start_key, + end_key, + safe_point, + callback, + }) + .or_else(handle_gc_task_schedule_error) } /// Cleans up all keys in a range and quickly free the disk space. The range might span over @@ -934,16 +924,20 @@ where callback: Callback<()>, ) -> Result<()> { GC_COMMAND_COUNTER_VEC_STATIC.unsafe_destroy_range.inc(); - self.check_is_busy(callback).map_or(Ok(()), |callback| { - self.worker_scheduler - .schedule(GcTask::UnsafeDestroyRange { - ctx, - start_key, - end_key, - callback, - }) - .or_else(handle_gc_task_schedule_error) - }) + + // Use schedule_force to allow unsafe_destroy_range to schedule even if + // the GC worker is full. This will help free up space in the case when + // the GC worker is busy with other tasks. + // Unsafe destroy range is in store level, so the number of them is + // quite small, so we don't need to worry about its memory usage. + self.worker_scheduler + .schedule_force(GcTask::UnsafeDestroyRange { + ctx, + start_key, + end_key, + callback, + }) + .or_else(handle_gc_task_schedule_error) } pub fn get_config_manager(&self) -> GcWorkerConfigManager { @@ -959,17 +953,16 @@ where callback: Callback>, ) -> Result<()> { GC_COMMAND_COUNTER_VEC_STATIC.physical_scan_lock.inc(); - self.check_is_busy(callback).map_or(Ok(()), |callback| { - self.worker_scheduler - .schedule(GcTask::PhysicalScanLock { - ctx, - max_ts, - start_key, - limit, - callback, - }) - .or_else(handle_gc_task_schedule_error) - }) + + self.worker_scheduler + .schedule(GcTask::PhysicalScanLock { + ctx, + max_ts, + start_key, + limit, + callback, + }) + .or_else(handle_gc_task_schedule_error) } pub fn start_collecting( @@ -1564,4 +1557,78 @@ mod tests { assert_eq!(runner.stats.write.seek, 1); assert_eq!(runner.stats.write.next, 100 * 2); } + + #[test] + fn delete_range_when_worker_is_full() { + let engine = PrefixedEngine(TestEngineBuilder::new().build().unwrap()); + must_prewrite_put(&engine, b"key", b"value", b"key", 10); + must_commit(&engine, b"key", 10, 20); + let db = engine.kv_engine().as_inner().clone(); + let cf = get_cf_handle(&db, CF_WRITE).unwrap(); + db.flush_cf(cf, true).unwrap(); + + let gate = FeatureGate::default(); + gate.set_version("5.0.0").unwrap(); + let (tx, _rx) = mpsc::channel(); + + let mut gc_worker = GcWorker::new( + engine.clone(), + RaftStoreBlackHole, + tx, + GcConfig::default(), + gate, + ); + + // Before starting gc_worker, fill the scheduler to full. + for _ in 0..GC_MAX_PENDING_TASKS { + assert!( + gc_worker + .scheduler() + .schedule(GcTask::Gc { + region_id: 0, + start_key: vec![], + end_key: vec![], + safe_point: TimeStamp::from(100), + callback: Box::new(|_res| {}) + }) + .is_ok() + ); + } + // Then, it will fail to schedule another gc command. + let (tx, rx) = mpsc::channel(); + assert!( + gc_worker + .gc( + TimeStamp::from(1), + Box::new(move |res| { + tx.send(res).unwrap(); + }) + ) + .is_err() + ); + assert!(rx.recv().unwrap().is_err()); + + let (tx, rx) = mpsc::channel(); + // When the gc_worker is full, scheduling an unsafe destroy range task should be + // still allowed. + assert!( + gc_worker + .unsafe_destroy_range( + Context::default(), + Key::from_raw(b"a"), + Key::from_raw(b"z"), + Box::new(move |res| { + tx.send(res).unwrap(); + }) + ) + .is_ok() + ); + + gc_worker.start().unwrap(); + + // After the worker starts running, the destroy range task should run, + // and the key in the range will be deleted. + assert!(rx.recv_timeout(Duration::from_secs(10)).unwrap().is_ok()); + must_get_none(&engine, b"key", 30); + } } From 790c744e582d4fddfab2b884b40d7d5af14a47e1 Mon Sep 17 00:00:00 2001 From: Connor Date: Tue, 25 Jan 2022 15:12:11 +0800 Subject: [PATCH 15/16] raftstore: Introduce raft log fetcher (#11900) * ref#11409 introduce raft log fetcher Signed-off-by: Connor1996 * update kvproto Signed-off-by: Connor1996 * address comment Signed-off-by: Connor1996 * fix test build Signed-off-by: Connor1996 * fix clippy Signed-off-by: Connor1996 * limit capacity Signed-off-by: Connor1996 * update kvproto Signed-off-by: Connor1996 * call stop on worker Signed-off-by: Connor1996 * rename worker Signed-off-by: Connor1996 --- Cargo.lock | 10 +- Cargo.toml | 2 +- cmd/tikv-ctl/Cargo.toml | 2 +- components/backup/Cargo.toml | 2 +- components/cdc/Cargo.toml | 4 +- components/engine_panic/Cargo.toml | 2 +- components/engine_rocks/Cargo.toml | 2 +- components/engine_rocks/src/raft_engine.rs | 7 +- components/engine_traits/Cargo.toml | 2 +- components/error_code/Cargo.toml | 2 +- components/into_other/Cargo.toml | 2 +- components/raft_log_engine/Cargo.toml | 2 +- components/raftstore/Cargo.toml | 4 +- components/raftstore/src/store/fsm/apply.rs | 4 + components/raftstore/src/store/fsm/peer.rs | 51 +++-- components/raftstore/src/store/fsm/store.rs | 23 ++- components/raftstore/src/store/metrics.rs | 26 ++- components/raftstore/src/store/mod.rs | 8 +- components/raftstore/src/store/msg.rs | 25 ++- components/raftstore/src/store/peer.rs | 30 ++- .../raftstore/src/store/peer_storage.rs | 189 ++++++++++++------ .../raftstore/src/store/region_snapshot.rs | 13 +- components/raftstore/src/store/worker/mod.rs | 2 + .../src/store/worker/raftlog_fetch.rs | 116 +++++++++++ .../raftstore/src/store/worker/region.rs | 2 +- components/resolved_ts/Cargo.toml | 4 +- components/server/Cargo.toml | 2 +- components/test_raftstore/Cargo.toml | 2 +- metrics/grafana/tikv_details.json | 2 +- src/server/debug.rs | 6 +- tests/Cargo.toml | 2 +- 31 files changed, 430 insertions(+), 120 deletions(-) create mode 100644 components/raftstore/src/store/worker/raftlog_fetch.rs diff --git a/Cargo.lock b/Cargo.lock index 15aacbb82ed..c728f105b64 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2357,7 +2357,7 @@ dependencies = [ [[package]] name = "kvproto" version = "0.0.2" -source = "git+https://github.com/pingcap/kvproto.git#7a8280c36daf05e44f72130f3fd4e585ea2c62d8" +source = "git+https://github.com/pingcap/kvproto.git#a3c198c11ad44bf57b72f6d6fd542603714aeda3" dependencies = [ "futures 0.3.15", "grpcio", @@ -3622,8 +3622,8 @@ dependencies = [ [[package]] name = "raft" -version = "0.6.0" -source = "git+https://github.com/tikv/raft-rs?branch=master#e6d28ef0f509c072e13288409835d5764a3b175e" +version = "0.7.0" +source = "git+https://github.com/tikv/raft-rs?branch=master#f1f95f86d1ef6ca103ddb522ffa093a31e5af632" dependencies = [ "bytes 1.0.1", "fxhash", @@ -3665,8 +3665,8 @@ dependencies = [ [[package]] name = "raft-proto" -version = "0.6.0" -source = "git+https://github.com/tikv/raft-rs?branch=master#e6d28ef0f509c072e13288409835d5764a3b175e" +version = "0.7.0" +source = "git+https://github.com/tikv/raft-rs?branch=master#f1f95f86d1ef6ca103ddb522ffa093a31e5af632" dependencies = [ "bytes 1.0.1", "protobuf", diff --git a/Cargo.toml b/Cargo.toml index 124134e2044..455a83f1a3c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -112,7 +112,7 @@ pin-project = "1.0" pnet_datalink = "0.23" pprof = { version = "^0.6", default-features = false, features = ["flamegraph", "protobuf"] } protobuf = { version = "2.8", features = ["bytes"] } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raftstore = { path = "components/raftstore", default-features = false } raft_log_engine = { path = "components/raft_log_engine", default-features = false } rand = "0.7.3" diff --git a/cmd/tikv-ctl/Cargo.toml b/cmd/tikv-ctl/Cargo.toml index 87af06f42dc..2d98feec972 100644 --- a/cmd/tikv-ctl/Cargo.toml +++ b/cmd/tikv-ctl/Cargo.toml @@ -67,7 +67,7 @@ pd_client = { path = "../../components/pd_client", default-features = false } prometheus = { version = "0.13", features = ["nightly"] } promptly = "0.3.0" protobuf = { version = "2.8", features = ["bytes"] } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raft_log_engine = { path = "../../components/raft_log_engine", default-features = false } raftstore = { path = "../../components/raftstore", default-features = false } rand = "0.8" diff --git a/components/backup/Cargo.toml b/components/backup/Cargo.toml index 13649551040..7625fbafd79 100644 --- a/components/backup/Cargo.toml +++ b/components/backup/Cargo.toml @@ -47,7 +47,7 @@ lazy_static = "1.3" log_wrappers = { path = "../log_wrappers" } pd_client = { path = "../pd_client", default-features = false } prometheus = { version = "0.13", default-features = false, features = ["nightly"] } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raftstore = { path = "../raftstore", default-features = false } security = { path = "../security", default-features = false } serde = "1.0" diff --git a/components/cdc/Cargo.toml b/components/cdc/Cargo.toml index ba769f8e7df..f6847acd3cb 100644 --- a/components/cdc/Cargo.toml +++ b/components/cdc/Cargo.toml @@ -31,7 +31,7 @@ grpcio = { version = "0.9", default-features = false, features = ["openssl-vend keys = { path = "../keys" } kvproto = { git = "https://github.com/pingcap/kvproto.git" } pd_client = { path = "../pd_client", default-features = false } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raftstore = { path = "../raftstore", default-features = false } resolved_ts = { path = "../resolved_ts", default-features = false } security = { path = "../security", default-features = false } @@ -62,7 +62,7 @@ tempfile = "3.0" test_raftstore = { path = "../test_raftstore", default-features = false } test_util = { path = "../test_util", default-features = false } panic_hook = { path = "../panic_hook" } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } time = "0.1" criterion = "0.3" diff --git a/components/engine_panic/Cargo.toml b/components/engine_panic/Cargo.toml index be75dd7ff38..8ccdccc2798 100644 --- a/components/engine_panic/Cargo.toml +++ b/components/engine_panic/Cargo.toml @@ -11,5 +11,5 @@ tikv_alloc = { path = "../tikv_alloc" } # FIXME: Remove this dep from the engine_traits interface tikv_util = { path = "../tikv_util", default-features = false } kvproto = { git = "https://github.com/pingcap/kvproto.git" } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } txn_types = { path = "../txn_types", default-features = false } diff --git a/components/engine_rocks/Cargo.toml b/components/engine_rocks/Cargo.toml index 8cb15be2163..e904e4dce13 100644 --- a/components/engine_rocks/Cargo.toml +++ b/components/engine_rocks/Cargo.toml @@ -45,7 +45,7 @@ tempfile = "3.0" serde = "1.0" serde_derive = "1.0" kvproto = { git = "https://github.com/pingcap/kvproto.git" } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } protobuf = "2" fail = "0.5" case_macros = { path = "../case_macros" } diff --git a/components/engine_rocks/src/raft_engine.rs b/components/engine_rocks/src/raft_engine.rs index df64211f090..b5832e10968 100644 --- a/components/engine_rocks/src/raft_engine.rs +++ b/components/engine_rocks/src/raft_engine.rs @@ -59,7 +59,7 @@ impl RaftEngineReadOnly for RocksEngine { return Ok(count); } - let (mut check_compacted, mut next_index) = (true, low); + let (mut check_compacted, mut compacted, mut next_index) = (true, false, low); let start_key = keys::raft_log_key(region_id, low); let end_key = keys::raft_log_key(region_id, high); self.scan( @@ -72,6 +72,7 @@ impl RaftEngineReadOnly for RocksEngine { if check_compacted { if entry.get_index() != low { + compacted = true; // May meet gap or has been compacted. return Ok(false); } @@ -94,6 +95,10 @@ impl RaftEngineReadOnly for RocksEngine { return Ok(count); } + if compacted { + return Err(Error::EntriesCompacted); + } + // Here means we don't fetch enough entries. Err(Error::EntriesUnavailable) } diff --git a/components/engine_traits/Cargo.toml b/components/engine_traits/Cargo.toml index 1bc3e25b278..6b631fddccb 100644 --- a/components/engine_traits/Cargo.toml +++ b/components/engine_traits/Cargo.toml @@ -20,7 +20,7 @@ serde = "1.0" slog = { version = "2.3", features = ["max_level_trace", "release_max_level_debug"] } slog-global = { version = "0.1", git = "https://github.com/breeswish/slog-global.git", rev = "d592f88e4dbba5eb439998463054f1a44fbf17b9" } kvproto = { git = "https://github.com/pingcap/kvproto.git" } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } fail = "0.5" case_macros = { path = "../case_macros" } diff --git a/components/error_code/Cargo.toml b/components/error_code/Cargo.toml index f13fcbc6c38..e39d9341a71 100644 --- a/components/error_code/Cargo.toml +++ b/components/error_code/Cargo.toml @@ -14,7 +14,7 @@ path = "bin.rs" [dependencies] lazy_static = "1.3" -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } kvproto = { git = "https://github.com/pingcap/kvproto.git" } tikv_alloc = { path = "../tikv_alloc" } grpcio = { version = "0.9", default-features = false, features = ["openssl-vendored", "protobuf-codec"] } diff --git a/components/into_other/Cargo.toml b/components/into_other/Cargo.toml index 6b1c4f20bac..be278cdc764 100644 --- a/components/into_other/Cargo.toml +++ b/components/into_other/Cargo.toml @@ -7,4 +7,4 @@ publish = false [dependencies] engine_traits = { path = "../engine_traits", default-features = false } kvproto = { git = "https://github.com/pingcap/kvproto.git" } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } diff --git a/components/raft_log_engine/Cargo.toml b/components/raft_log_engine/Cargo.toml index fdf6aa8a460..b3a5fc15a0a 100644 --- a/components/raft_log_engine/Cargo.toml +++ b/components/raft_log_engine/Cargo.toml @@ -18,6 +18,6 @@ online_config = { path = "../online_config" } serde = "1.0" serde_derive = "1.0" kvproto = { git = "https://github.com/pingcap/kvproto.git" } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raft-engine = { git = "https://github.com/tikv/raft-engine", branch = "master" } protobuf = "2" diff --git a/components/raftstore/Cargo.toml b/components/raftstore/Cargo.toml index ac507d849b7..ffd32dea794 100644 --- a/components/raftstore/Cargo.toml +++ b/components/raftstore/Cargo.toml @@ -54,8 +54,8 @@ pd_client = { path = "../pd_client", default-features = false } prometheus = { version = "0.13", features = ["nightly"] } prometheus-static-metric = "0.5" protobuf = { version = "2.8", features = ["bytes"] } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } -raft-proto = { version = "0.6.0-alpha", default-features = false } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } +raft-proto = { version = "0.7.0", default-features = false } rand = "0.8.3" resource_metering = { path = "../resource_metering" } serde = "1.0" diff --git a/components/raftstore/src/store/fsm/apply.rs b/components/raftstore/src/store/fsm/apply.rs index 1ca6724f17d..3c21690906a 100644 --- a/components/raftstore/src/store/fsm/apply.rs +++ b/components/raftstore/src/store/fsm/apply.rs @@ -1504,6 +1504,7 @@ where ctx: &mut ApplyContext, req: &Request, ) -> Result<()> { + PEER_WRITE_CMD_COUNTER.put.inc(); let (key, value) = (req.get_put().get_key(), req.get_put().get_value()); // region key range has no data prefix, so we must use origin key to check. util::check_key_in_region(key, &self.region)?; @@ -1550,6 +1551,7 @@ where ctx: &mut ApplyContext, req: &Request, ) -> Result<()> { + PEER_WRITE_CMD_COUNTER.delete.inc(); let key = req.get_delete().get_key(); // region key range has no data prefix, so we must use origin key to check. util::check_key_in_region(key, &self.region)?; @@ -1599,6 +1601,7 @@ where ranges: &mut Vec, use_delete_range: bool, ) -> Result<()> { + PEER_WRITE_CMD_COUNTER.delete_range.inc(); let s_key = req.get_delete_range().get_start_key(); let e_key = req.get_delete_range().get_end_key(); let notify_only = req.get_delete_range().get_notify_only(); @@ -1670,6 +1673,7 @@ where req: &Request, ssts: &mut Vec, ) -> Result<()> { + PEER_WRITE_CMD_COUNTER.ingest_sst.inc(); let sst = req.get_ingest_sst().get_sst(); if let Err(e) = check_sst_for_ingestion(sst, &self.region) { diff --git a/components/raftstore/src/store/fsm/peer.rs b/components/raftstore/src/store/fsm/peer.rs index b8a1110cdc7..2413198c30a 100644 --- a/components/raftstore/src/store/fsm/peer.rs +++ b/components/raftstore/src/store/fsm/peer.rs @@ -35,7 +35,10 @@ use kvproto::replication_modepb::{DrAutoSyncState, ReplicationMode}; use parking_lot::RwLockWriteGuard; use protobuf::Message; use raft::eraftpb::{self, ConfChangeType, MessageType}; -use raft::{self, Progress, ReadState, SnapshotStatus, StateRole, INVALID_INDEX, NO_LIMIT}; +use raft::{ + self, GetEntriesContext, Progress, ReadState, SnapshotStatus, StateRole, INVALID_INDEX, + NO_LIMIT, +}; use smallvec::SmallVec; use tikv_alloc::trace::TraceEvent; use tikv_util::mpsc::{self, LooseBoundedSender, Receiver}; @@ -68,7 +71,7 @@ use crate::store::read_queue::ReadIndexRequest; use crate::store::transport::Transport; use crate::store::util::{is_learner, KeysInfoFormatter}; use crate::store::worker::{ - ConsistencyCheckTask, RaftlogGcTask, ReadDelegate, RegionTask, SplitCheckTask, + ConsistencyCheckTask, RaftlogFetchTask, RaftlogGcTask, ReadDelegate, RegionTask, SplitCheckTask, }; use crate::store::PdTask; use crate::store::{ @@ -211,7 +214,8 @@ where pub fn create( store_id: u64, cfg: &Config, - sched: Scheduler>, + region_scheduler: Scheduler>, + raftlog_fetch_scheduler: Scheduler, engines: Engines, region: &metapb::Region, ) -> Result> { @@ -236,7 +240,15 @@ where Ok(( tx, Box::new(PeerFsm { - peer: Peer::new(store_id, cfg, sched, engines, region, meta_peer)?, + peer: Peer::new( + store_id, + cfg, + region_scheduler, + raftlog_fetch_scheduler, + engines, + region, + meta_peer, + )?, tick_registry: [false; PeerTick::VARIANT_COUNT], missing_ticks: 0, hibernate_state: HibernateState::ordered(), @@ -260,7 +272,8 @@ where pub fn replicate( store_id: u64, cfg: &Config, - sched: Scheduler>, + region_scheduler: Scheduler>, + raftlog_fetch_scheduler: Scheduler, engines: Engines, region_id: u64, peer: metapb::Peer, @@ -280,7 +293,15 @@ where Ok(( tx, Box::new(PeerFsm { - peer: Peer::new(store_id, cfg, sched, engines, ®ion, peer)?, + peer: Peer::new( + store_id, + cfg, + region_scheduler, + raftlog_fetch_scheduler, + engines, + ®ion, + peer, + )?, tick_registry: [false; PeerTick::VARIANT_COUNT], missing_ticks: 0, hibernate_state: HibernateState::ordered(), @@ -1162,6 +1183,10 @@ where SignificantMsg::RaftLogGcFlushed => { self.on_raft_log_gc_flushed(); } + SignificantMsg::RaftlogFetched { context, .. } => { + self.fsm.peer.raft_group.on_entries_fetched(context); + self.fsm.has_ready = true; + } } } @@ -2965,6 +2990,7 @@ where self.ctx.store_id(), &self.ctx.cfg, self.ctx.region_scheduler.clone(), + self.ctx.raftlog_fetch_scheduler.clone(), self.ctx.engines.clone(), &new_region, ) { @@ -3210,12 +3236,13 @@ where let entries = if low > state.get_commit() { vec![] } else { - match self - .fsm - .peer - .get_store() - .entries(low, state.get_commit() + 1, NO_LIMIT) - { + // TODO: fetch entries in async way + match self.fsm.peer.get_store().entries( + low, + state.get_commit() + 1, + NO_LIMIT, + GetEntriesContext::empty(false), + ) { Ok(ents) => ents, Err(e) => panic!( "[region {}] {} failed to get merge entires: {:?}, low:{}, commit: {}", diff --git a/components/raftstore/src/store/fsm/store.rs b/components/raftstore/src/store/fsm/store.rs index 1c68b39053a..75cbb0005a2 100644 --- a/components/raftstore/src/store/fsm/store.rs +++ b/components/raftstore/src/store/fsm/store.rs @@ -74,8 +74,8 @@ use crate::store::util::{is_initial_msg, RegionReadProgressRegistry}; use crate::store::worker::{ AutoSplitController, CleanupRunner, CleanupSSTRunner, CleanupSSTTask, CleanupTask, CompactRunner, CompactTask, ConsistencyCheckRunner, ConsistencyCheckTask, PdRunner, - RaftlogGcRunner, RaftlogGcTask, ReadDelegate, RefreshConfigRunner, RefreshConfigTask, - RegionRunner, RegionTask, SplitCheckTask, + RaftlogFetchRunner, RaftlogFetchTask, RaftlogGcRunner, RaftlogGcTask, ReadDelegate, + RefreshConfigRunner, RefreshConfigTask, RegionRunner, RegionTask, SplitCheckTask, }; use crate::store::{ util, Callback, CasualMessage, GlobalReplicationState, InspectedRaftMessage, MergeResultKind, @@ -360,6 +360,7 @@ where // handle Compact, CleanupSST task pub cleanup_scheduler: Scheduler, pub raftlog_gc_scheduler: Scheduler, + pub raftlog_fetch_scheduler: Scheduler, pub region_scheduler: Scheduler>, pub apply_router: ApplyRouter, pub router: RaftRouter, @@ -937,6 +938,7 @@ pub struct RaftPollerBuilder { split_check_scheduler: Scheduler, cleanup_scheduler: Scheduler, raftlog_gc_scheduler: Scheduler, + raftlog_fetch_scheduler: Scheduler, pub region_scheduler: Scheduler>, apply_router: ApplyRouter, pub router: RaftRouter, @@ -1011,6 +1013,7 @@ impl RaftPollerBuilder { store_id, &self.cfg.value(), self.region_scheduler.clone(), + self.raftlog_fetch_scheduler.clone(), self.engines.clone(), region, )); @@ -1050,6 +1053,7 @@ impl RaftPollerBuilder { store_id, &self.cfg.value(), self.region_scheduler.clone(), + self.raftlog_fetch_scheduler.clone(), self.engines.clone(), ®ion, )?; @@ -1155,6 +1159,7 @@ where apply_router: self.apply_router.clone(), router: self.router.clone(), cleanup_scheduler: self.cleanup_scheduler.clone(), + raftlog_fetch_scheduler: self.raftlog_fetch_scheduler.clone(), raftlog_gc_scheduler: self.raftlog_gc_scheduler.clone(), importer: self.importer.clone(), store_meta: self.store_meta.clone(), @@ -1219,6 +1224,7 @@ where split_check_scheduler: self.split_check_scheduler.clone(), cleanup_scheduler: self.cleanup_scheduler.clone(), raftlog_gc_scheduler: self.raftlog_gc_scheduler.clone(), + raftlog_fetch_scheduler: self.raftlog_fetch_scheduler.clone(), region_scheduler: self.region_scheduler.clone(), apply_router: self.apply_router.clone(), router: self.router.clone(), @@ -1251,6 +1257,8 @@ struct Workers { // engine implementations. purge_worker: Worker, + raftlog_fetch_worker: Worker, + coprocessor_host: CoprocessorHost, refresh_config_worker: LazyWorker, @@ -1317,6 +1325,7 @@ impl RaftBatchSystem { cleanup_worker: Worker::new("cleanup-worker"), region_worker: Worker::new("region-worker"), purge_worker: Worker::new("purge-worker"), + raftlog_fetch_worker: Worker::new("raftlog-fetch-worker"), coprocessor_host: coprocessor_host.clone(), refresh_config_worker: LazyWorker::new("refreash-config-worker"), }; @@ -1361,6 +1370,12 @@ impl RaftBatchSystem { }; }, ); + + let raftlog_fetch_scheduler = workers.raftlog_fetch_worker.start( + "raftlog-fetch-worker", + RaftlogFetchRunner::new(self.router.clone(), engines.raft.clone()), + ); + let compact_runner = CompactRunner::new(engines.kv.clone()); let cleanup_sst_runner = CleanupSSTRunner::new( meta.get_id(), @@ -1392,6 +1407,7 @@ impl RaftBatchSystem { consistency_check_scheduler, cleanup_scheduler, raftlog_gc_scheduler, + raftlog_fetch_scheduler, apply_router: self.apply_router.clone(), trans, coprocessor_host, @@ -1557,6 +1573,7 @@ impl RaftBatchSystem { workers.background_worker.stop(); workers.purge_worker.stop(); workers.refresh_config_worker.stop(); + workers.raftlog_fetch_worker.stop(); } } @@ -1973,6 +1990,7 @@ impl<'a, EK: KvEngine, ER: RaftEngine, T: Transport> StoreFsmDelegate<'a, EK, ER self.ctx.store_id(), &self.ctx.cfg, self.ctx.region_scheduler.clone(), + self.ctx.raftlog_fetch_scheduler.clone(), self.ctx.engines.clone(), region_id, target.clone(), @@ -2633,6 +2651,7 @@ impl<'a, EK: KvEngine, ER: RaftEngine, T: Transport> StoreFsmDelegate<'a, EK, ER self.ctx.store.get_id(), &self.ctx.cfg, self.ctx.region_scheduler.clone(), + self.ctx.raftlog_fetch_scheduler.clone(), self.ctx.engines.clone(), ®ion, ) { diff --git a/components/raftstore/src/store/metrics.rs b/components/raftstore/src/store/metrics.rs index 5b976ec17a9..9a3ed329171 100644 --- a/components/raftstore/src/store/metrics.rs +++ b/components/raftstore/src/store/metrics.rs @@ -27,6 +27,13 @@ make_auto_flush_static_metric! { dropped_read_index, } + pub label_enum WriteCmdType { + put, + delete, + delete_range, + ingest_sst, + } + pub label_enum AdminCmdType { conf_change, add_peer, @@ -111,7 +118,11 @@ make_auto_flush_static_metric! { pub label_enum RaftEntryType { hit, - miss + miss, + async_fetch, + sync_fetch, + fallback_fetch, + fetch_invalid, } pub label_enum RaftInvalidProposal { @@ -181,6 +192,10 @@ make_auto_flush_static_metric! { "status" => AdminCmdStatus, } + pub struct WriteCmdVec : LocalIntCounter { + "type" => WriteCmdType, + } + pub struct RaftReadyVec : LocalIntCounter { "type" => RaftReadyType, } @@ -363,6 +378,15 @@ lazy_static! { pub static ref PEER_ADMIN_CMD_COUNTER: AdminCmdVec = auto_flush_from!(PEER_ADMIN_CMD_COUNTER_VEC, AdminCmdVec); + pub static ref PEER_WRITE_CMD_COUNTER_VEC: IntCounterVec = + register_int_counter_vec!( + "tikv_raftstore_write_cmd_total", + "Total number of write cmd processed.", + &["type"] + ).unwrap(); + pub static ref PEER_WRITE_CMD_COUNTER: WriteCmdVec = + auto_flush_from!(PEER_WRITE_CMD_COUNTER_VEC, WriteCmdVec); + pub static ref PEER_COMMIT_LOG_HISTOGRAM: Histogram = register_histogram!( "tikv_raftstore_commit_log_duration_seconds", diff --git a/components/raftstore/src/store/mod.rs b/components/raftstore/src/store/mod.rs index f73446c0746..56d91959704 100644 --- a/components/raftstore/src/store/mod.rs +++ b/components/raftstore/src/store/mod.rs @@ -36,16 +36,16 @@ pub use self::hibernate_state::{GroupState, HibernateState}; pub use self::memory::*; pub use self::msg::{ Callback, CasualMessage, ExtCallback, InspectedRaftMessage, MergeResultKind, PeerMsg, PeerTick, - RaftCmdExtraOpts, RaftCommand, ReadCallback, ReadResponse, SignificantMsg, StoreMsg, StoreTick, - WriteCallback, WriteResponse, + RaftCmdExtraOpts, RaftCommand, RaftlogFetchResult, ReadCallback, ReadResponse, SignificantMsg, + StoreMsg, StoreTick, WriteCallback, WriteResponse, }; pub use self::peer::{ AbstractPeer, Peer, PeerStat, ProposalContext, RequestInspector, RequestPolicy, }; pub use self::peer_storage::{ clear_meta, do_snapshot, write_initial_apply_state, write_initial_raft_state, write_peer_state, - PeerStorage, SnapState, INIT_EPOCH_CONF_VER, INIT_EPOCH_VER, RAFT_INIT_LOG_INDEX, - RAFT_INIT_LOG_TERM, + PeerStorage, SnapState, INIT_EPOCH_CONF_VER, INIT_EPOCH_VER, MAX_INIT_ENTRY_COUNT, + RAFT_INIT_LOG_INDEX, RAFT_INIT_LOG_TERM, }; pub use self::read_queue::ReadIndexContext; pub use self::region_snapshot::{RegionIterator, RegionSnapshot}; diff --git a/components/raftstore/src/store/msg.rs b/components/raftstore/src/store/msg.rs index d994ccabd11..a9722cf4ab2 100644 --- a/components/raftstore/src/store/msg.rs +++ b/components/raftstore/src/store/msg.rs @@ -13,7 +13,7 @@ use kvproto::raft_cmdpb::{RaftCmdRequest, RaftCmdResponse}; use kvproto::raft_serverpb::RaftMessage; use kvproto::replication_modepb::ReplicationStatus; use kvproto::{import_sstpb::SstMeta, kvrpcpb::DiskFullOpt}; -use raft::SnapshotStatus; +use raft::{eraftpb::Entry, GetEntriesContext, SnapshotStatus}; use smallvec::{smallvec, SmallVec}; use crate::store::fsm::apply::TaskRes as ApplyTaskRes; @@ -303,6 +303,29 @@ where }, LeaderCallback(Callback), RaftLogGcFlushed, + // Reports the result of asynchronous Raft logs fetching. + RaftlogFetched { + context: GetEntriesContext, + res: RaftlogFetchResult, + }, +} + +#[derive(Debug)] +pub enum RaftlogFetchResult { + Fetching, + Fetched { + ents: raft::Result>, + // because entries may be empty, so store the original low index that the task issued + low: u64, + // the original max size that the task issued + max_size: u64, + // if the ents hit max_size + hit_size_limit: bool, + // the times that async fetch have already tried + tried_cnt: usize, + // the term when the task issued + term: u64, + }, } /// Message that will be sent to a peer. diff --git a/components/raftstore/src/store/peer.rs b/components/raftstore/src/store/peer.rs index 73f020ee92e..e46f6641802 100644 --- a/components/raftstore/src/store/peer.rs +++ b/components/raftstore/src/store/peer.rs @@ -34,8 +34,8 @@ use kvproto::replication_modepb::{ use protobuf::Message; use raft::eraftpb::{self, ConfChangeType, Entry, EntryType, MessageType}; use raft::{ - self, Changer, LightReady, ProgressState, ProgressTracker, RawNode, Ready, SnapshotStatus, - StateRole, INVALID_INDEX, NO_LIMIT, + self, Changer, GetEntriesContext, LightReady, ProgressState, ProgressTracker, RawNode, Ready, + SnapshotStatus, StateRole, INVALID_INDEX, NO_LIMIT, }; use raft_proto::ConfChangeI; use smallvec::SmallVec; @@ -54,7 +54,8 @@ use crate::store::memory::{needs_evict_entry_cache, MEMTRACE_RAFT_ENTRIES}; use crate::store::msg::RaftCommand; use crate::store::util::{admin_cmd_epoch_lookup, RegionReadProgress}; use crate::store::worker::{ - HeartbeatTask, RaftlogGcTask, ReadDelegate, ReadExecutor, ReadProgress, RegionTask, + HeartbeatTask, RaftlogFetchTask, RaftlogGcTask, ReadDelegate, ReadExecutor, ReadProgress, + RegionTask, }; use crate::store::{ Callback, Config, GlobalReplicationState, PdTask, ReadIndexContext, ReadResponse, TxnExt, @@ -603,7 +604,8 @@ where pub fn new( store_id: u64, cfg: &Config, - sched: Scheduler>, + region_scheduler: Scheduler>, + raftlog_fetch_scheduler: Scheduler, engines: Engines, region: &metapb::Region, peer: metapb::Peer, @@ -614,7 +616,14 @@ where let tag = format!("[region {}] {}", region.get_id(), peer.get_id()); - let ps = PeerStorage::new(engines, region, sched, peer.get_id(), tag.clone())?; + let ps = PeerStorage::new( + engines, + region, + region_scheduler, + raftlog_fetch_scheduler, + peer.get_id(), + tag.clone(), + )?; let applied_index = ps.applied_index(); @@ -3516,12 +3525,11 @@ where )); } let mut entry_size = 0; - for entry in self - .raft_group - .raft - .raft_log - .entries(min_committed + 1, NO_LIMIT)? - { + for entry in self.raft_group.raft.raft_log.entries( + min_committed + 1, + NO_LIMIT, + GetEntriesContext::empty(false), + )? { // commit merge only contains entries start from min_matched + 1 if entry.index > min_matched { entry_size += entry.get_data().len(); diff --git a/components/raftstore/src/store/peer_storage.rs b/components/raftstore/src/store/peer_storage.rs index 03829522700..76f3df1cb46 100644 --- a/components/raftstore/src/store/peer_storage.rs +++ b/components/raftstore/src/store/peer_storage.rs @@ -19,13 +19,14 @@ use kvproto::raft_serverpb::{ }; use protobuf::Message; use raft::eraftpb::{self, ConfState, Entry, HardState, Snapshot}; -use raft::{self, Error as RaftError, RaftState, Ready, Storage, StorageError}; +use raft::{self, Error as RaftError, GetEntriesContext, RaftState, Ready, Storage, StorageError}; use crate::store::async_io::write::WriteTask; use crate::store::fsm::GenSnapTask; use crate::store::memory::*; use crate::store::peer::PersistSnapshotResult; use crate::store::util; +use crate::store::worker::RaftlogFetchTask; use crate::{bytes_capacity, Error, Result}; use engine_traits::{RaftEngine, RaftLogBatch}; use into_other::into_other; @@ -44,6 +45,8 @@ pub const RAFT_INIT_LOG_TERM: u64 = 5; pub const RAFT_INIT_LOG_INDEX: u64 = 5; const MAX_SNAP_TRY_CNT: usize = 5; +pub const MAX_INIT_ENTRY_COUNT: usize = 1024; + /// The initial region epoch version. pub const INIT_EPOCH_VER: u64 = 1; /// The initial region epoch conf_version. @@ -630,11 +633,14 @@ where snap_state: RefCell, gen_snap_task: RefCell>, - region_sched: Scheduler>, + region_scheduler: Scheduler>, snap_tried_cnt: RefCell, cache: EntryCache, + #[allow(dead_code)] + raftlog_fetch_scheduler: Scheduler, + pub tag: String, } @@ -652,8 +658,10 @@ where low: u64, high: u64, max_size: impl Into>, + context: GetEntriesContext, ) -> raft::Result> { - self.entries(low, high, max_size.into().unwrap_or(u64::MAX)) + let max_size = max_size.into(); + self.entries(low, high, max_size.unwrap_or(u64::MAX), context) } fn term(&self, idx: u64) -> raft::Result { @@ -668,8 +676,8 @@ where Ok(self.last_index()) } - fn snapshot(&self, request_index: u64) -> raft::Result { - self.snapshot(request_index) + fn snapshot(&self, request_index: u64, to: u64) -> raft::Result { + self.snapshot(request_index, to) } } @@ -681,7 +689,8 @@ where pub fn new( engines: Engines, region: &metapb::Region, - region_sched: Scheduler>, + region_scheduler: Scheduler>, + raftlog_fetch_scheduler: Scheduler, peer_id: u64, tag: String, ) -> Result> { @@ -707,7 +716,8 @@ where apply_state, snap_state: RefCell::new(SnapState::Relax), gen_snap_task: RefCell::new(None), - region_sched, + region_scheduler, + raftlog_fetch_scheduler, snap_tried_cnt: RefCell::new(0), tag, applied_index_term, @@ -757,9 +767,16 @@ where Ok(()) } - pub fn entries(&self, low: u64, high: u64, max_size: u64) -> raft::Result> { + pub fn entries( + &self, + low: u64, + high: u64, + max_size: u64, + _context: GetEntriesContext, + ) -> raft::Result> { self.check_range(low, high)?; - let mut ents = Vec::with_capacity((high - low) as usize); + let mut ents = + Vec::with_capacity(std::cmp::min((high - low) as usize, MAX_INIT_ENTRY_COUNT)); if low == high { return Ok(ents); } @@ -978,7 +995,7 @@ where /// Gets a snapshot. Returns `SnapshotTemporarilyUnavailable` if there is no unavailable /// snapshot. - pub fn snapshot(&self, request_index: u64) -> raft::Result { + pub fn snapshot(&self, request_index: u64, to: u64) -> raft::Result { let mut snap_state = self.snap_state.borrow_mut(); let mut tried_cnt = self.snap_tried_cnt.borrow_mut(); @@ -1016,6 +1033,7 @@ where "region_id" => self.region.get_id(), "peer_id" => self.peer_id, "times" => *tried_cnt, + "request_peer" => to, ); } } @@ -1039,6 +1057,7 @@ where "region_id" => self.region.get_id(), "peer_id" => self.peer_id, "request_index" => request_index, + "request_peer" => to, ); if !tried || !last_canceled { @@ -1292,7 +1311,7 @@ where let (start_key, end_key) = (enc_start_key(self.region()), enc_end_key(self.region())); let region_id = self.get_region_id(); box_try!( - self.region_sched + self.region_scheduler .schedule(RegionTask::destroy(region_id, start_key, end_key)) ); Ok(()) @@ -1307,14 +1326,14 @@ where let (old_start_key, old_end_key) = (enc_start_key(old_region), enc_end_key(old_region)); let (new_start_key, new_end_key) = (enc_start_key(new_region), enc_end_key(new_region)); if old_start_key < new_start_key { - box_try!(self.region_sched.schedule(RegionTask::destroy( + box_try!(self.region_scheduler.schedule(RegionTask::destroy( old_region.get_id(), old_start_key, new_start_key ))); } if new_end_key < old_end_key { - box_try!(self.region_sched.schedule(RegionTask::destroy( + box_try!(self.region_scheduler.schedule(RegionTask::destroy( old_region.get_id(), new_end_key, old_end_key @@ -1325,7 +1344,7 @@ where /// Delete all extra split data from the `start_key` to `end_key`. pub fn clear_extra_split_data(&self, start_key: Vec, end_key: Vec) -> Result<()> { - box_try!(self.region_sched.schedule(RegionTask::destroy( + box_try!(self.region_scheduler.schedule(RegionTask::destroy( self.get_region_id(), start_key, end_key @@ -1460,7 +1479,7 @@ where fail_point!("skip_schedule_applying_snapshot", |_| {}); // TODO: gracefully remove region instead. - if let Err(e) = self.region_sched.schedule(task) { + if let Err(e) = self.region_scheduler.schedule(task) { info!( "failed to to schedule apply job, are we shutting down?"; "region_id" => self.region.get_id(), @@ -1786,8 +1805,7 @@ mod tests { use crate::coprocessor::CoprocessorHost; use crate::store::async_io::write::write_to_db_for_test; use crate::store::fsm::apply::compact_raft_log; - use crate::store::worker::RegionRunner; - use crate::store::worker::RegionTask; + use crate::store::worker::{RegionRunner, RegionTask}; use crate::store::{bootstrap_store, initial_region, prepare_bootstrap_cluster}; use engine_test::kv::{KvTestEngine, KvTestSnapshot}; use engine_test::raft::RaftTestEngine; @@ -1797,7 +1815,7 @@ mod tests { use kvproto::raft_serverpb::RaftSnapshotData; use raft::eraftpb::HardState; use raft::eraftpb::{ConfState, Entry}; - use raft::{Error as RaftError, StorageError}; + use raft::{Error as RaftError, GetEntriesContext, StorageError}; use std::cell::RefCell; use std::path::Path; use std::sync::atomic::*; @@ -1805,7 +1823,7 @@ mod tests { use std::sync::*; use std::time::Duration; use tempfile::{Builder, TempDir}; - use tikv_util::worker::{LazyWorker, Scheduler, Worker}; + use tikv_util::worker::{dummy_scheduler, LazyWorker, Scheduler, Worker}; use super::*; @@ -1825,7 +1843,8 @@ mod tests { } fn new_storage( - sched: Scheduler>, + region_scheduler: Scheduler>, + raftlog_fetch_scheduler: Scheduler, path: &TempDir, ) -> PeerStorage { let kv_db = engine_test::kv::new_engine(path.path().to_str().unwrap(), None, ALL_CFS, None) @@ -1839,17 +1858,27 @@ mod tests { let region = initial_region(1, 1, 1); prepare_bootstrap_cluster(&engines, ®ion).unwrap(); - PeerStorage::new(engines, ®ion, sched, 1, "".to_owned()).unwrap() + PeerStorage::new( + engines, + ®ion, + region_scheduler, + raftlog_fetch_scheduler, + 1, + "".to_owned(), + ) + .unwrap() } fn new_storage_from_ents( - sched: Scheduler>, + region_scheduler: Scheduler>, + raftlog_fetch_scheduler: Scheduler, path: &TempDir, ents: &[Entry], ) -> PeerStorage { - let mut store = new_storage(sched, path); + let mut store = new_storage(region_scheduler, raftlog_fetch_scheduler, path); let mut write_task = WriteTask::new(store.get_region_id(), store.peer_id, 1); store.append(ents[1..].to_vec(), &mut write_task); + store.update_cache_persisted(ents.last().unwrap().get_index()); store .apply_state .mut_truncated_state() @@ -1918,7 +1947,8 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let worker = Worker::new("snap-manager").lazy_build("snap-manager"); let sched = worker.scheduler(); - let store = new_storage_from_ents(sched, &td, &ents); + let (dummy_scheduler, _) = dummy_scheduler(); + let store = new_storage_from_ents(sched, dummy_scheduler, &td, &ents); let t = store.term(idx); if wterm != t { panic!("#{}: expect res {:?}, got {:?}", i, wterm, t); @@ -1974,7 +2004,13 @@ mod tests { for (first_index, left) in cases { let td = Builder::new().prefix("tikv-store").tempdir().unwrap(); let sched = worker.scheduler(); - let mut store = new_storage_from_ents(sched, &td, &[new_entry(3, 3), new_entry(4, 4)]); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut store = new_storage_from_ents( + sched, + dummy_scheduler, + &td, + &[new_entry(3, 3), new_entry(4, 4)], + ); append_ents(&mut store, &[new_entry(5, 5), new_entry(6, 6)]); assert_eq!(6, get_meta_key_count(&store)); @@ -2053,10 +2089,13 @@ mod tests { for (i, (lo, hi, maxsize, wentries)) in tests.drain(..).enumerate() { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); - let worker = Worker::new("snap-manager").lazy_build("snap-manager"); - let sched = worker.scheduler(); - let store = new_storage_from_ents(sched, &td, &ents); - let e = store.entries(lo, hi, maxsize); + let region_worker = Worker::new("snap-manager").lazy_build("snap-manager"); + let region_scheduler = region_worker.scheduler(); + let raftlog_fetch_worker = Worker::new("raftlog-fetch").lazy_build("raftlog-fetch"); + let raftlog_fetch_scheduler = raftlog_fetch_worker.scheduler(); + let store = + new_storage_from_ents(region_scheduler, raftlog_fetch_scheduler, &td, &ents); + let e = store.entries(lo, hi, maxsize, GetEntriesContext::empty(false)); if e != wentries { panic!("#{}: expect entries {:?}, got {:?}", i, wentries, e); } @@ -2079,7 +2118,8 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let worker = Worker::new("snap-manager").lazy_build("snap-manager"); let sched = worker.scheduler(); - let mut store = new_storage_from_ents(sched, &td, &ents); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut store = new_storage_from_ents(sched, dummy_scheduler, &td, &ents); let res = store .term(idx) .map_err(From::from) @@ -2129,9 +2169,10 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let snap_dir = Builder::new().prefix("snap_dir").tempdir().unwrap(); let mgr = SnapManager::new(snap_dir.path().to_str().unwrap()); - let mut worker = Worker::new("region-worker").lazy_build("la"); + let mut worker = Worker::new("region-worker").lazy_build("region-worker"); let sched = worker.scheduler(); - let mut s = new_storage_from_ents(sched.clone(), &td, &ents); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut s = new_storage_from_ents(sched.clone(), dummy_scheduler, &td, &ents); let (router, _) = mpsc::sync_channel(100); let runner = RegionRunner::new( s.engines.kv.clone(), @@ -2143,7 +2184,7 @@ mod tests { router, ); worker.start_with_timer(runner); - let snap = s.snapshot(0); + let snap = s.snapshot(0, 0); let unavailable = RaftError::Store(StorageError::SnapshotTemporarilyUnavailable); assert_eq!(snap.unwrap_err(), unavailable); assert_eq!(*s.snap_tried_cnt.borrow(), 1); @@ -2167,11 +2208,11 @@ mod tests { let (tx, rx) = channel(); s.set_snap_state(gen_snap_for_test(rx)); // Empty channel should cause snapshot call to wait. - assert_eq!(s.snapshot(0).unwrap_err(), unavailable); + assert_eq!(s.snapshot(0, 0).unwrap_err(), unavailable); assert_eq!(*s.snap_tried_cnt.borrow(), 1); tx.send(snap.clone()).unwrap(); - assert_eq!(s.snapshot(0), Ok(snap.clone())); + assert_eq!(s.snapshot(0, 0), Ok(snap.clone())); assert_eq!(*s.snap_tried_cnt.borrow(), 0); let (tx, rx) = channel(); @@ -2179,7 +2220,8 @@ mod tests { s.set_snap_state(gen_snap_for_test(rx)); // stale snapshot should be abandoned, snapshot index < request index. assert_eq!( - s.snapshot(snap.get_metadata().get_index() + 1).unwrap_err(), + s.snapshot(snap.get_metadata().get_index() + 1, 0) + .unwrap_err(), unavailable ); assert_eq!(*s.snap_tried_cnt.borrow(), 1); @@ -2212,7 +2254,7 @@ mod tests { s.set_snap_state(gen_snap_for_test(rx)); *s.snap_tried_cnt.borrow_mut() = 1; // stale snapshot should be abandoned, snapshot index < truncated index. - assert_eq!(s.snapshot(0).unwrap_err(), unavailable); + assert_eq!(s.snapshot(0, 0).unwrap_err(), unavailable); assert_eq!(*s.snap_tried_cnt.borrow(), 1); let gen_task = s.gen_snap_task.borrow_mut().take().unwrap(); @@ -2229,7 +2271,7 @@ mod tests { ref s => panic!("unexpected state {:?}", s), } // Disconnected channel should trigger another try. - assert_eq!(s.snapshot(0).unwrap_err(), unavailable); + assert_eq!(s.snapshot(0, 0).unwrap_err(), unavailable); let gen_task = s.gen_snap_task.borrow_mut().take().unwrap(); generate_and_schedule_snapshot(gen_task, &s.engines, &sched).unwrap_err(); assert_eq!(*s.snap_tried_cnt.borrow(), 2); @@ -2244,13 +2286,13 @@ mod tests { } // Scheduled job failed should trigger . - assert_eq!(s.snapshot(0).unwrap_err(), unavailable); + assert_eq!(s.snapshot(0, 0).unwrap_err(), unavailable); let gen_task = s.gen_snap_task.borrow_mut().take().unwrap(); generate_and_schedule_snapshot(gen_task, &s.engines, &sched).unwrap_err(); } // When retry too many times, it should report a different error. - match s.snapshot(0) { + match s.snapshot(0, 0) { Err(RaftError::Store(StorageError::Other(_))) => {} res => panic!("unexpected res: {:?}", res), } @@ -2294,10 +2336,13 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let worker = LazyWorker::new("snap-manager"); let sched = worker.scheduler(); - let mut store = new_storage_from_ents(sched, &td, &ents); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut store = new_storage_from_ents(sched, dummy_scheduler, &td, &ents); append_ents(&mut store, &entries); let li = store.last_index(); - let actual_entries = store.entries(4, li + 1, u64::max_value()).unwrap(); + let actual_entries = store + .entries(4, li + 1, u64::max_value(), GetEntriesContext::empty(false)) + .unwrap(); if actual_entries != wentries { panic!("#{}: want {:?}, got {:?}", i, wentries, actual_entries); } @@ -2310,10 +2355,13 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let worker = LazyWorker::new("snap-manager"); let sched = worker.scheduler(); - let mut store = new_storage_from_ents(sched, &td, &ents); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut store = new_storage_from_ents(sched, dummy_scheduler, &td, &ents); store.cache.cache.clear(); // empty cache should fetch data from rocksdb directly. - let mut res = store.entries(4, 6, u64::max_value()).unwrap(); + let mut res = store + .entries(4, 6, u64::max_value(), GetEntriesContext::empty(false)) + .unwrap(); assert_eq!(*res, ents[1..]); let entries = vec![new_entry(6, 5), new_entry(7, 5)]; @@ -2321,27 +2369,37 @@ mod tests { validate_cache(&store, &entries); // direct cache access - res = store.entries(6, 8, u64::max_value()).unwrap(); + res = store + .entries(6, 8, u64::max_value(), GetEntriesContext::empty(false)) + .unwrap(); assert_eq!(res, entries); // size limit should be supported correctly. - res = store.entries(4, 8, 0).unwrap(); + res = store + .entries(4, 8, 0, GetEntriesContext::empty(false)) + .unwrap(); assert_eq!(res, vec![new_entry(4, 4)]); let mut size = ents[1..].iter().map(|e| u64::from(e.compute_size())).sum(); - res = store.entries(4, 8, size).unwrap(); + res = store + .entries(4, 8, size, GetEntriesContext::empty(false)) + .unwrap(); let mut exp_res = ents[1..].to_vec(); assert_eq!(res, exp_res); for e in &entries { size += u64::from(e.compute_size()); exp_res.push(e.clone()); - res = store.entries(4, 8, size).unwrap(); + res = store + .entries(4, 8, size, GetEntriesContext::empty(false)) + .unwrap(); assert_eq!(res, exp_res); } // range limit should be supported correctly. for low in 4..9 { for high in low..9 { - let res = store.entries(low, high, u64::max_value()).unwrap(); + let res = store + .entries(low, high, u64::max_value(), GetEntriesContext::empty(false)) + .unwrap(); assert_eq!(*res, exp_res[low as usize - 4..high as usize - 4]); } } @@ -2353,7 +2411,8 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let worker = LazyWorker::new("snap-manager"); let sched = worker.scheduler(); - let mut store = new_storage_from_ents(sched, &td, &ents); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut store = new_storage_from_ents(sched, dummy_scheduler, &td, &ents); store.cache.cache.clear(); // initial cache @@ -2509,7 +2568,8 @@ mod tests { let mgr = SnapManager::new(snap_dir.path().to_str().unwrap()); let mut worker = LazyWorker::new("snap-manager"); let sched = worker.scheduler(); - let s1 = new_storage_from_ents(sched.clone(), &td1, &ents); + let (dummy_scheduler, _) = dummy_scheduler(); + let s1 = new_storage_from_ents(sched.clone(), dummy_scheduler.clone(), &td1, &ents); let (router, _) = mpsc::sync_channel(100); let runner = RegionRunner::new( s1.engines.kv.clone(), @@ -2521,7 +2581,7 @@ mod tests { router, ); worker.start(runner); - assert!(s1.snapshot(0).is_err()); + assert!(s1.snapshot(0, 0).is_err()); let gen_task = s1.gen_snap_task.borrow_mut().take().unwrap(); generate_and_schedule_snapshot(gen_task, &s1.engines, &sched).unwrap(); @@ -2536,7 +2596,7 @@ mod tests { worker.stop(); let td2 = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); - let mut s2 = new_storage(sched.clone(), &td2); + let mut s2 = new_storage(sched.clone(), dummy_scheduler.clone(), &td2); assert_eq!(s2.first_index(), s2.applied_index() + 1); let mut write_task = WriteTask::new(s2.get_region_id(), s2.peer_id, 1); let snap_region = s2.apply_snapshot(&snap1, &mut write_task, &[]).unwrap(); @@ -2553,7 +2613,7 @@ mod tests { let td3 = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let ents = &[new_entry(3, 3), new_entry(4, 3)]; - let mut s3 = new_storage_from_ents(sched, &td3, ents); + let mut s3 = new_storage_from_ents(sched, dummy_scheduler, &td3, ents); validate_cache(&s3, &ents[1..]); let mut write_task = WriteTask::new(s3.get_region_id(), s3.peer_id, 1); let snap_region = s3.apply_snapshot(&snap1, &mut write_task, &[]).unwrap(); @@ -2573,7 +2633,8 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let worker = LazyWorker::new("snap-manager"); let sched = worker.scheduler(); - let mut s = new_storage(sched, &td); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut s = new_storage(sched, dummy_scheduler, &td); // PENDING can be canceled directly. s.snap_state = RefCell::new(SnapState::Applying(Arc::new(AtomicUsize::new( @@ -2619,7 +2680,8 @@ mod tests { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); let worker = LazyWorker::new("snap-manager"); let sched = worker.scheduler(); - let mut s = new_storage(sched, &td); + let (dummy_scheduler, _) = dummy_scheduler(); + let mut s = new_storage(sched, dummy_scheduler, &td); // PENDING can be finished. let mut snap_state = SnapState::Applying(Arc::new(AtomicUsize::new(JOB_STATUS_PENDING))); @@ -2666,8 +2728,10 @@ mod tests { #[test] fn test_validate_states() { let td = Builder::new().prefix("tikv-store-test").tempdir().unwrap(); - let worker = LazyWorker::new("snap-manager"); - let sched = worker.scheduler(); + let region_worker = LazyWorker::new("snap-manager"); + let region_sched = region_worker.scheduler(); + let raftlog_fetch_worker = LazyWorker::new("raftlog-fetch-worker"); + let raftlog_fetch_sched = raftlog_fetch_worker.scheduler(); let kv_db = engine_test::kv::new_engine(td.path().to_str().unwrap(), None, ALL_CFS, None).unwrap(); let raft_path = td.path().join(Path::new("raft")); @@ -2680,7 +2744,14 @@ mod tests { let region = initial_region(1, 1, 1); prepare_bootstrap_cluster(&engines, ®ion).unwrap(); let build_storage = || -> Result> { - PeerStorage::new(engines.clone(), ®ion, sched.clone(), 0, "".to_owned()) + PeerStorage::new( + engines.clone(), + ®ion, + region_sched.clone(), + raftlog_fetch_sched.clone(), + 0, + "".to_owned(), + ) }; let mut s = build_storage().unwrap(); let mut raft_state = RaftLocalState::default(); diff --git a/components/raftstore/src/store/region_snapshot.rs b/components/raftstore/src/store/region_snapshot.rs index 83a955ddd5b..2c95aa134ac 100644 --- a/components/raftstore/src/store/region_snapshot.rs +++ b/components/raftstore/src/store/region_snapshot.rs @@ -410,8 +410,17 @@ mod tests { EK: KvEngine, ER: RaftEngine, { - let (sched, _) = worker::dummy_scheduler(); - PeerStorage::new(engines, r, sched, 0, "".to_owned()).unwrap() + let (region_sched, _) = worker::dummy_scheduler(); + let (raftlog_fetch_sched, _) = worker::dummy_scheduler(); + PeerStorage::new( + engines, + r, + region_sched, + raftlog_fetch_sched, + 0, + "".to_owned(), + ) + .unwrap() } fn load_default_dataset(engines: Engines) -> (PeerStorage, DataSet) diff --git a/components/raftstore/src/store/worker/mod.rs b/components/raftstore/src/store/worker/mod.rs index d755f790ae7..36ec89cf9a8 100644 --- a/components/raftstore/src/store/worker/mod.rs +++ b/components/raftstore/src/store/worker/mod.rs @@ -8,6 +8,7 @@ mod consistency_check; mod metrics; mod pd; mod query_stats; +mod raftlog_fetch; mod raftlog_gc; mod read; mod refresh_config; @@ -25,6 +26,7 @@ pub use self::pd::{ FlowStatistics, FlowStatsReporter, HeartbeatTask, Runner as PdRunner, Task as PdTask, }; pub use self::query_stats::QueryStats; +pub use self::raftlog_fetch::{Runner as RaftlogFetchRunner, Task as RaftlogFetchTask}; pub use self::raftlog_gc::{Runner as RaftlogGcRunner, Task as RaftlogGcTask}; pub use self::read::{LocalReader, Progress as ReadProgress, ReadDelegate, ReadExecutor, TrackVer}; pub use self::refresh_config::{ diff --git a/components/raftstore/src/store/worker/raftlog_fetch.rs b/components/raftstore/src/store/worker/raftlog_fetch.rs new file mode 100644 index 00000000000..d92a461f76c --- /dev/null +++ b/components/raftstore/src/store/worker/raftlog_fetch.rs @@ -0,0 +1,116 @@ +// Copyright 2021 TiKV Project Authors. Licensed under Apache-2.0. + +use engine_traits::{KvEngine, RaftEngine}; +use raft::GetEntriesContext; +use std::fmt; +use tikv_util::worker::Runnable; + +use crate::store::{RaftlogFetchResult, SignificantMsg, SignificantRouter, MAX_INIT_ENTRY_COUNT}; + +pub enum Task { + PeerStorage { + region_id: u64, + context: GetEntriesContext, + low: u64, + high: u64, + max_size: usize, + tried_cnt: usize, + term: u64, + }, + // More to support, suck as fetch entries ayschronously when apply and schedule merge +} + +impl fmt::Display for Task { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Task::PeerStorage { + region_id, + context, + low, + high, + max_size, + tried_cnt, + term, + } => write!( + f, + "Fetch Raft Logs [region: {}, low: {}, high: {}, max_size: {}] for sending with context {:?}, tried: {}, term: {}", + region_id, low, high, max_size, context, tried_cnt, term, + ), + } + } +} + +pub struct Runner +where + EK: KvEngine, + ER: RaftEngine, + R: SignificantRouter, +{ + router: R, + raft_engine: ER, + _phantom: std::marker::PhantomData, +} + +impl> Runner { + pub fn new(router: R, raft_engine: ER) -> Runner { + Runner { + router, + raft_engine, + _phantom: std::marker::PhantomData, + } + } +} + +impl Runnable for Runner +where + EK: KvEngine, + ER: RaftEngine, + R: SignificantRouter, +{ + type Task = Task; + + fn run(&mut self, task: Task) { + match task { + Task::PeerStorage { + region_id, + low, + high, + max_size, + context, + tried_cnt, + term, + } => { + let mut ents = + Vec::with_capacity(std::cmp::min((high - low) as usize, MAX_INIT_ENTRY_COUNT)); + let res = self.raft_engine.fetch_entries_to( + region_id, + low, + high, + Some(max_size), + &mut ents, + ); + + let hit_size_limit = res + .as_ref() + .map(|c| (*c as u64) != high - low) + .unwrap_or(false); + + // it may return a region not found error as the region could be merged. + let _ = self.router.significant_send( + region_id, + SignificantMsg::RaftlogFetched { + context, + res: RaftlogFetchResult::Fetched { + ents: res.map(|_| ents).map_err(|e| e.into()), + low, + max_size: max_size as u64, + hit_size_limit, + tried_cnt, + term, + }, + }, + ); + } + } + } +} diff --git a/components/raftstore/src/store/worker/region.rs b/components/raftstore/src/store/worker/region.rs index 67ee76e70a5..81c419d29d1 100644 --- a/components/raftstore/src/store/worker/region.rs +++ b/components/raftstore/src/store/worker/region.rs @@ -968,7 +968,7 @@ mod tests { let snap_dir = Builder::new().prefix("snap_dir").tempdir().unwrap(); let mgr = SnapManager::new(snap_dir.path().to_str().unwrap()); let bg_worker = Worker::new("snap-manager"); - let mut worker = bg_worker.lazy_build("snapshot-worker"); + let mut worker = bg_worker.lazy_build("snap-manager"); let sched = worker.scheduler(); let (router, receiver) = mpsc::sync_channel(1); let runner = RegionRunner::new( diff --git a/components/resolved_ts/Cargo.toml b/components/resolved_ts/Cargo.toml index 4a4099e6bdf..036d07a64df 100644 --- a/components/resolved_ts/Cargo.toml +++ b/components/resolved_ts/Cargo.toml @@ -32,7 +32,7 @@ log_wrappers = { path = "../log_wrappers" } pd_client = { path = "../pd_client", default-features = false } prometheus = { version = "0.13", default-features = false, features = ["nightly"] } protobuf = { version = "2.8", features = ["bytes"] } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raftstore = { path = "../raftstore", default-features = false } thiserror = "1.0" tikv = { path = "../../", default-features = false } @@ -49,7 +49,7 @@ tempfile = "3.0" test_raftstore = { path = "../test_raftstore", default-features = false } test_util = { path = "../test_util", default-features = false } panic_hook = { path = "../panic_hook" } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } tikv_kv = { path = "../tikv_kv" } [[test]] diff --git a/components/server/Cargo.toml b/components/server/Cargo.toml index 319e800d901..94f1cd45bf2 100644 --- a/components/server/Cargo.toml +++ b/components/server/Cargo.toml @@ -54,7 +54,7 @@ nix = "0.11" pd_client = { path = "../pd_client", default-features = false } prometheus = { version = "0.13", features = ["nightly"] } protobuf = { version = "2.8", features = ["bytes"] } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raft_log_engine = { path = "../raft_log_engine", default-features = false } raftstore = { path = "../raftstore", default-features = false } rand = "0.8" diff --git a/components/test_raftstore/Cargo.toml b/components/test_raftstore/Cargo.toml index 0c20544c09d..83bb6e681f4 100644 --- a/components/test_raftstore/Cargo.toml +++ b/components/test_raftstore/Cargo.toml @@ -30,7 +30,7 @@ keys = { path = "../keys", default-features = false } kvproto = { git = "https://github.com/pingcap/kvproto.git" } pd_client = { path = "../pd_client", default-features = false } protobuf = { version = "2.8", features = ["bytes"] } -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raftstore = { path = "../raftstore", default-features = false } rand = "0.8" resolved_ts = { path = "../resolved_ts" } diff --git a/metrics/grafana/tikv_details.json b/metrics/grafana/tikv_details.json index c6304c66dd7..a916fe3280a 100644 --- a/metrics/grafana/tikv_details.json +++ b/metrics/grafana/tikv_details.json @@ -28996,7 +28996,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(delta(tikv_engine_ingestion_picked_level_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", + "expr": "sum(delta(tikv_engine_ingestion_picked_level_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", db=\"$db\"}[1m])) by (le)", "format": "heatmap", "interval": "", "intervalFactor": 2, diff --git a/src/server/debug.rs b/src/server/debug.rs index 84567f1c786..3ccec6d558e 100644 --- a/src/server/debug.rs +++ b/src/server/debug.rs @@ -486,8 +486,9 @@ impl Debugger { let mut iter = box_try!(self.engines.kv.iterator_cf_opt(CF_RAFT, readopts)); iter.seek(SeekKey::from(from.as_ref())).unwrap(); - let fake_worker = Worker::new("fake-snap-worker"); - let fake_snap_worker = fake_worker.lazy_build("fake-snap"); + let fake_snap_worker = Worker::new("fake-snap-worker").lazy_build("fake-snap"); + let fake_raftlog_fetch_worker = + Worker::new("fake-raftlog-fetch-worker").lazy_build("fake-raftlog-fetch"); let check_value = |value: &[u8]| -> Result<()> { let mut local_state = RegionLocalState::default(); @@ -512,6 +513,7 @@ impl Debugger { self.engines.clone(), region, fake_snap_worker.scheduler(), + fake_raftlog_fetch_worker.scheduler(), peer_id, tag, )); diff --git a/tests/Cargo.toml b/tests/Cargo.toml index 6408504a89f..945fc104e98 100644 --- a/tests/Cargo.toml +++ b/tests/Cargo.toml @@ -83,7 +83,7 @@ paste = "1.0" pd_client = { path = "../components/pd_client", default-features = false } protobuf = { version = "2.8", features = ["bytes"] } more-asserts = "0.2" -raft = { version = "0.6.0-alpha", default-features = false, features = ["protobuf-codec"] } +raft = { version = "0.7.0", default-features = false, features = ["protobuf-codec"] } raft_log_engine = { path = "../components/raft_log_engine", default-features = false } raftstore = { path = "../components/raftstore", default-features = false } rand = "0.8.3" From 19f521896be77b6bf9ecc4eec3cecd3cd8004263 Mon Sep 17 00:00:00 2001 From: Suhaha Date: Wed, 26 Jan 2022 10:44:45 +0800 Subject: [PATCH 16/16] status_server: add pprof flamegraph response header (#10951) * tweak(status_server): add pprof flamegraph response header Signed-off-by: Suhaha * test(status_server): add Content-Type asset to test_pprof_profile_service Signed-off-by: Suhaha * tweak(status_server): add pprof flamegraph response header Signed-off-by: Suhaha * tweak(status_server): add pprof flamegraph response header close #11917 Signed-off-by: Suhaha * close #11917 Signed-off-by: Suhaha Co-authored-by: goroutine Co-authored-by: Connor --- src/server/status_server/mod.rs | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/server/status_server/mod.rs b/src/server/status_server/mod.rs index bf3c276b056..bf198bc40fb 100644 --- a/src/server/status_server/mod.rs +++ b/src/server/status_server/mod.rs @@ -349,7 +349,18 @@ where match start_one_cpu_profile(end, frequency, output_protobuf).await { Ok(body) => { info!("dump cpu profile successfully"); - Ok(make_response(StatusCode::OK, body)) + let mut response = Response::builder() + .header( + "Content-Disposition", + "attachment; filename=\"cpu_profile\"", + ) + .header("Content-Length", body.len()); + response = if output_protobuf { + response.header("Content-Type", mime::APPLICATION_OCTET_STREAM.to_string()) + } else { + response.header("Content-Type", mime::IMAGE_SVG.to_string()) + }; + Ok(response.body(body.into()).unwrap()) } Err(e) => { info!("dump cpu profile fail: {}", e); @@ -1278,6 +1289,10 @@ mod tests { .spawn(async move { client.get(uri).await.unwrap() }); let resp = block_on(handle).unwrap(); assert_eq!(resp.status(), StatusCode::OK); + assert_eq!( + resp.headers().get("Content-Type").unwrap(), + &mime::IMAGE_SVG.to_string() + ); status_server.stop(); }