From 943d76c583f5740b1d922275a673233a27fe1693 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 2 Sep 2022 15:34:21 +0200 Subject: [PATCH 1/6] Ability to dynamically set resync tranquility --- src/block/manager.rs | 62 ++++++++++++++++++++++++++++++++------- src/block/repair.rs | 26 +++++++++++++--- src/garage/admin.rs | 19 ++++++++++++ src/garage/cli/structs.rs | 16 ++++++++++ src/model/garage.rs | 1 - src/util/config.rs | 7 ----- 6 files changed, 108 insertions(+), 23 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 017ba9da..ef48107f 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -3,7 +3,7 @@ use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; -use arc_swap::ArcSwapOption; +use arc_swap::{ArcSwap, ArcSwapOption}; use async_trait::async_trait; use serde::{Deserialize, Serialize}; @@ -25,6 +25,7 @@ use garage_util::background::*; use garage_util::data::*; use garage_util::error::*; use garage_util::metrics::RecordDuration; +use garage_util::persister::Persister; use garage_util::time::*; use garage_util::tranquilizer::Tranquilizer; @@ -55,6 +56,10 @@ const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60); // The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour) const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6; +// Resync tranquility is initially set to 2, but can be changed in the CLI +// and the updated version is persisted over Garage restarts +const INITIAL_RESYNC_TRANQUILITY: u32 = 2; + // The delay between the moment when the reference counter // drops to zero, and the moment where we allow ourselves // to delete the block locally. @@ -90,7 +95,6 @@ pub struct BlockManager { pub data_dir: PathBuf, compression_level: Option, - background_tranquility: u32, mutation_lock: Mutex, @@ -100,6 +104,9 @@ pub struct BlockManager { resync_notify: Notify, resync_errors: CountedTree, + resync_persister: Persister, + resync_persisted: ArcSwap, + pub(crate) system: Arc, endpoint: Arc>, @@ -124,7 +131,6 @@ impl BlockManager { db: &db::Db, data_dir: PathBuf, compression_level: Option, - background_tranquility: u32, replication: TableShardedReplication, system: Arc, ) -> Arc { @@ -145,6 +151,14 @@ impl BlockManager { let resync_errors = CountedTree::new(resync_errors).expect("Could not count block_local_resync_errors"); + let resync_persister = Persister::new(&system.metadata_dir, "resync_cfg"); + let resync_persisted = match resync_persister.load() { + Ok(v) => v, + Err(_) => ResyncPersistedConfig { + tranquility: INITIAL_RESYNC_TRANQUILITY, + }, + }; + let endpoint = system .netapp .endpoint("garage_block/manager.rs/Rpc".to_string()); @@ -157,12 +171,13 @@ impl BlockManager { replication, data_dir, compression_level, - background_tranquility, mutation_lock: Mutex::new(manager_locked), rc, resync_queue, resync_notify: Notify::new(), resync_errors, + resync_persister, + resync_persisted: ArcSwap::new(Arc::new(resync_persisted)), system, endpoint, metrics, @@ -716,6 +731,23 @@ impl BlockManager { Ok(()) } + + async fn update_resync_persisted( + &self, + update: impl Fn(&mut ResyncPersistedConfig), + ) -> Result<(), Error> { + let mut cfg: ResyncPersistedConfig = *self.resync_persisted.load().as_ref(); + update(&mut cfg); + self.resync_persister.save_async(&cfg).await?; + self.resync_persisted.store(Arc::new(cfg)); + self.resync_notify.notify_one(); + Ok(()) + } + + pub async fn set_resync_tranquility(&self, tranquility: u32) -> Result<(), Error> { + self.update_resync_persisted(|cfg| cfg.tranquility = tranquility) + .await + } } #[async_trait] @@ -734,6 +766,11 @@ impl EndpointHandler for BlockManager { } } +#[derive(Serialize, Deserialize, Clone, Copy)] +struct ResyncPersistedConfig { + tranquility: u32, +} + struct ResyncWorker { manager: Arc, tranquilizer: Tranquilizer, @@ -758,19 +795,22 @@ impl Worker for ResyncWorker { fn info(&self) -> Option { let mut ret = vec![]; + ret.push(format!( + "tranquility = {}", + self.manager.resync_persisted.load().tranquility + )); + let qlen = self.manager.resync_queue_len().unwrap_or(0); - let elen = self.manager.resync_errors_len().unwrap_or(0); if qlen > 0 { ret.push(format!("{} blocks in queue", qlen)); } + + let elen = self.manager.resync_errors_len().unwrap_or(0); if elen > 0 { ret.push(format!("{} blocks in error state", elen)); } - if !ret.is_empty() { - Some(ret.join(", ")) - } else { - None - } + + Some(ret.join(", ")) } async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { @@ -778,7 +818,7 @@ impl Worker for ResyncWorker { match self.manager.resync_iter().await { Ok(ResyncIterResult::BusyDidSomething) => Ok(self .tranquilizer - .tranquilize_worker(self.manager.background_tranquility)), + .tranquilize_worker(self.manager.resync_persisted.load().tranquility)), Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerState::Busy), Ok(ResyncIterResult::IdleFor(delay)) => { self.next_delay = delay; diff --git a/src/block/repair.rs b/src/block/repair.rs index 07ff6772..18e1de95 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -19,7 +19,17 @@ use garage_util::tranquilizer::Tranquilizer; use crate::manager::*; -const SCRUB_INTERVAL: Duration = Duration::from_secs(3600 * 24 * 30); // full scrub every 30 days +// Full scrub every 30 days +const SCRUB_INTERVAL: Duration = Duration::from_secs(3600 * 24 * 30); +// Scrub tranquility is initially set to 4, but can be changed in the CLI +// and the updated version is persisted over Garage restarts +const INITIAL_SCRUB_TRANQUILITY: u32 = 4; + +// ---- ---- ---- +// FIRST KIND OF REPAIR: FINDING MISSING BLOCKS/USELESS BLOCKS +// This is a one-shot repair operation that can be launched, +// checks everything, and then exits. +// ---- ---- ---- pub struct RepairWorker { manager: Arc, @@ -128,7 +138,13 @@ impl Worker for RepairWorker { } } -// ---- +// ---- ---- ---- +// SECOND KIND OF REPAIR: SCRUBBING THE DATASTORE +// This is significantly more complex than the process above, +// as it is a continuously-running task that triggers automatically +// every SCRUB_INTERVAL, but can also be triggered manually +// and whose parameter (esp. speed) can be controlled at runtime. +// ---- ---- ---- pub struct ScrubWorker { manager: Arc, @@ -176,7 +192,7 @@ impl ScrubWorker { Ok(v) => v, Err(_) => ScrubWorkerPersisted { time_last_complete_scrub: 0, - tranquility: 4, + tranquility: INITIAL_SCRUB_TRANQUILITY, corruptions_detected: 0, }, }; @@ -343,7 +359,9 @@ impl Worker for ScrubWorker { } } -// ---- +// ---- ---- ---- +// UTILITY FOR ENUMERATING THE BLOCK STORE +// ---- ---- ---- struct BlockStoreIterator { path: Vec, diff --git a/src/garage/admin.rs b/src/garage/admin.rs index 71ee608c..1d80889c 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -15,6 +15,8 @@ use garage_table::*; use garage_rpc::*; +use garage_block::repair::ScrubWorkerCommand; + use garage_model::bucket_alias_table::*; use garage_model::bucket_table::*; use garage_model::garage::Garage; @@ -836,6 +838,23 @@ impl AdminRpcHandler { let workers = self.garage.background.get_worker_info(); Ok(AdminRpc::WorkerList(workers, opt)) } + WorkerCmd::Set { opt } => match opt { + WorkerSetCmd::ScrubTranquility { tranquility } => { + let scrub_command = ScrubWorkerCommand::SetTranquility(tranquility); + self.garage + .block_manager + .send_scrub_command(scrub_command) + .await; + Ok(AdminRpc::Ok("Scrub tranquility updated".into())) + } + WorkerSetCmd::ResyncTranquility { tranquility } => { + self.garage + .block_manager + .set_resync_tranquility(tranquility) + .await?; + Ok(AdminRpc::Ok("Resync tranquility updated".into())) + } + }, } } } diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index 9274f80f..1fba934f 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -501,6 +501,12 @@ pub enum WorkerCmd { #[structopt(flatten)] opt: WorkerListOpt, }, + /// Set worker parameter + #[structopt(name = "set", version = version::garage())] + Set { + #[structopt(subcommand)] + opt: WorkerSetCmd, + }, } #[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone, Copy)] @@ -512,3 +518,13 @@ pub struct WorkerListOpt { #[structopt(short = "e", long = "errors")] pub errors: bool, } + +#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)] +pub enum WorkerSetCmd { + /// Set tranquility of scrub operations + #[structopt(name = "scrub-tranquility", version = version::garage())] + ScrubTranquility { tranquility: u32 }, + /// Set tranquility of resync operations + #[structopt(name = "resync-tranquility", version = version::garage())] + ResyncTranquility { tranquility: u32 }, +} diff --git a/src/model/garage.rs b/src/model/garage.rs index 15769a17..4dd24582 100644 --- a/src/model/garage.rs +++ b/src/model/garage.rs @@ -159,7 +159,6 @@ impl Garage { &db, config.data_dir.clone(), config.compression_level, - config.block_manager_background_tranquility, data_rep_param, system.clone(), ); diff --git a/src/util/config.rs b/src/util/config.rs index e8ef4fdd..a2bb8fb3 100644 --- a/src/util/config.rs +++ b/src/util/config.rs @@ -23,10 +23,6 @@ pub struct Config { #[serde(default = "default_block_size")] pub block_size: usize, - /// Size of data blocks to save to disk - #[serde(default = "default_block_manager_background_tranquility")] - pub block_manager_background_tranquility: u32, - /// Replication mode. Supported values: /// - none, 1 -> no replication /// - 2 -> 2-way replication @@ -147,9 +143,6 @@ fn default_sled_flush_every_ms() -> u64 { fn default_block_size() -> usize { 1048576 } -fn default_block_manager_background_tranquility() -> u32 { - 2 -} /// Read and parse configuration pub fn read_config(config_file: PathBuf) -> Result { From 47be652a1fe08a8e6dab6aa2c4a41d8eb119f392 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 2 Sep 2022 16:47:15 +0200 Subject: [PATCH 2/6] block manager: refactor: split resync into separate file --- src/block/lib.rs | 1 + src/block/manager.rs | 598 ++++--------------------------------------- src/block/repair.rs | 8 +- src/block/resync.rs | 536 ++++++++++++++++++++++++++++++++++++++ src/garage/admin.rs | 7 +- 5 files changed, 595 insertions(+), 555 deletions(-) create mode 100644 src/block/resync.rs diff --git a/src/block/lib.rs b/src/block/lib.rs index ebdb95d8..d2814f77 100644 --- a/src/block/lib.rs +++ b/src/block/lib.rs @@ -3,6 +3,7 @@ extern crate tracing; pub mod manager; pub mod repair; +pub mod resync; mod block; mod metrics; diff --git a/src/block/manager.rs b/src/block/manager.rs index ef48107f..efb5349c 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -1,33 +1,19 @@ -use std::convert::TryInto; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; -use arc_swap::{ArcSwap, ArcSwapOption}; use async_trait::async_trait; use serde::{Deserialize, Serialize}; -use futures::future::*; use tokio::fs; use tokio::io::{AsyncReadExt, AsyncWriteExt}; -use tokio::select; -use tokio::sync::{mpsc, watch, Mutex, Notify}; - -use opentelemetry::{ - trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer}, - Context, KeyValue, -}; +use tokio::sync::{mpsc, Mutex}; use garage_db as db; -use garage_db::counted_tree_hack::CountedTree; -use garage_util::background::*; use garage_util::data::*; use garage_util::error::*; use garage_util::metrics::RecordDuration; -use garage_util::persister::Persister; -use garage_util::time::*; -use garage_util::tranquilizer::Tranquilizer; use garage_rpc::system::System; use garage_rpc::*; @@ -38,27 +24,13 @@ use crate::block::*; use crate::metrics::*; use crate::rc::*; use crate::repair::*; +use crate::resync::*; /// Size under which data will be stored inlined in database instead of as files pub const INLINE_THRESHOLD: usize = 3072; // Timeout for RPCs that read and write blocks to remote nodes -const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(30); -// Timeout for RPCs that ask other nodes whether they need a copy -// of a given block before we delete it locally -const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5); - -// The delay between the time where a resync operation fails -// and the time when it is retried, with exponential backoff -// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure). -const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60); -// The minimum retry delay is 60 seconds = 1 minute -// The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour) -const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6; - -// Resync tranquility is initially set to 2, but can be changed in the CLI -// and the updated version is persisted over Garage restarts -const INITIAL_RESYNC_TRANQUILITY: u32 = 2; +pub(crate) const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(30); // The delay between the moment when the reference counter // drops to zero, and the moment where we allow ourselves @@ -96,35 +68,23 @@ pub struct BlockManager { compression_level: Option, - mutation_lock: Mutex, + pub(crate) mutation_lock: Mutex, pub(crate) rc: BlockRc, - - resync_queue: CountedTree, - resync_notify: Notify, - resync_errors: CountedTree, - - resync_persister: Persister, - resync_persisted: ArcSwap, + pub resync: BlockResyncManager, pub(crate) system: Arc, - endpoint: Arc>, + pub(crate) endpoint: Arc>, - metrics: BlockManagerMetrics, + pub(crate) metrics: BlockManagerMetrics, - tx_scrub_command: ArcSwapOption>, + tx_scrub_command: mpsc::Sender, } // This custom struct contains functions that must only be ran // when the lock is held. We ensure that it is the case by storing // it INSIDE a Mutex. -struct BlockManagerLocked(); - -enum ResyncIterResult { - BusyDidSomething, - BusyDidNothing, - IdleFor(Duration), -} +pub(crate) struct BlockManagerLocked(); impl BlockManager { pub fn new( @@ -139,25 +99,7 @@ impl BlockManager { .expect("Unable to open block_local_rc tree"); let rc = BlockRc::new(rc); - let resync_queue = db - .open_tree("block_local_resync_queue") - .expect("Unable to open block_local_resync_queue tree"); - let resync_queue = - CountedTree::new(resync_queue).expect("Could not count block_local_resync_queue"); - - let resync_errors = db - .open_tree("block_local_resync_errors") - .expect("Unable to open block_local_resync_errors tree"); - let resync_errors = - CountedTree::new(resync_errors).expect("Could not count block_local_resync_errors"); - - let resync_persister = Persister::new(&system.metadata_dir, "resync_cfg"); - let resync_persisted = match resync_persister.load() { - Ok(v) => v, - Err(_) => ResyncPersistedConfig { - tranquility: INITIAL_RESYNC_TRANQUILITY, - }, - }; + let resync = BlockResyncManager::new(db, &system); let endpoint = system .netapp @@ -165,7 +107,9 @@ impl BlockManager { let manager_locked = BlockManagerLocked(); - let metrics = BlockManagerMetrics::new(resync_queue.clone(), resync_errors.clone()); + let metrics = BlockManagerMetrics::new(resync.queue.clone(), resync.errors.clone()); + + let (scrub_tx, scrub_rx) = mpsc::channel(1); let block_manager = Arc::new(Self { replication, @@ -173,25 +117,31 @@ impl BlockManager { compression_level, mutation_lock: Mutex::new(manager_locked), rc, - resync_queue, - resync_notify: Notify::new(), - resync_errors, - resync_persister, - resync_persisted: ArcSwap::new(Arc::new(resync_persisted)), + resync, system, endpoint, metrics, - tx_scrub_command: ArcSwapOption::new(None), + tx_scrub_command: scrub_tx, }); block_manager.endpoint.set_handler(block_manager.clone()); - block_manager.clone().spawn_background_workers(); + // Spawn one resync worker + let background = block_manager.system.background.clone(); + let worker = ResyncWorker::new(block_manager.clone()); + tokio::spawn(async move { + tokio::time::sleep(Duration::from_secs(10)).await; + background.spawn_worker(worker); + }); + + // Spawn scrub worker + let scrub_worker = ScrubWorker::new(block_manager.clone(), scrub_rx); + block_manager.system.background.spawn_worker(scrub_worker); block_manager } /// Ask nodes that might have a (possibly compressed) block for it - async fn rpc_get_raw_block(&self, hash: &Hash) -> Result { + pub(crate) async fn rpc_get_raw_block(&self, hash: &Hash) -> Result { let who = self.replication.read_nodes(hash); let resps = self .system @@ -243,20 +193,6 @@ impl BlockManager { Ok(()) } - /// Get lenght of resync queue - pub fn resync_queue_len(&self) -> Result { - // This currently can't return an error because the CountedTree hack - // doesn't error on .len(), but this will change when we remove the hack - // (hopefully someday!) - Ok(self.resync_queue.len()) - } - - /// Get number of blocks that have an error - pub fn resync_errors_len(&self) -> Result { - // (see resync_queue_len comment) - Ok(self.resync_errors.len()) - } - /// Get number of items in the refcount table pub fn rc_len(&self) -> Result { Ok(self.rc.rc.len()?) @@ -264,13 +200,7 @@ impl BlockManager { /// Send command to start/stop/manager scrub worker pub async fn send_scrub_command(&self, cmd: ScrubWorkerCommand) { - let _ = self - .tx_scrub_command - .load() - .as_ref() - .unwrap() - .send(cmd) - .await; + let _ = self.tx_scrub_command.send(cmd).await; } //// ----- Managing the reference counter ---- @@ -291,7 +221,7 @@ impl BlockManager { // we will fecth it from someone. let this = self.clone(); tokio::spawn(async move { - if let Err(e) = this.put_to_resync(&hash, 2 * BLOCK_RW_TIMEOUT) { + if let Err(e) = this.resync.put_to_resync(&hash, 2 * BLOCK_RW_TIMEOUT) { error!("Block {:?} could not be put in resync queue: {}.", hash, e); } }); @@ -313,7 +243,9 @@ impl BlockManager { // after that delay has passed. let this = self.clone(); tokio::spawn(async move { - if let Err(e) = this.put_to_resync(&hash, BLOCK_GC_DELAY + Duration::from_secs(10)) + if let Err(e) = this + .resync + .put_to_resync(&hash, BLOCK_GC_DELAY + Duration::from_secs(10)) { error!("Block {:?} could not be put in resync queue: {}.", hash, e); } @@ -325,7 +257,11 @@ impl BlockManager { // ---- Reading and writing blocks locally ---- /// Write a block to disk - async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result { + pub(crate) async fn write_block( + &self, + hash: &Hash, + data: &DataBlock, + ) -> Result { let write_size = data.inner_buffer().len() as u64; let res = self @@ -361,7 +297,7 @@ impl BlockManager { Ok(c) => c, Err(e) => { // Not found but maybe we should have had it ?? - self.put_to_resync(hash, 2 * BLOCK_RW_TIMEOUT)?; + self.resync.put_to_resync(hash, 2 * BLOCK_RW_TIMEOUT)?; return Err(Into::into(e)); } }; @@ -388,7 +324,7 @@ impl BlockManager { .await .move_block_to_corrupted(hash, self) .await?; - self.put_to_resync(hash, Duration::from_millis(0))?; + self.resync.put_to_resync(hash, Duration::from_millis(0))?; return Err(Error::CorruptData(*hash)); } @@ -432,322 +368,6 @@ impl BlockManager { path.set_extension(""); fs::metadata(&path).await.map(|_| false).map_err(Into::into) } - - // ---- Resync loop ---- - - // This part manages a queue of blocks that need to be - // "resynchronized", i.e. that need to have a check that - // they are at present if we need them, or that they are - // deleted once the garbage collection delay has passed. - // - // Here are some explanations on how the resync queue works. - // There are two Sled trees that are used to have information - // about the status of blocks that need to be resynchronized: - // - // - resync_queue: a tree that is ordered first by a timestamp - // (in milliseconds since Unix epoch) that is the time at which - // the resync must be done, and second by block hash. - // The key in this tree is just: - // concat(timestamp (8 bytes), hash (32 bytes)) - // The value is the same 32-byte hash. - // - // - resync_errors: a tree that indicates for each block - // if the last resync resulted in an error, and if so, - // the following two informations (see the ErrorCounter struct): - // - how many consecutive resync errors for this block? - // - when was the last try? - // These two informations are used to implement an - // exponential backoff retry strategy. - // The key in this tree is the 32-byte hash of the block, - // and the value is the encoded ErrorCounter value. - // - // We need to have these two trees, because the resync queue - // is not just a queue of items to process, but a set of items - // that are waiting a specific delay until we can process them - // (the delay being necessary both internally for the exponential - // backoff strategy, and exposed as a parameter when adding items - // to the queue, e.g. to wait until the GC delay has passed). - // This is why we need one tree ordered by time, and one - // ordered by identifier of item to be processed (block hash). - // - // When the worker wants to process an item it takes from - // resync_queue, it checks in resync_errors that if there is an - // exponential back-off delay to await, it has passed before we - // process the item. If not, the item in the queue is skipped - // (but added back for later processing after the time of the - // delay). - // - // An alternative that would have seemed natural is to - // only add items to resync_queue with a processing time that is - // after the delay, but there are several issues with this: - // - This requires to synchronize updates to resync_queue and - // resync_errors (with the current model, there is only one thread, - // the worker thread, that accesses resync_errors, - // so no need to synchronize) by putting them both in a lock. - // This would mean that block_incref might need to take a lock - // before doing its thing, meaning it has much more chances of - // not completing successfully if something bad happens to Garage. - // Currently Garage is not able to recover from block_incref that - // doesn't complete successfully, because it is necessary to ensure - // the consistency between the state of the block manager and - // information in the BlockRef table. - // - If a resync fails, we put that block in the resync_errors table, - // and also add it back to resync_queue to be processed after - // the exponential back-off delay, - // but maybe the block is already scheduled to be resynced again - // at another time that is before the exponential back-off delay, - // and we have no way to check that easily. This means that - // in all cases, we need to check the resync_errors table - // in the resync loop at the time when a block is popped from - // the resync_queue. - // Overall, the current design is therefore simpler and more robust - // because it tolerates inconsistencies between the resync_queue - // and resync_errors table (items being scheduled in resync_queue - // for times that are earlier than the exponential back-off delay - // is a natural condition that is handled properly). - - fn spawn_background_workers(self: Arc) { - // Launch a background workers for background resync loop processing - let background = self.system.background.clone(); - let worker = ResyncWorker::new(self.clone()); - tokio::spawn(async move { - tokio::time::sleep(Duration::from_secs(10)).await; - background.spawn_worker(worker); - }); - - // Launch a background worker for data store scrubs - let (scrub_tx, scrub_rx) = mpsc::channel(1); - self.tx_scrub_command.store(Some(Arc::new(scrub_tx))); - let scrub_worker = ScrubWorker::new(self.clone(), scrub_rx); - self.system.background.spawn_worker(scrub_worker); - } - - pub(crate) fn put_to_resync(&self, hash: &Hash, delay: Duration) -> db::Result<()> { - let when = now_msec() + delay.as_millis() as u64; - self.put_to_resync_at(hash, when) - } - - fn put_to_resync_at(&self, hash: &Hash, when: u64) -> db::Result<()> { - trace!("Put resync_queue: {} {:?}", when, hash); - let mut key = u64::to_be_bytes(when).to_vec(); - key.extend(hash.as_ref()); - self.resync_queue.insert(key, hash.as_ref())?; - self.resync_notify.notify_waiters(); - Ok(()) - } - - async fn resync_iter(&self) -> Result { - if let Some((time_bytes, hash_bytes)) = self.resync_queue.first()? { - let time_msec = u64::from_be_bytes(time_bytes[0..8].try_into().unwrap()); - let now = now_msec(); - - if now >= time_msec { - let hash = Hash::try_from(&hash_bytes[..]).unwrap(); - - if let Some(ec) = self.resync_errors.get(hash.as_slice())? { - let ec = ErrorCounter::decode(&ec); - if now < ec.next_try() { - // if next retry after an error is not yet, - // don't do resync and return early, but still - // make sure the item is still in queue at expected time - self.put_to_resync_at(&hash, ec.next_try())?; - // ec.next_try() > now >= time_msec, so this remove - // is not removing the one we added just above - // (we want to do the remove after the insert to ensure - // that the item is not lost if we crash in-between) - self.resync_queue.remove(time_bytes)?; - return Ok(ResyncIterResult::BusyDidNothing); - } - } - - let tracer = opentelemetry::global::tracer("garage"); - let trace_id = gen_uuid(); - let span = tracer - .span_builder("Resync block") - .with_trace_id( - opentelemetry::trace::TraceId::from_hex(&hex::encode( - &trace_id.as_slice()[..16], - )) - .unwrap(), - ) - .with_attributes(vec![KeyValue::new("block", format!("{:?}", hash))]) - .start(&tracer); - - let res = self - .resync_block(&hash) - .with_context(Context::current_with_span(span)) - .bound_record_duration(&self.metrics.resync_duration) - .await; - - self.metrics.resync_counter.add(1); - - if let Err(e) = &res { - self.metrics.resync_error_counter.add(1); - warn!("Error when resyncing {:?}: {}", hash, e); - - let err_counter = match self.resync_errors.get(hash.as_slice())? { - Some(ec) => ErrorCounter::decode(&ec).add1(now + 1), - None => ErrorCounter::new(now + 1), - }; - - self.resync_errors - .insert(hash.as_slice(), err_counter.encode())?; - - self.put_to_resync_at(&hash, err_counter.next_try())?; - // err_counter.next_try() >= now + 1 > now, - // the entry we remove from the queue is not - // the entry we inserted with put_to_resync_at - self.resync_queue.remove(time_bytes)?; - } else { - self.resync_errors.remove(hash.as_slice())?; - self.resync_queue.remove(time_bytes)?; - } - - Ok(ResyncIterResult::BusyDidSomething) - } else { - Ok(ResyncIterResult::IdleFor(Duration::from_millis( - time_msec - now, - ))) - } - } else { - // Here we wait either for a notification that an item has been - // added to the queue, or for a constant delay of 10 secs to expire. - // The delay avoids a race condition where the notification happens - // between the time we checked the queue and the first poll - // to resync_notify.notified(): if that happens, we'll just loop - // back 10 seconds later, which is fine. - Ok(ResyncIterResult::IdleFor(Duration::from_secs(10))) - } - } - - async fn resync_block(&self, hash: &Hash) -> Result<(), Error> { - let BlockStatus { exists, needed } = self - .mutation_lock - .lock() - .await - .check_block_status(hash, self) - .await?; - - if exists != needed.is_needed() || exists != needed.is_nonzero() { - debug!( - "Resync block {:?}: exists {}, nonzero rc {}, deletable {}", - hash, - exists, - needed.is_nonzero(), - needed.is_deletable(), - ); - } - - if exists && needed.is_deletable() { - info!("Resync block {:?}: offloading and deleting", hash); - - let mut who = self.replication.write_nodes(hash); - if who.len() < self.replication.write_quorum() { - return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string())); - } - who.retain(|id| *id != self.system.id); - - let msg = Arc::new(BlockRpc::NeedBlockQuery(*hash)); - let who_needs_fut = who.iter().map(|to| { - self.system.rpc.call_arc( - &self.endpoint, - *to, - msg.clone(), - RequestStrategy::with_priority(PRIO_BACKGROUND) - .with_timeout(NEED_BLOCK_QUERY_TIMEOUT), - ) - }); - let who_needs_resps = join_all(who_needs_fut).await; - - let mut need_nodes = vec![]; - for (node, needed) in who.iter().zip(who_needs_resps.into_iter()) { - match needed.err_context("NeedBlockQuery RPC")? { - BlockRpc::NeedBlockReply(needed) => { - if needed { - need_nodes.push(*node); - } - } - m => { - return Err(Error::unexpected_rpc_message(m)); - } - } - } - - if !need_nodes.is_empty() { - trace!( - "Block {:?} needed by {} nodes, sending", - hash, - need_nodes.len() - ); - - for node in need_nodes.iter() { - self.metrics - .resync_send_counter - .add(1, &[KeyValue::new("to", format!("{:?}", node))]); - } - - let put_block_message = self.read_block(hash).await?; - self.system - .rpc - .try_call_many( - &self.endpoint, - &need_nodes[..], - put_block_message, - RequestStrategy::with_priority(PRIO_BACKGROUND) - .with_quorum(need_nodes.len()) - .with_timeout(BLOCK_RW_TIMEOUT), - ) - .await - .err_context("PutBlock RPC")?; - } - info!( - "Deleting unneeded block {:?}, offload finished ({} / {})", - hash, - need_nodes.len(), - who.len() - ); - - self.mutation_lock - .lock() - .await - .delete_if_unneeded(hash, self) - .await?; - - self.rc.clear_deleted_block_rc(hash)?; - } - - if needed.is_nonzero() && !exists { - info!( - "Resync block {:?}: fetching absent but needed block (refcount > 0)", - hash - ); - - let block_data = self.rpc_get_raw_block(hash).await?; - - self.metrics.resync_recv_counter.add(1); - - self.write_block(hash, &block_data).await?; - } - - Ok(()) - } - - async fn update_resync_persisted( - &self, - update: impl Fn(&mut ResyncPersistedConfig), - ) -> Result<(), Error> { - let mut cfg: ResyncPersistedConfig = *self.resync_persisted.load().as_ref(); - update(&mut cfg); - self.resync_persister.save_async(&cfg).await?; - self.resync_persisted.store(Arc::new(cfg)); - self.resync_notify.notify_one(); - Ok(()) - } - - pub async fn set_resync_tranquility(&self, tranquility: u32) -> Result<(), Error> { - self.update_resync_persisted(|cfg| cfg.tranquility = tranquility) - .await - } } #[async_trait] @@ -766,92 +386,13 @@ impl EndpointHandler for BlockManager { } } -#[derive(Serialize, Deserialize, Clone, Copy)] -struct ResyncPersistedConfig { - tranquility: u32, -} - -struct ResyncWorker { - manager: Arc, - tranquilizer: Tranquilizer, - next_delay: Duration, -} - -impl ResyncWorker { - fn new(manager: Arc) -> Self { - Self { - manager, - tranquilizer: Tranquilizer::new(30), - next_delay: Duration::from_secs(10), - } - } -} - -#[async_trait] -impl Worker for ResyncWorker { - fn name(&self) -> String { - "Block resync worker".into() - } - - fn info(&self) -> Option { - let mut ret = vec![]; - ret.push(format!( - "tranquility = {}", - self.manager.resync_persisted.load().tranquility - )); - - let qlen = self.manager.resync_queue_len().unwrap_or(0); - if qlen > 0 { - ret.push(format!("{} blocks in queue", qlen)); - } - - let elen = self.manager.resync_errors_len().unwrap_or(0); - if elen > 0 { - ret.push(format!("{} blocks in error state", elen)); - } - - Some(ret.join(", ")) - } - - async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { - self.tranquilizer.reset(); - match self.manager.resync_iter().await { - Ok(ResyncIterResult::BusyDidSomething) => Ok(self - .tranquilizer - .tranquilize_worker(self.manager.resync_persisted.load().tranquility)), - Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerState::Busy), - Ok(ResyncIterResult::IdleFor(delay)) => { - self.next_delay = delay; - Ok(WorkerState::Idle) - } - Err(e) => { - // The errors that we have here are only Sled errors - // We don't really know how to handle them so just ¯\_(ツ)_/¯ - // (there is kind of an assumption that Sled won't error on us, - // if it does there is not much we can do -- TODO should we just panic?) - // Here we just give the error to the worker manager, - // it will print it to the logs and increment a counter - Err(e.into()) - } - } - } - - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { - select! { - _ = tokio::time::sleep(self.next_delay) => (), - _ = self.manager.resync_notify.notified() => (), - }; - WorkerState::Busy - } -} - -struct BlockStatus { - exists: bool, - needed: RcEntry, +pub(crate) struct BlockStatus { + pub(crate) exists: bool, + pub(crate) needed: RcEntry, } impl BlockManagerLocked { - async fn check_block_status( + pub(crate) async fn check_block_status( &self, hash: &Hash, mgr: &BlockManager, @@ -938,7 +479,11 @@ impl BlockManagerLocked { Ok(()) } - async fn delete_if_unneeded(&self, hash: &Hash, mgr: &BlockManager) -> Result<(), Error> { + pub(crate) async fn delete_if_unneeded( + &self, + hash: &Hash, + mgr: &BlockManager, + ) -> Result<(), Error> { let BlockStatus { exists, needed } = self.check_block_status(hash, mgr).await?; if exists && needed.is_deletable() { @@ -952,50 +497,3 @@ impl BlockManagerLocked { Ok(()) } } - -/// Counts the number of errors when resyncing a block, -/// and the time of the last try. -/// Used to implement exponential backoff. -#[derive(Clone, Copy, Debug)] -struct ErrorCounter { - errors: u64, - last_try: u64, -} - -impl ErrorCounter { - fn new(now: u64) -> Self { - Self { - errors: 1, - last_try: now, - } - } - - fn decode(data: &[u8]) -> Self { - Self { - errors: u64::from_be_bytes(data[0..8].try_into().unwrap()), - last_try: u64::from_be_bytes(data[8..16].try_into().unwrap()), - } - } - fn encode(&self) -> Vec { - [ - u64::to_be_bytes(self.errors), - u64::to_be_bytes(self.last_try), - ] - .concat() - } - - fn add1(self, now: u64) -> Self { - Self { - errors: self.errors + 1, - last_try: now, - } - } - - fn delay_msec(&self) -> u64 { - (RESYNC_RETRY_DELAY.as_millis() as u64) - << std::cmp::min(self.errors - 1, RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER) - } - fn next_try(&self) -> u64 { - self.last_try + self.delay_msec() - } -} diff --git a/src/block/repair.rs b/src/block/repair.rs index 18e1de95..e2884b69 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -112,7 +112,9 @@ impl Worker for RepairWorker { } for hash in batch_of_hashes.into_iter() { - self.manager.put_to_resync(&hash, Duration::from_secs(0))?; + self.manager + .resync + .put_to_resync(&hash, Duration::from_secs(0))?; self.next_start = Some(hash) } @@ -124,7 +126,9 @@ impl Worker for RepairWorker { // This allows us to find blocks we are storing but don't actually need, // so that we can offload them if necessary and then delete them locally. if let Some(hash) = bi.next().await? { - self.manager.put_to_resync(&hash, Duration::from_secs(0))?; + self.manager + .resync + .put_to_resync(&hash, Duration::from_secs(0))?; Ok(WorkerState::Busy) } else { Ok(WorkerState::Done) diff --git a/src/block/resync.rs b/src/block/resync.rs new file mode 100644 index 00000000..2a8184b7 --- /dev/null +++ b/src/block/resync.rs @@ -0,0 +1,536 @@ +use std::convert::TryInto; +use std::sync::Arc; +use std::time::Duration; + +use arc_swap::ArcSwap; +use async_trait::async_trait; +use serde::{Deserialize, Serialize}; + +use futures::future::*; +use tokio::select; +use tokio::sync::{watch, Notify}; + +use opentelemetry::{ + trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer}, + Context, KeyValue, +}; + +use garage_db as db; +use garage_db::counted_tree_hack::CountedTree; + +use garage_util::background::*; +use garage_util::data::*; +use garage_util::error::*; +use garage_util::metrics::RecordDuration; +use garage_util::persister::Persister; +use garage_util::time::*; +use garage_util::tranquilizer::Tranquilizer; + +use garage_rpc::system::System; +use garage_rpc::*; + +use garage_table::replication::TableReplication; + +use crate::manager::*; + +// Timeout for RPCs that ask other nodes whether they need a copy +// of a given block before we delete it locally +pub(crate) const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5); + +// The delay between the time where a resync operation fails +// and the time when it is retried, with exponential backoff +// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure). +pub(crate) const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60); +// The minimum retry delay is 60 seconds = 1 minute +// The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour) +pub(crate) const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6; +// Resync tranquility is initially set to 2, but can be changed in the CLI +// and the updated version is persisted over Garage restarts +const INITIAL_RESYNC_TRANQUILITY: u32 = 2; + +pub struct BlockResyncManager { + pub(crate) queue: CountedTree, + pub(crate) notify: Notify, + pub(crate) errors: CountedTree, + + persister: Persister, + persisted: ArcSwap, +} + +#[derive(Serialize, Deserialize, Clone, Copy)] +struct ResyncPersistedConfig { + tranquility: u32, +} + +enum ResyncIterResult { + BusyDidSomething, + BusyDidNothing, + IdleFor(Duration), +} + +impl BlockResyncManager { + pub(crate) fn new(db: &db::Db, system: &System) -> Self { + let queue = db + .open_tree("block_local_resync_queue") + .expect("Unable to open block_local_resync_queue tree"); + let queue = CountedTree::new(queue).expect("Could not count block_local_resync_queue"); + + let errors = db + .open_tree("block_local_resync_errors") + .expect("Unable to open block_local_resync_errors tree"); + let errors = CountedTree::new(errors).expect("Could not count block_local_resync_errors"); + + let persister = Persister::new(&system.metadata_dir, "resync_cfg"); + let persisted = match persister.load() { + Ok(v) => v, + Err(_) => ResyncPersistedConfig { + tranquility: INITIAL_RESYNC_TRANQUILITY, + }, + }; + + Self { + queue, + notify: Notify::new(), + errors, + persister, + persisted: ArcSwap::new(Arc::new(persisted)), + } + } + + /// Get lenght of resync queue + pub fn queue_len(&self) -> Result { + // This currently can't return an error because the CountedTree hack + // doesn't error on .len(), but this will change when we remove the hack + // (hopefully someday!) + Ok(self.queue.len()) + } + + /// Get number of blocks that have an error + pub fn errors_len(&self) -> Result { + // (see queue_len comment) + Ok(self.errors.len()) + } + + // ---- Resync loop ---- + + // This part manages a queue of blocks that need to be + // "resynchronized", i.e. that need to have a check that + // they are at present if we need them, or that they are + // deleted once the garbage collection delay has passed. + // + // Here are some explanations on how the resync queue works. + // There are two Sled trees that are used to have information + // about the status of blocks that need to be resynchronized: + // + // - resync.queue: a tree that is ordered first by a timestamp + // (in milliseconds since Unix epoch) that is the time at which + // the resync must be done, and second by block hash. + // The key in this tree is just: + // concat(timestamp (8 bytes), hash (32 bytes)) + // The value is the same 32-byte hash. + // + // - resync.errors: a tree that indicates for each block + // if the last resync resulted in an error, and if so, + // the following two informations (see the ErrorCounter struct): + // - how many consecutive resync errors for this block? + // - when was the last try? + // These two informations are used to implement an + // exponential backoff retry strategy. + // The key in this tree is the 32-byte hash of the block, + // and the value is the encoded ErrorCounter value. + // + // We need to have these two trees, because the resync queue + // is not just a queue of items to process, but a set of items + // that are waiting a specific delay until we can process them + // (the delay being necessary both internally for the exponential + // backoff strategy, and exposed as a parameter when adding items + // to the queue, e.g. to wait until the GC delay has passed). + // This is why we need one tree ordered by time, and one + // ordered by identifier of item to be processed (block hash). + // + // When the worker wants to process an item it takes from + // resync.queue, it checks in resync.errors that if there is an + // exponential back-off delay to await, it has passed before we + // process the item. If not, the item in the queue is skipped + // (but added back for later processing after the time of the + // delay). + // + // An alternative that would have seemed natural is to + // only add items to resync.queue with a processing time that is + // after the delay, but there are several issues with this: + // - This requires to synchronize updates to resync.queue and + // resync.errors (with the current model, there is only one thread, + // the worker thread, that accesses resync.errors, + // so no need to synchronize) by putting them both in a lock. + // This would mean that block_incref might need to take a lock + // before doing its thing, meaning it has much more chances of + // not completing successfully if something bad happens to Garage. + // Currently Garage is not able to recover from block_incref that + // doesn't complete successfully, because it is necessary to ensure + // the consistency between the state of the block manager and + // information in the BlockRef table. + // - If a resync fails, we put that block in the resync.errors table, + // and also add it back to resync.queue to be processed after + // the exponential back-off delay, + // but maybe the block is already scheduled to be resynced again + // at another time that is before the exponential back-off delay, + // and we have no way to check that easily. This means that + // in all cases, we need to check the resync.errors table + // in the resync loop at the time when a block is popped from + // the resync.queue. + // Overall, the current design is therefore simpler and more robust + // because it tolerates inconsistencies between the resync.queue + // and resync.errors table (items being scheduled in resync.queue + // for times that are earlier than the exponential back-off delay + // is a natural condition that is handled properly). + + pub(crate) fn put_to_resync(&self, hash: &Hash, delay: Duration) -> db::Result<()> { + let when = now_msec() + delay.as_millis() as u64; + self.put_to_resync_at(hash, when) + } + + pub(crate) fn put_to_resync_at(&self, hash: &Hash, when: u64) -> db::Result<()> { + trace!("Put resync_queue: {} {:?}", when, hash); + let mut key = u64::to_be_bytes(when).to_vec(); + key.extend(hash.as_ref()); + self.queue.insert(key, hash.as_ref())?; + self.notify.notify_waiters(); + Ok(()) + } + + async fn resync_iter(&self, manager: &BlockManager) -> Result { + if let Some((time_bytes, hash_bytes)) = self.queue.first()? { + let time_msec = u64::from_be_bytes(time_bytes[0..8].try_into().unwrap()); + let now = now_msec(); + + if now >= time_msec { + let hash = Hash::try_from(&hash_bytes[..]).unwrap(); + + if let Some(ec) = self.errors.get(hash.as_slice())? { + let ec = ErrorCounter::decode(&ec); + if now < ec.next_try() { + // if next retry after an error is not yet, + // don't do resync and return early, but still + // make sure the item is still in queue at expected time + self.put_to_resync_at(&hash, ec.next_try())?; + // ec.next_try() > now >= time_msec, so this remove + // is not removing the one we added just above + // (we want to do the remove after the insert to ensure + // that the item is not lost if we crash in-between) + self.queue.remove(time_bytes)?; + return Ok(ResyncIterResult::BusyDidNothing); + } + } + + let tracer = opentelemetry::global::tracer("garage"); + let trace_id = gen_uuid(); + let span = tracer + .span_builder("Resync block") + .with_trace_id( + opentelemetry::trace::TraceId::from_hex(&hex::encode( + &trace_id.as_slice()[..16], + )) + .unwrap(), + ) + .with_attributes(vec![KeyValue::new("block", format!("{:?}", hash))]) + .start(&tracer); + + let res = self + .resync_block(manager, &hash) + .with_context(Context::current_with_span(span)) + .bound_record_duration(&manager.metrics.resync_duration) + .await; + + manager.metrics.resync_counter.add(1); + + if let Err(e) = &res { + manager.metrics.resync_error_counter.add(1); + warn!("Error when resyncing {:?}: {}", hash, e); + + let err_counter = match self.errors.get(hash.as_slice())? { + Some(ec) => ErrorCounter::decode(&ec).add1(now + 1), + None => ErrorCounter::new(now + 1), + }; + + self.errors.insert(hash.as_slice(), err_counter.encode())?; + + self.put_to_resync_at(&hash, err_counter.next_try())?; + // err_counter.next_try() >= now + 1 > now, + // the entry we remove from the queue is not + // the entry we inserted with put_to_resync_at + self.queue.remove(time_bytes)?; + } else { + self.errors.remove(hash.as_slice())?; + self.queue.remove(time_bytes)?; + } + + Ok(ResyncIterResult::BusyDidSomething) + } else { + Ok(ResyncIterResult::IdleFor(Duration::from_millis( + time_msec - now, + ))) + } + } else { + // Here we wait either for a notification that an item has been + // added to the queue, or for a constant delay of 10 secs to expire. + // The delay avoids a race condition where the notification happens + // between the time we checked the queue and the first poll + // to resync_notify.notified(): if that happens, we'll just loop + // back 10 seconds later, which is fine. + Ok(ResyncIterResult::IdleFor(Duration::from_secs(10))) + } + } + + async fn resync_block(&self, manager: &BlockManager, hash: &Hash) -> Result<(), Error> { + let BlockStatus { exists, needed } = manager + .mutation_lock + .lock() + .await + .check_block_status(hash, manager) + .await?; + + if exists != needed.is_needed() || exists != needed.is_nonzero() { + debug!( + "Resync block {:?}: exists {}, nonzero rc {}, deletable {}", + hash, + exists, + needed.is_nonzero(), + needed.is_deletable(), + ); + } + + if exists && needed.is_deletable() { + info!("Resync block {:?}: offloading and deleting", hash); + + let mut who = manager.replication.write_nodes(hash); + if who.len() < manager.replication.write_quorum() { + return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string())); + } + who.retain(|id| *id != manager.system.id); + + let msg = Arc::new(BlockRpc::NeedBlockQuery(*hash)); + let who_needs_fut = who.iter().map(|to| { + manager.system.rpc.call_arc( + &manager.endpoint, + *to, + msg.clone(), + RequestStrategy::with_priority(PRIO_BACKGROUND) + .with_timeout(NEED_BLOCK_QUERY_TIMEOUT), + ) + }); + let who_needs_resps = join_all(who_needs_fut).await; + + let mut need_nodes = vec![]; + for (node, needed) in who.iter().zip(who_needs_resps.into_iter()) { + match needed.err_context("NeedBlockQuery RPC")? { + BlockRpc::NeedBlockReply(needed) => { + if needed { + need_nodes.push(*node); + } + } + m => { + return Err(Error::unexpected_rpc_message(m)); + } + } + } + + if !need_nodes.is_empty() { + trace!( + "Block {:?} needed by {} nodes, sending", + hash, + need_nodes.len() + ); + + for node in need_nodes.iter() { + manager + .metrics + .resync_send_counter + .add(1, &[KeyValue::new("to", format!("{:?}", node))]); + } + + let put_block_message = manager.read_block(hash).await?; + manager + .system + .rpc + .try_call_many( + &manager.endpoint, + &need_nodes[..], + put_block_message, + RequestStrategy::with_priority(PRIO_BACKGROUND) + .with_quorum(need_nodes.len()) + .with_timeout(BLOCK_RW_TIMEOUT), + ) + .await + .err_context("PutBlock RPC")?; + } + info!( + "Deleting unneeded block {:?}, offload finished ({} / {})", + hash, + need_nodes.len(), + who.len() + ); + + manager + .mutation_lock + .lock() + .await + .delete_if_unneeded(hash, manager) + .await?; + + manager.rc.clear_deleted_block_rc(hash)?; + } + + if needed.is_nonzero() && !exists { + info!( + "Resync block {:?}: fetching absent but needed block (refcount > 0)", + hash + ); + + let block_data = manager.rpc_get_raw_block(hash).await?; + + manager.metrics.resync_recv_counter.add(1); + + manager.write_block(hash, &block_data).await?; + } + + Ok(()) + } + + async fn update_persisted( + &self, + update: impl Fn(&mut ResyncPersistedConfig), + ) -> Result<(), Error> { + let mut cfg: ResyncPersistedConfig = *self.persisted.load().as_ref(); + update(&mut cfg); + self.persister.save_async(&cfg).await?; + self.persisted.store(Arc::new(cfg)); + self.notify.notify_one(); + Ok(()) + } + + pub async fn set_tranquility(&self, tranquility: u32) -> Result<(), Error> { + self.update_persisted(|cfg| cfg.tranquility = tranquility) + .await + } +} + +pub(crate) struct ResyncWorker { + manager: Arc, + tranquilizer: Tranquilizer, + next_delay: Duration, +} + +impl ResyncWorker { + pub(crate) fn new(manager: Arc) -> Self { + Self { + manager, + tranquilizer: Tranquilizer::new(30), + next_delay: Duration::from_secs(10), + } + } +} + +#[async_trait] +impl Worker for ResyncWorker { + fn name(&self) -> String { + "Block resync worker".into() + } + + fn info(&self) -> Option { + let mut ret = vec![]; + ret.push(format!( + "tranquility = {}", + self.manager.resync.persisted.load().tranquility + )); + + let qlen = self.manager.resync.queue_len().unwrap_or(0); + if qlen > 0 { + ret.push(format!("{} blocks in queue", qlen)); + } + + let elen = self.manager.resync.errors_len().unwrap_or(0); + if elen > 0 { + ret.push(format!("{} blocks in error state", elen)); + } + + Some(ret.join(", ")) + } + + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { + self.tranquilizer.reset(); + match self.manager.resync.resync_iter(&self.manager).await { + Ok(ResyncIterResult::BusyDidSomething) => Ok(self + .tranquilizer + .tranquilize_worker(self.manager.resync.persisted.load().tranquility)), + Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerState::Busy), + Ok(ResyncIterResult::IdleFor(delay)) => { + self.next_delay = delay; + Ok(WorkerState::Idle) + } + Err(e) => { + // The errors that we have here are only Sled errors + // We don't really know how to handle them so just ¯\_(ツ)_/¯ + // (there is kind of an assumption that Sled won't error on us, + // if it does there is not much we can do -- TODO should we just panic?) + // Here we just give the error to the worker manager, + // it will print it to the logs and increment a counter + Err(e.into()) + } + } + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { + select! { + _ = tokio::time::sleep(self.next_delay) => (), + _ = self.manager.resync.notify.notified() => (), + }; + WorkerState::Busy + } +} + +/// Counts the number of errors when resyncing a block, +/// and the time of the last try. +/// Used to implement exponential backoff. +#[derive(Clone, Copy, Debug)] +struct ErrorCounter { + errors: u64, + last_try: u64, +} + +impl ErrorCounter { + fn new(now: u64) -> Self { + Self { + errors: 1, + last_try: now, + } + } + + fn decode(data: &[u8]) -> Self { + Self { + errors: u64::from_be_bytes(data[0..8].try_into().unwrap()), + last_try: u64::from_be_bytes(data[8..16].try_into().unwrap()), + } + } + fn encode(&self) -> Vec { + [ + u64::to_be_bytes(self.errors), + u64::to_be_bytes(self.last_try), + ] + .concat() + } + + fn add1(self, now: u64) -> Self { + Self { + errors: self.errors + 1, + last_try: now, + } + } + + fn delay_msec(&self) -> u64 { + (RESYNC_RETRY_DELAY.as_millis() as u64) + << std::cmp::min(self.errors - 1, RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER) + } + fn next_try(&self) -> u64 { + self.last_try + self.delay_msec() + } +} diff --git a/src/garage/admin.rs b/src/garage/admin.rs index 1d80889c..9f4764df 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -781,13 +781,13 @@ impl AdminRpcHandler { writeln!( &mut ret, " resync queue length: {}", - self.garage.block_manager.resync_queue_len()? + self.garage.block_manager.resync.queue_len()? ) .unwrap(); writeln!( &mut ret, " blocks with resync errors: {}", - self.garage.block_manager.resync_errors_len()? + self.garage.block_manager.resync.errors_len()? ) .unwrap(); @@ -850,7 +850,8 @@ impl AdminRpcHandler { WorkerSetCmd::ResyncTranquility { tranquility } => { self.garage .block_manager - .set_resync_tranquility(tranquility) + .resync + .set_tranquility(tranquility) .await?; Ok(AdminRpc::Ok("Resync tranquility updated".into())) } From 5e8baa433d743a06ab3ee90f375f24c3c36fc236 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 2 Sep 2022 16:52:22 +0200 Subject: [PATCH 3/6] Make BlockManagerLocked fully private again --- src/block/manager.rs | 35 ++++++++++++++++++++++------------- src/block/resync.rs | 14 ++------------ 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index efb5349c..62ef96b9 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -68,7 +68,7 @@ pub struct BlockManager { compression_level: Option, - pub(crate) mutation_lock: Mutex, + mutation_lock: Mutex, pub(crate) rc: BlockRc, pub resync: BlockResyncManager, @@ -84,7 +84,7 @@ pub struct BlockManager { // This custom struct contains functions that must only be ran // when the lock is held. We ensure that it is the case by storing // it INSIDE a Mutex. -pub(crate) struct BlockManagerLocked(); +struct BlockManagerLocked(); impl BlockManager { pub fn new( @@ -331,17 +331,30 @@ impl BlockManager { Ok(data) } - /// Check if this node should have a block, but don't actually have it - async fn need_block(&self, hash: &Hash) -> Result { - let BlockStatus { exists, needed } = self - .mutation_lock + /// Check if this node has a block and whether it needs it + pub(crate) async fn check_block_status(&self, hash: &Hash) -> Result { + self.mutation_lock .lock() .await .check_block_status(hash, self) - .await?; + .await + } + + /// Check if this node should have a block, but don't actually have it + async fn need_block(&self, hash: &Hash) -> Result { + let BlockStatus { exists, needed } = self.check_block_status(hash).await?; Ok(needed.is_nonzero() && !exists) } + /// Delete block if it is not needed anymore + pub(crate) async fn delete_if_unneeded(&self, hash: &Hash) -> Result<(), Error> { + self.mutation_lock + .lock() + .await + .delete_if_unneeded(hash, self) + .await + } + /// Utility: gives the path of the directory in which a block should be found fn block_dir(&self, hash: &Hash) -> PathBuf { let mut path = self.data_dir.clone(); @@ -392,7 +405,7 @@ pub(crate) struct BlockStatus { } impl BlockManagerLocked { - pub(crate) async fn check_block_status( + async fn check_block_status( &self, hash: &Hash, mgr: &BlockManager, @@ -479,11 +492,7 @@ impl BlockManagerLocked { Ok(()) } - pub(crate) async fn delete_if_unneeded( - &self, - hash: &Hash, - mgr: &BlockManager, - ) -> Result<(), Error> { + async fn delete_if_unneeded(&self, hash: &Hash, mgr: &BlockManager) -> Result<(), Error> { let BlockStatus { exists, needed } = self.check_block_status(hash, mgr).await?; if exists && needed.is_deletable() { diff --git a/src/block/resync.rs b/src/block/resync.rs index 2a8184b7..dab08338 100644 --- a/src/block/resync.rs +++ b/src/block/resync.rs @@ -282,12 +282,7 @@ impl BlockResyncManager { } async fn resync_block(&self, manager: &BlockManager, hash: &Hash) -> Result<(), Error> { - let BlockStatus { exists, needed } = manager - .mutation_lock - .lock() - .await - .check_block_status(hash, manager) - .await?; + let BlockStatus { exists, needed } = manager.check_block_status(hash).await?; if exists != needed.is_needed() || exists != needed.is_nonzero() { debug!( @@ -370,12 +365,7 @@ impl BlockResyncManager { who.len() ); - manager - .mutation_lock - .lock() - .await - .delete_if_unneeded(hash, manager) - .await?; + manager.delete_if_unneeded(hash).await?; manager.rc.clear_deleted_block_rc(hash)?; } From 5d4b937a00882b9bf8b36f7430f3d1fe9db58903 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 2 Sep 2022 17:18:13 +0200 Subject: [PATCH 4/6] Ability to have up to 4 concurrently working resync workers --- src/block/manager.rs | 12 +++-- src/block/resync.rs | 92 +++++++++++++++++++++++++++++++++------ src/garage/admin.rs | 8 ++++ src/garage/cli/structs.rs | 5 ++- 4 files changed, 95 insertions(+), 22 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 62ef96b9..9240db25 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -125,13 +125,11 @@ impl BlockManager { }); block_manager.endpoint.set_handler(block_manager.clone()); - // Spawn one resync worker - let background = block_manager.system.background.clone(); - let worker = ResyncWorker::new(block_manager.clone()); - tokio::spawn(async move { - tokio::time::sleep(Duration::from_secs(10)).await; - background.spawn_worker(worker); - }); + // Spawn a bunch of resync workers + for index in 0..MAX_RESYNC_WORKERS { + let worker = ResyncWorker::new(index, block_manager.clone()); + block_manager.system.background.spawn_worker(worker); + } // Spawn scrub worker let scrub_worker = ScrubWorker::new(block_manager.clone(), scrub_rx); diff --git a/src/block/resync.rs b/src/block/resync.rs index dab08338..0f358d48 100644 --- a/src/block/resync.rs +++ b/src/block/resync.rs @@ -1,5 +1,6 @@ +use std::collections::HashSet; use std::convert::TryInto; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use std::time::Duration; use arc_swap::ArcSwap; @@ -44,6 +45,9 @@ pub(crate) const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60); // The minimum retry delay is 60 seconds = 1 minute // The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour) pub(crate) const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6; + +// No more than 4 resync workers can be running in the system +pub(crate) const MAX_RESYNC_WORKERS: usize = 4; // Resync tranquility is initially set to 2, but can be changed in the CLI // and the updated version is persisted over Garage restarts const INITIAL_RESYNC_TRANQUILITY: u32 = 2; @@ -53,12 +57,15 @@ pub struct BlockResyncManager { pub(crate) notify: Notify, pub(crate) errors: CountedTree, + busy_set: BusySet, + persister: Persister, persisted: ArcSwap, } #[derive(Serialize, Deserialize, Clone, Copy)] struct ResyncPersistedConfig { + n_workers: usize, tranquility: u32, } @@ -68,6 +75,14 @@ enum ResyncIterResult { IdleFor(Duration), } +type BusySet = Arc>>>; + +struct BusyBlock { + time_bytes: Vec, + hash_bytes: Vec, + busy_set: BusySet, +} + impl BlockResyncManager { pub(crate) fn new(db: &db::Db, system: &System) -> Self { let queue = db @@ -84,6 +99,7 @@ impl BlockResyncManager { let persisted = match persister.load() { Ok(v) => v, Err(_) => ResyncPersistedConfig { + n_workers: 1, tranquility: INITIAL_RESYNC_TRANQUILITY, }, }; @@ -92,6 +108,7 @@ impl BlockResyncManager { queue, notify: Notify::new(), errors, + busy_set: Arc::new(Mutex::new(HashSet::new())), persister, persisted: ArcSwap::new(Arc::new(persisted)), } @@ -199,12 +216,12 @@ impl BlockResyncManager { } async fn resync_iter(&self, manager: &BlockManager) -> Result { - if let Some((time_bytes, hash_bytes)) = self.queue.first()? { - let time_msec = u64::from_be_bytes(time_bytes[0..8].try_into().unwrap()); + if let Some(block) = self.get_block_to_resync()? { + let time_msec = u64::from_be_bytes(block.time_bytes[0..8].try_into().unwrap()); let now = now_msec(); if now >= time_msec { - let hash = Hash::try_from(&hash_bytes[..]).unwrap(); + let hash = Hash::try_from(&block.hash_bytes[..]).unwrap(); if let Some(ec) = self.errors.get(hash.as_slice())? { let ec = ErrorCounter::decode(&ec); @@ -217,7 +234,7 @@ impl BlockResyncManager { // is not removing the one we added just above // (we want to do the remove after the insert to ensure // that the item is not lost if we crash in-between) - self.queue.remove(time_bytes)?; + self.queue.remove(&block.time_bytes)?; return Ok(ResyncIterResult::BusyDidNothing); } } @@ -258,10 +275,10 @@ impl BlockResyncManager { // err_counter.next_try() >= now + 1 > now, // the entry we remove from the queue is not // the entry we inserted with put_to_resync_at - self.queue.remove(time_bytes)?; + self.queue.remove(&block.time_bytes)?; } else { self.errors.remove(hash.as_slice())?; - self.queue.remove(time_bytes)?; + self.queue.remove(&block.time_bytes)?; } Ok(ResyncIterResult::BusyDidSomething) @@ -281,6 +298,22 @@ impl BlockResyncManager { } } + fn get_block_to_resync(&self) -> Result, db::Error> { + let mut busy = self.busy_set.lock().unwrap(); + for it in self.queue.iter()? { + let (time_bytes, hash_bytes) = it?; + if !busy.contains(&time_bytes) { + busy.insert(time_bytes.clone()); + return Ok(Some(BusyBlock { + time_bytes, + hash_bytes, + busy_set: self.busy_set.clone(), + })); + } + } + return Ok(None); + } + async fn resync_block(&self, manager: &BlockManager, hash: &Hash) -> Result<(), Error> { let BlockStatus { exists, needed } = manager.check_block_status(hash).await?; @@ -394,25 +427,44 @@ impl BlockResyncManager { update(&mut cfg); self.persister.save_async(&cfg).await?; self.persisted.store(Arc::new(cfg)); - self.notify.notify_one(); + self.notify.notify_waiters(); Ok(()) } + pub async fn set_n_workers(&self, n_workers: usize) -> Result<(), Error> { + if n_workers < 1 || n_workers > MAX_RESYNC_WORKERS { + return Err(Error::Message(format!( + "Invalid number of resync workers, must be between 1 and {}", + MAX_RESYNC_WORKERS + ))); + } + self.update_persisted(|cfg| cfg.n_workers = n_workers).await + } + pub async fn set_tranquility(&self, tranquility: u32) -> Result<(), Error> { self.update_persisted(|cfg| cfg.tranquility = tranquility) .await } } +impl Drop for BusyBlock { + fn drop(&mut self) { + let mut busy = self.busy_set.lock().unwrap(); + busy.remove(&self.time_bytes); + } +} + pub(crate) struct ResyncWorker { + index: usize, manager: Arc, tranquilizer: Tranquilizer, next_delay: Duration, } impl ResyncWorker { - pub(crate) fn new(manager: Arc) -> Self { + pub(crate) fn new(index: usize, manager: Arc) -> Self { Self { + index, manager, tranquilizer: Tranquilizer::new(30), next_delay: Duration::from_secs(10), @@ -423,15 +475,18 @@ impl ResyncWorker { #[async_trait] impl Worker for ResyncWorker { fn name(&self) -> String { - "Block resync worker".into() + format!("Block resync worker #{}", self.index + 1) } fn info(&self) -> Option { + let persisted = self.manager.resync.persisted.load(); + + if self.index >= persisted.n_workers { + return Some("(unused)".into()); + } + let mut ret = vec![]; - ret.push(format!( - "tranquility = {}", - self.manager.resync.persisted.load().tranquility - )); + ret.push(format!("tranquility = {}", persisted.tranquility)); let qlen = self.manager.resync.queue_len().unwrap_or(0); if qlen > 0 { @@ -447,6 +502,10 @@ impl Worker for ResyncWorker { } async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { + if self.index >= self.manager.resync.persisted.load().n_workers { + return Ok(WorkerState::Idle); + } + self.tranquilizer.reset(); match self.manager.resync.resync_iter(&self.manager).await { Ok(ResyncIterResult::BusyDidSomething) => Ok(self @@ -470,10 +529,15 @@ impl Worker for ResyncWorker { } async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { + while self.index >= self.manager.resync.persisted.load().n_workers { + self.manager.resync.notify.notified().await + } + select! { _ = tokio::time::sleep(self.next_delay) => (), _ = self.manager.resync.notify.notified() => (), }; + WorkerState::Busy } } diff --git a/src/garage/admin.rs b/src/garage/admin.rs index 9f4764df..76261050 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -847,6 +847,14 @@ impl AdminRpcHandler { .await; Ok(AdminRpc::Ok("Scrub tranquility updated".into())) } + WorkerSetCmd::ResyncNWorkers { n_workers } => { + self.garage + .block_manager + .resync + .set_n_workers(n_workers) + .await?; + Ok(AdminRpc::Ok("Number of resync workers updated".into())) + } WorkerSetCmd::ResyncTranquility { tranquility } => { self.garage .block_manager diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index 1fba934f..0388cef5 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -524,7 +524,10 @@ pub enum WorkerSetCmd { /// Set tranquility of scrub operations #[structopt(name = "scrub-tranquility", version = version::garage())] ScrubTranquility { tranquility: u32 }, - /// Set tranquility of resync operations + /// Set number of concurrent block resync workers + #[structopt(name = "resync-n-workers", version = version::garage())] + ResyncNWorkers { n_workers: usize }, + /// Set tranquility of block resync operations #[structopt(name = "resync-tranquility", version = version::garage())] ResyncTranquility { tranquility: u32 }, } From e1751c8a9cb2a0d91b5aed636ee72ca4fa31ca68 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 2 Sep 2022 17:24:26 +0200 Subject: [PATCH 5/6] fix clippy --- src/block/resync.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/block/resync.rs b/src/block/resync.rs index 0f358d48..39e4d50f 100644 --- a/src/block/resync.rs +++ b/src/block/resync.rs @@ -311,7 +311,7 @@ impl BlockResyncManager { })); } } - return Ok(None); + Ok(None) } async fn resync_block(&self, manager: &BlockManager, hash: &Hash) -> Result<(), Error> { @@ -432,7 +432,7 @@ impl BlockResyncManager { } pub async fn set_n_workers(&self, n_workers: usize) -> Result<(), Error> { - if n_workers < 1 || n_workers > MAX_RESYNC_WORKERS { + if !(1..=MAX_RESYNC_WORKERS).contains(&n_workers) { return Err(Error::Message(format!( "Invalid number of resync workers, must be between 1 and {}", MAX_RESYNC_WORKERS From fd8074ad9b026f195d72d53b96b6c1a05182069a Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 5 Sep 2022 16:09:01 +0200 Subject: [PATCH 6/6] Update .drone.yml signature --- .drone.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.drone.yml b/.drone.yml index ce576278..277bfbc2 100644 --- a/.drone.yml +++ b/.drone.yml @@ -269,6 +269,6 @@ trigger: --- kind: signature -hmac: fa1f98f327abf88486c0c54984287285a4b951efa3776af9dd33b4d782b50815 +hmac: 362639b4c9541ad9bd06ff7f72b5235b2b0216bcb16eececd25285b6fe94ba6f ...