From e12bc3b5959c0aa5ae3c8a746c62bab2e7343a62 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 12:37:52 +0200 Subject: [PATCH 01/33] First try on background worker manager --- Cargo.lock | 3 +- src/db/Cargo.toml | 2 +- src/db/sqlite_adapter.rs | 2 +- src/util/Cargo.toml | 1 + src/util/background.rs | 160 ------------------------------ src/util/background/job_worker.rs | 52 ++++++++++ src/util/background/mod.rs | 91 +++++++++++++++++ src/util/background/worker.rs | 145 +++++++++++++++++++++++++++ src/util/lib.rs | 1 - 9 files changed, 293 insertions(+), 164 deletions(-) delete mode 100644 src/util/background.rs create mode 100644 src/util/background/job_worker.rs create mode 100644 src/util/background/mod.rs create mode 100644 src/util/background/worker.rs diff --git a/Cargo.lock b/Cargo.lock index ecdf8a57..486ad4a1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1065,11 +1065,11 @@ dependencies = [ "err-derive 0.3.1", "heed", "hexdump", - "log", "mktemp", "pretty_env_logger", "rusqlite", "sled", + "tracing", ] [[package]] @@ -1258,6 +1258,7 @@ dependencies = [ name = "garage_util" version = "0.7.0" dependencies = [ + "async-trait", "blake2", "chrono", "err-derive 0.3.1", diff --git a/src/db/Cargo.toml b/src/db/Cargo.toml index 6d8f64be..f697054b 100644 --- a/src/db/Cargo.toml +++ b/src/db/Cargo.toml @@ -19,7 +19,7 @@ required-features = ["cli"] [dependencies] err-derive = "0.3" hexdump = "0.1" -log = "0.4" +tracing = "0.1.30" heed = "0.11" rusqlite = { version = "0.27", features = ["bundled"] } diff --git a/src/db/sqlite_adapter.rs b/src/db/sqlite_adapter.rs index 68d96ca0..97a78b07 100644 --- a/src/db/sqlite_adapter.rs +++ b/src/db/sqlite_adapter.rs @@ -6,7 +6,7 @@ use std::pin::Pin; use std::ptr::NonNull; use std::sync::{Arc, Mutex, MutexGuard}; -use log::trace; +use tracing::trace; use rusqlite::{params, Connection, Rows, Statement, Transaction}; diff --git a/src/util/Cargo.toml b/src/util/Cargo.toml index 5d073436..57c70ffb 100644 --- a/src/util/Cargo.toml +++ b/src/util/Cargo.toml @@ -16,6 +16,7 @@ path = "lib.rs" [dependencies] garage_db = { version = "0.8.0", path = "../db" } +async-trait = "0.1" blake2 = "0.9" err-derive = "0.3" xxhash-rust = { version = "0.8", default-features = false, features = ["xxh3"] } diff --git a/src/util/background.rs b/src/util/background.rs deleted file mode 100644 index d35425f5..00000000 --- a/src/util/background.rs +++ /dev/null @@ -1,160 +0,0 @@ -//! Job runner for futures and async functions -use core::future::Future; -use std::pin::Pin; -use std::sync::Arc; -use std::time::Duration; - -use futures::future::*; -use futures::select; -use futures::stream::FuturesUnordered; -use futures::StreamExt; -use tokio::sync::{mpsc, mpsc::error::TryRecvError, watch, Mutex}; - -use crate::error::Error; - -type JobOutput = Result<(), Error>; -type Job = Pin + Send>>; - -/// Job runner for futures and async functions -pub struct BackgroundRunner { - stop_signal: watch::Receiver, - queue_in: mpsc::UnboundedSender<(Job, bool)>, - worker_in: mpsc::UnboundedSender>, -} - -impl BackgroundRunner { - /// Create a new BackgroundRunner - pub fn new( - n_runners: usize, - stop_signal: watch::Receiver, - ) -> (Arc, tokio::task::JoinHandle<()>) { - let (worker_in, mut worker_out) = mpsc::unbounded_channel(); - - let stop_signal_2 = stop_signal.clone(); - let await_all_done = tokio::spawn(async move { - let mut workers = FuturesUnordered::new(); - let mut shutdown_timer = 0; - loop { - let closed = match worker_out.try_recv() { - Ok(wkr) => { - workers.push(wkr); - false - } - Err(TryRecvError::Empty) => false, - Err(TryRecvError::Disconnected) => true, - }; - select! { - res = workers.next() => { - if let Some(Err(e)) = res { - error!("Worker exited with error: {}", e); - } - } - _ = tokio::time::sleep(Duration::from_secs(1)).fuse() => { - if closed || *stop_signal_2.borrow() { - shutdown_timer += 1; - if shutdown_timer >= 10 { - break; - } - } - } - } - } - }); - - let (queue_in, queue_out) = mpsc::unbounded_channel(); - let queue_out = Arc::new(Mutex::new(queue_out)); - - for i in 0..n_runners { - let queue_out = queue_out.clone(); - let stop_signal = stop_signal.clone(); - - worker_in - .send(tokio::spawn(async move { - loop { - let (job, cancellable) = { - select! { - item = wait_job(&queue_out).fuse() => match item { - // We received a task, process it - Some(x) => x, - // We received a signal that no more tasks will ever be sent - // because the sending side was dropped. Exit now. - None => break, - }, - _ = tokio::time::sleep(Duration::from_secs(5)).fuse() => { - if *stop_signal.borrow() { - // Nothing has been going on for 5 secs, and we are shutting - // down. Exit now. - break; - } else { - // Nothing is going on but we don't want to exit. - continue; - } - } - } - }; - if cancellable && *stop_signal.borrow() { - continue; - } - if let Err(e) = job.await { - error!("Job failed: {}", e) - } - } - info!("Background worker {} exiting", i); - })) - .unwrap(); - } - - let bgrunner = Arc::new(Self { - stop_signal, - queue_in, - worker_in, - }); - (bgrunner, await_all_done) - } - - /// Spawn a task to be run in background - pub fn spawn(&self, job: T) - where - T: Future + Send + 'static, - { - let boxed: Job = Box::pin(job); - self.queue_in - .send((boxed, false)) - .map_err(|_| "could not put job in queue") - .unwrap(); - } - - /// Spawn a task to be run in background. It may get discarded before running if spawned while - /// the runner is stopping - pub fn spawn_cancellable(&self, job: T) - where - T: Future + Send + 'static, - { - let boxed: Job = Box::pin(job); - self.queue_in - .send((boxed, true)) - .map_err(|_| "could not put job in queue") - .unwrap(); - } - - pub fn spawn_worker(&self, name: String, worker: F) - where - F: FnOnce(watch::Receiver) -> T + Send + 'static, - T: Future + Send + 'static, - { - let stop_signal = self.stop_signal.clone(); - let task = tokio::spawn(async move { - info!("Worker started: {}", name); - worker(stop_signal).await; - info!("Worker exited: {}", name); - }); - self.worker_in - .send(task) - .map_err(|_| "could not put job in queue") - .unwrap(); - } -} - -async fn wait_job(q: &Mutex>) -> Option<(Job, bool)> { - q.lock().await.recv().await -} diff --git a/src/util/background/job_worker.rs b/src/util/background/job_worker.rs new file mode 100644 index 00000000..8cc660f8 --- /dev/null +++ b/src/util/background/job_worker.rs @@ -0,0 +1,52 @@ +//! Job worker: a generic worker that just processes incoming +//! jobs one by one + +use std::sync::Arc; + +use async_trait::async_trait; +use tokio::sync::{mpsc, Mutex}; + +use crate::background::worker::*; +use crate::background::*; + +pub(crate) struct JobWorker { + pub(crate) index: usize, + pub(crate) job_chan: Arc>>, + pub(crate) next_job: Option, +} + +#[async_trait] +impl Worker for JobWorker { + fn name(&self) -> String { + format!("Job worker #{}", self.index) + } + + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + match self.next_job.take() { + None => return Ok(WorkerStatus::Idle), + Some(job) => { + job.await?; + Ok(WorkerStatus::Busy) + } + } + } + + async fn wait_for_work(&mut self, must_exit: &mut watch::Receiver) -> WorkerStatus { + loop { + match self.job_chan.lock().await.recv().await { + Some((job, cancellable)) => { + if cancellable && *must_exit.borrow() { + // skip job + continue; + } + self.next_job = Some(job); + return WorkerStatus::Busy + } + None => return WorkerStatus::Done, + } + } + } +} diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs new file mode 100644 index 00000000..97d25784 --- /dev/null +++ b/src/util/background/mod.rs @@ -0,0 +1,91 @@ +//! Job runner for futures and async functions + +pub mod job_worker; +pub mod worker; + +use core::future::Future; +use std::pin::Pin; +use std::sync::Arc; + +use tokio::sync::{mpsc, watch, Mutex}; + +use crate::error::Error; +use worker::{Worker, WorkerProcessor}; + +pub(crate) type JobOutput = Result<(), Error>; +pub(crate) type Job = Pin + Send>>; + +/// Job runner for futures and async functions +pub struct BackgroundRunner { + send_job: mpsc::UnboundedSender<(Job, bool)>, + send_worker: mpsc::UnboundedSender>, +} + +impl BackgroundRunner { + /// Create a new BackgroundRunner + pub fn new( + n_runners: usize, + stop_signal: watch::Receiver, + ) -> (Arc, tokio::task::JoinHandle<()>) { + let (send_worker, worker_out) = mpsc::unbounded_channel::>(); + + let await_all_done = + tokio::spawn( + async move { WorkerProcessor::new(worker_out, stop_signal).run().await }, + ); + + let (send_job, queue_out) = mpsc::unbounded_channel(); + let queue_out = Arc::new(Mutex::new(queue_out)); + + for i in 0..n_runners { + let queue_out = queue_out.clone(); + + send_worker.send(Box::new(job_worker::JobWorker { + index: i, + job_chan: queue_out.clone(), + next_job: None, + })).ok().unwrap(); + } + + let bgrunner = Arc::new(Self { + send_job, + send_worker, + }); + (bgrunner, await_all_done) + } + + /// Spawn a task to be run in background + pub fn spawn(&self, job: T) + where + T: Future + Send + 'static, + { + let boxed: Job = Box::pin(job); + self.send_job + .send((boxed, false)) + .ok() + .expect("Could not put job in queue"); + } + + /// Spawn a task to be run in background. It may get discarded before running if spawned while + /// the runner is stopping + pub fn spawn_cancellable(&self, job: T) + where + T: Future + Send + 'static, + { + let boxed: Job = Box::pin(job); + self.send_job + .send((boxed, true)) + .ok() + .expect("Could not put job in queue"); + } + + pub fn spawn_worker(&self, worker: W) + where + W: Worker + 'static, + { + self.send_worker + .send(Box::new(worker)) + .ok() + .expect("Could not put worker in queue"); + } +} diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs new file mode 100644 index 00000000..a173902c --- /dev/null +++ b/src/util/background/worker.rs @@ -0,0 +1,145 @@ +use std::time::{Duration, Instant}; + +use tracing::*; +use async_trait::async_trait; +use futures::future::*; +use tokio::select; +use futures::stream::FuturesUnordered; +use futures::StreamExt; +use tokio::sync::{mpsc, watch}; + +use crate::error::Error; + +#[derive(PartialEq, Copy, Clone)] +pub enum WorkerStatus { + Busy, + Idle, + Done, +} + +#[async_trait] +pub trait Worker: Send { + fn name(&self) -> String; + async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result; + async fn wait_for_work(&mut self, must_exit: &mut watch::Receiver) -> WorkerStatus; +} + +pub(crate) struct WorkerProcessor { + stop_signal: watch::Receiver, + worker_chan: mpsc::UnboundedReceiver>, +} + +impl WorkerProcessor { + pub(crate) fn new( + worker_chan: mpsc::UnboundedReceiver>, + stop_signal: watch::Receiver, + ) -> Self { + Self { + stop_signal, + worker_chan, + } + } + + pub(crate) async fn run(&mut self) { + let mut workers = FuturesUnordered::new(); + let mut next_task_id = 1; + + while !*self.stop_signal.borrow() { + let await_next_worker = async { + if workers.is_empty() { + futures::future::pending().await + } else { + workers.next().await + } + }; + select! { + new_worker_opt = self.worker_chan.recv() => { + if let Some(new_worker) = new_worker_opt { + let task_id = next_task_id; + next_task_id += 1; + let stop_signal = self.stop_signal.clone(); + workers.push(async move { + let mut worker = WorkerHandler { + task_id, + stop_signal, + worker: new_worker, + status: WorkerStatus::Busy, + }; + worker.step().await; + worker + }.boxed()); + } + } + worker = await_next_worker => { + if let Some(mut worker) = worker { + // TODO save new worker status somewhere + if worker.status == WorkerStatus::Done { + info!("Worker {} (TID {}) exited", worker.worker.name(), worker.task_id); + } else { + workers.push(async move { + worker.step().await; + worker + }.boxed()); + } + } + } + _ = self.stop_signal.changed() => (), + } + } + + // We are exiting, drain everything + let drain_half_time = Instant::now() + Duration::from_secs(5); + let drain_everything = async move { + while let Some(mut worker) = workers.next().await { + if worker.status == WorkerStatus::Busy + || (worker.status == WorkerStatus::Idle && Instant::now() < drain_half_time) + { + workers.push(async move { + worker.step().await; + worker + }.boxed()); + } else { + info!("Worker {} (TID {}) exited", worker.worker.name(), worker.task_id); + } + } + }; + + select! { + _ = drain_everything => { + info!("All workers exited in time \\o/"); + } + _ = tokio::time::sleep(Duration::from_secs(9)) => { + warn!("Some workers could not exit in time, we are cancelling some things in the middle."); + } + } + } +} + +// TODO add tranquilizer +struct WorkerHandler { + task_id: usize, + stop_signal: watch::Receiver, + worker: Box, + status: WorkerStatus, +} + +impl WorkerHandler { + async fn step(&mut self) { + match self.status { + WorkerStatus::Busy => { + match self.worker.work(&mut self.stop_signal).await { + Ok(s) => { + self.status = s; + } + Err(e) => { + error!("Error in worker {}: {}", self.worker.name(), e); + } + } + } + WorkerStatus::Idle => { + self.status = self.worker.wait_for_work(&mut self.stop_signal).await; + } + WorkerStatus::Done => unreachable!() + } + } +} diff --git a/src/util/lib.rs b/src/util/lib.rs index 8ca6e310..fce151af 100644 --- a/src/util/lib.rs +++ b/src/util/lib.rs @@ -11,7 +11,6 @@ pub mod error; pub mod formater; pub mod metrics; pub mod persister; -//pub mod sled_counter; pub mod time; pub mod token_bucket; pub mod tranquilizer; -- 2.43.0 From 3119ea59b08e62ce14cddeb4809a397785b662bb Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 13:50:55 +0200 Subject: [PATCH 02/33] New worker semantics applied to garage_table --- src/table/gc.rs | 80 ++++++++----- src/table/merkle.rs | 67 +++++------ src/table/sync.rs | 187 +++++++++++++----------------- src/util/background/job_worker.rs | 5 +- src/util/background/mod.rs | 20 ++-- src/util/background/worker.rs | 91 +++++++++++---- 6 files changed, 248 insertions(+), 202 deletions(-) diff --git a/src/table/gc.rs b/src/table/gc.rs index e7fbbcb0..36124c2f 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -8,12 +8,11 @@ use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; use futures::future::join_all; -use futures::select; -use futures_util::future::*; use tokio::sync::watch; use garage_db::counted_tree_hack::CountedTree; +use garage_util::background::*; use garage_util::data::*; use garage_util::error::*; use garage_util::time::*; @@ -69,35 +68,11 @@ where gc.endpoint.set_handler(gc.clone()); - let gc1 = gc.clone(); - system.background.spawn_worker( - format!("GC loop for {}", F::TABLE_NAME), - move |must_exit: watch::Receiver| gc1.gc_loop(must_exit), - ); + system.background.spawn_worker(GcWorker::new(gc.clone())); gc } - async fn gc_loop(self: Arc, mut must_exit: watch::Receiver) { - while !*must_exit.borrow() { - match self.gc_loop_iter().await { - Ok(None) => { - // Stuff was done, loop immediately - } - Ok(Some(wait_delay)) => { - // Nothing was done, wait specified delay. - select! { - _ = tokio::time::sleep(wait_delay).fuse() => {}, - _ = must_exit.changed().fuse() => {}, - } - } - Err(e) => { - warn!("({}) Error doing GC: {}", F::TABLE_NAME, e); - } - } - } - } - async fn gc_loop_iter(&self) -> Result, Error> { let now = now_msec(); @@ -328,6 +303,57 @@ where } } +struct GcWorker +where + F: TableSchema + 'static, + R: TableReplication + 'static, +{ + gc: Arc>, + wait_delay: Duration, +} + +impl GcWorker +where + F: TableSchema + 'static, + R: TableReplication + 'static, +{ + fn new(gc: Arc>) -> Self { + Self { + gc, + wait_delay: Duration::from_secs(0), + } + } +} + +#[async_trait] +impl Worker for GcWorker +where + F: TableSchema + 'static, + R: TableReplication + 'static, +{ + fn name(&self) -> String { + format!("Table GC: {}", F::TABLE_NAME) + } + + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + match self.gc.gc_loop_iter().await? { + None => Ok(WorkerStatus::Busy), + Some(delay) => { + self.wait_delay = delay; + Ok(WorkerStatus::Idle) + } + } + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + tokio::time::sleep(self.wait_delay).await; + WorkerStatus::Busy + } +} + /// An entry stored in the gc_todo Sled tree associated with the table /// Contains helper function for parsing, saving, and removing /// such entry in Sled diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 7685b193..d4d2717f 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -1,14 +1,13 @@ use std::sync::Arc; use std::time::Duration; -use futures::select; -use futures_util::future::*; +use async_trait::async_trait; use serde::{Deserialize, Serialize}; use tokio::sync::watch; use garage_db as db; -use garage_util::background::BackgroundRunner; +use garage_util::background::*; use garage_util::data::*; use garage_util::error::Error; @@ -78,43 +77,17 @@ where empty_node_hash, }); - let ret2 = ret.clone(); - background.spawn_worker( - format!("Merkle tree updater for {}", F::TABLE_NAME), - |must_exit: watch::Receiver| ret2.updater_loop(must_exit), - ); + background.spawn_worker(MerkleWorker(ret.clone())); ret } - async fn updater_loop(self: Arc, mut must_exit: watch::Receiver) { - while !*must_exit.borrow() { - match self.updater_loop_iter() { - Ok(true) => (), - Ok(false) => { - select! { - _ = self.data.merkle_todo_notify.notified().fuse() => {}, - _ = must_exit.changed().fuse() => {}, - } - } - Err(e) => { - warn!( - "({}) Error while updating Merkle tree item: {}", - F::TABLE_NAME, - e - ); - tokio::time::sleep(Duration::from_secs(10)).await; - } - } - } - } - - fn updater_loop_iter(&self) -> Result { + fn updater_loop_iter(&self) -> Result { if let Some((key, valhash)) = self.data.merkle_todo.first()? { self.update_item(&key, &valhash)?; - Ok(true) + Ok(WorkerStatus::Busy) } else { - Ok(false) + Ok(WorkerStatus::Idle) } } @@ -325,6 +298,34 @@ where } } +struct MerkleWorker(Arc>) +where + F: TableSchema + 'static, + R: TableReplication + 'static; + +#[async_trait] +impl Worker for MerkleWorker +where + F: TableSchema + 'static, + R: TableReplication + 'static, +{ + fn name(&self) -> String { + format!("Merkle tree updater: {}", F::TABLE_NAME) + } + + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + self.0.updater_loop_iter() + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + tokio::time::sleep(Duration::from_secs(10)).await; + WorkerStatus::Busy + } +} + impl MerkleNodeKey { fn encode(&self) -> Vec { let mut ret = Vec::with_capacity(2 + self.prefix.len()); diff --git a/src/table/sync.rs b/src/table/sync.rs index 4c83e991..be081d96 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -1,17 +1,17 @@ use std::collections::VecDeque; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; use std::time::{Duration, Instant}; use async_trait::async_trait; -use futures::select; -use futures_util::future::*; use futures_util::stream::*; use opentelemetry::KeyValue; use rand::Rng; use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; +use tokio::select; use tokio::sync::{mpsc, watch}; +use garage_util::background::*; use garage_util::data::*; use garage_util::error::Error; @@ -34,7 +34,7 @@ pub struct TableSyncer data: Arc>, merkle: Arc>, - todo: Mutex, + add_full_sync_tx: mpsc::UnboundedSender<()>, endpoint: Arc>, } @@ -52,10 +52,6 @@ impl Rpc for SyncRpc { type Response = Result; } -struct SyncTodo { - todo: Vec, -} - #[derive(Debug, Clone)] struct TodoPartition { partition: Partition, @@ -80,118 +76,40 @@ where .netapp .endpoint(format!("garage_table/sync.rs/Rpc:{}", F::TABLE_NAME)); - let todo = SyncTodo { todo: vec![] }; + let (add_full_sync_tx, add_full_sync_rx) = mpsc::unbounded_channel(); let syncer = Arc::new(Self { system: system.clone(), data, merkle, - todo: Mutex::new(todo), + add_full_sync_tx, endpoint, }); syncer.endpoint.set_handler(syncer.clone()); - let (busy_tx, busy_rx) = mpsc::unbounded_channel(); - - let s1 = syncer.clone(); - system.background.spawn_worker( - format!("table sync watcher for {}", F::TABLE_NAME), - move |must_exit: watch::Receiver| s1.watcher_task(must_exit, busy_rx), - ); - - let s2 = syncer.clone(); - system.background.spawn_worker( - format!("table syncer for {}", F::TABLE_NAME), - move |must_exit: watch::Receiver| s2.syncer_task(must_exit, busy_tx), - ); - - let s3 = syncer.clone(); - tokio::spawn(async move { - tokio::time::sleep(Duration::from_secs(20)).await; - s3.add_full_sync(); + system.background.spawn_worker(SyncWorker { + syncer: syncer.clone(), + ring_recv: system.ring.clone(), + ring: system.ring.borrow().clone(), + add_full_sync_rx, + todo: vec![], + next_full_sync: Instant::now() + Duration::from_secs(20), }); syncer } - async fn watcher_task( - self: Arc, - mut must_exit: watch::Receiver, - mut busy_rx: mpsc::UnboundedReceiver, - ) { - let mut prev_ring: Arc = self.system.ring.borrow().clone(); - let mut ring_recv: watch::Receiver> = self.system.ring.clone(); - let mut nothing_to_do_since = Some(Instant::now()); - - while !*must_exit.borrow() { - select! { - _ = ring_recv.changed().fuse() => { - let new_ring = ring_recv.borrow(); - if !Arc::ptr_eq(&new_ring, &prev_ring) { - debug!("({}) Ring changed, adding full sync to syncer todo list", F::TABLE_NAME); - self.add_full_sync(); - prev_ring = new_ring.clone(); - } - } - busy_opt = busy_rx.recv().fuse() => { - if let Some(busy) = busy_opt { - if busy { - nothing_to_do_since = None; - } else if nothing_to_do_since.is_none() { - nothing_to_do_since = Some(Instant::now()); - } - } - } - _ = must_exit.changed().fuse() => {}, - _ = tokio::time::sleep(Duration::from_secs(1)).fuse() => { - if nothing_to_do_since.map(|t| Instant::now() - t >= ANTI_ENTROPY_INTERVAL).unwrap_or(false) { - nothing_to_do_since = None; - debug!("({}) Interval passed, adding full sync to syncer todo list", F::TABLE_NAME); - self.add_full_sync(); - } - } - } - } - } - pub fn add_full_sync(&self) { - self.todo - .lock() - .unwrap() - .add_full_sync(&self.data, &self.system); - } - - async fn syncer_task( - self: Arc, - mut must_exit: watch::Receiver, - busy_tx: mpsc::UnboundedSender, - ) { - while !*must_exit.borrow() { - let task = self.todo.lock().unwrap().pop_task(); - if let Some(partition) = task { - busy_tx.send(true).unwrap(); - let res = self - .clone() - .sync_partition(&partition, &mut must_exit) - .await; - if let Err(e) = res { - warn!( - "({}) Error while syncing {:?}: {}", - F::TABLE_NAME, - partition, - e - ); - } - } else { - busy_tx.send(false).unwrap(); - tokio::time::sleep(Duration::from_secs(1)).await; - } + if self.add_full_sync_tx.send(()).is_err() { + error!("({}) Could not add full sync", F::TABLE_NAME); } } + // ---- + async fn sync_partition( - self: Arc, + self: &Arc, partition: &TodoPartition, must_exit: &mut watch::Receiver, ) -> Result<(), Error> { @@ -577,12 +495,22 @@ where } } -impl SyncTodo { - fn add_full_sync( - &mut self, - data: &TableData, - system: &System, - ) { +// -------- Sync Worker --------- + +struct SyncWorker { + syncer: Arc>, + ring_recv: watch::Receiver>, + ring: Arc, + add_full_sync_rx: mpsc::UnboundedReceiver<()>, + todo: Vec, + next_full_sync: Instant, +} + +impl SyncWorker { + fn add_full_sync(&mut self) { + let system = &self.syncer.system; + let data = &self.syncer.data; + let my_id = system.id; self.todo.clear(); @@ -623,6 +551,8 @@ impl SyncTodo { retain, }); } + + self.next_full_sync = Instant::now() + ANTI_ENTROPY_INTERVAL; } fn pop_task(&mut self) -> Option { @@ -641,6 +571,51 @@ impl SyncTodo { } } +#[async_trait] +impl Worker for SyncWorker { + fn name(&self) -> String { + format!("Table sync worker for {}", F::TABLE_NAME) + } + + async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { + if let Some(partition) = self.pop_task() { + self.syncer.sync_partition(&partition, must_exit).await?; + Ok(WorkerStatus::Busy) + } else { + Ok(WorkerStatus::Idle) + } + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + select! { + s = self.add_full_sync_rx.recv() => match s { + Some(()) => { + self.add_full_sync(); + } + None => (), + }, + _ = self.ring_recv.changed() => { + let new_ring = self.ring_recv.borrow(); + if !Arc::ptr_eq(&new_ring, &self.ring) { + self.ring = new_ring.clone(); + drop(new_ring); + debug!("({}) Ring changed, adding full sync to syncer todo list", F::TABLE_NAME); + self.add_full_sync(); + } + }, + _ = tokio::time::sleep(self.next_full_sync - Instant::now()) => { + self.add_full_sync(); + } + } + match self.todo.is_empty() { + false => WorkerStatus::Busy, + true => WorkerStatus::Idle, + } + } +} + +// ---- UTIL ---- + fn hash_of(x: &T) -> Result { Ok(blake2sum(&rmp_to_vec_all_named(x)?[..])) } diff --git a/src/util/background/job_worker.rs b/src/util/background/job_worker.rs index 8cc660f8..fcdac582 100644 --- a/src/util/background/job_worker.rs +++ b/src/util/background/job_worker.rs @@ -34,16 +34,15 @@ impl Worker for JobWorker { } } - async fn wait_for_work(&mut self, must_exit: &mut watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { loop { match self.job_chan.lock().await.recv().await { Some((job, cancellable)) => { if cancellable && *must_exit.borrow() { - // skip job continue; } self.next_job = Some(job); - return WorkerStatus::Busy + return WorkerStatus::Busy; } None => return WorkerStatus::Done, } diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index 97d25784..c06e2225 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -10,7 +10,8 @@ use std::sync::Arc; use tokio::sync::{mpsc, watch, Mutex}; use crate::error::Error; -use worker::{Worker, WorkerProcessor}; +use worker::WorkerProcessor; +pub use worker::{Worker, WorkerStatus}; pub(crate) type JobOutput = Result<(), Error>; pub(crate) type Job = Pin + Send>>; @@ -30,9 +31,7 @@ impl BackgroundRunner { let (send_worker, worker_out) = mpsc::unbounded_channel::>(); let await_all_done = - tokio::spawn( - async move { WorkerProcessor::new(worker_out, stop_signal).run().await }, - ); + tokio::spawn(async move { WorkerProcessor::new(worker_out, stop_signal).run().await }); let (send_job, queue_out) = mpsc::unbounded_channel(); let queue_out = Arc::new(Mutex::new(queue_out)); @@ -40,11 +39,14 @@ impl BackgroundRunner { for i in 0..n_runners { let queue_out = queue_out.clone(); - send_worker.send(Box::new(job_worker::JobWorker { - index: i, - job_chan: queue_out.clone(), - next_job: None, - })).ok().unwrap(); + send_worker + .send(Box::new(job_worker::JobWorker { + index: i, + job_chan: queue_out.clone(), + next_job: None, + })) + .ok() + .unwrap(); } let bgrunner = Arc::new(Self { diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index a173902c..92f7990c 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -1,16 +1,16 @@ use std::time::{Duration, Instant}; -use tracing::*; use async_trait::async_trait; use futures::future::*; -use tokio::select; use futures::stream::FuturesUnordered; use futures::StreamExt; +use tokio::select; use tokio::sync::{mpsc, watch}; +use tracing::*; use crate::error::Error; -#[derive(PartialEq, Copy, Clone)] +#[derive(PartialEq, Copy, Clone, Debug)] pub enum WorkerStatus { Busy, Idle, @@ -20,8 +20,20 @@ pub enum WorkerStatus { #[async_trait] pub trait Worker: Send { fn name(&self) -> String; + + /// Work: do a basic unit of work, if one is available (otherwise, should return + /// WorkerStatus::Idle immediately). We will do our best to not interrupt this future in the + /// middle of processing, it will only be interrupted at the last minute when Garage is trying + /// to exit and this hasn't returned yet. This function may return an error to indicate that + /// its unit of work could not be processed due to an error: the error will be logged and + /// .work() will be called again immediately. async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result; - async fn wait_for_work(&mut self, must_exit: &mut watch::Receiver) -> WorkerStatus; + + /// Wait for work: await for some task to become available. This future can be interrupted in + /// the middle for any reason. This future doesn't have to await on must_exit.changed(), we + /// are doing it for you. Therefore it only receives a read refernce to must_exit which allows + /// it to check if we are exiting. + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus; } pub(crate) struct WorkerProcessor { @@ -58,10 +70,12 @@ impl WorkerProcessor { let task_id = next_task_id; next_task_id += 1; let stop_signal = self.stop_signal.clone(); + let stop_signal_worker = self.stop_signal.clone(); workers.push(async move { let mut worker = WorkerHandler { task_id, stop_signal, + stop_signal_worker, worker: new_worker, status: WorkerStatus::Busy, }; @@ -91,15 +105,22 @@ impl WorkerProcessor { let drain_half_time = Instant::now() + Duration::from_secs(5); let drain_everything = async move { while let Some(mut worker) = workers.next().await { - if worker.status == WorkerStatus::Busy - || (worker.status == WorkerStatus::Idle && Instant::now() < drain_half_time) - { - workers.push(async move { - worker.step().await; - worker - }.boxed()); + if worker.status == WorkerStatus::Done { + info!( + "Worker {} (TID {}) exited", + worker.worker.name(), + worker.task_id + ); + } else if Instant::now() > drain_half_time { + warn!("Worker {} (TID {}) interrupted between two iterations in state {:?} (this should be fine)", worker.worker.name(), worker.task_id, worker.status); } else { - info!("Worker {} (TID {}) exited", worker.worker.name(), worker.task_id); + workers.push( + async move { + worker.step().await; + worker + } + .boxed(), + ); } } }; @@ -109,7 +130,7 @@ impl WorkerProcessor { info!("All workers exited in time \\o/"); } _ = tokio::time::sleep(Duration::from_secs(9)) => { - warn!("Some workers could not exit in time, we are cancelling some things in the middle."); + error!("Some workers could not exit in time, we are cancelling some things in the middle"); } } } @@ -119,27 +140,49 @@ impl WorkerProcessor { struct WorkerHandler { task_id: usize, stop_signal: watch::Receiver, + stop_signal_worker: watch::Receiver, worker: Box, status: WorkerStatus, } impl WorkerHandler { - async fn step(&mut self) { + async fn step(&mut self) { match self.status { - WorkerStatus::Busy => { - match self.worker.work(&mut self.stop_signal).await { - Ok(s) => { - self.status = s; + WorkerStatus::Busy => match self.worker.work(&mut self.stop_signal).await { + Ok(s) => { + self.status = s; + } + Err(e) => { + error!( + "Error in worker {} (TID {}): {}", + self.worker.name(), + self.task_id, + e + ); + } + }, + WorkerStatus::Idle => { + if *self.stop_signal.borrow() { + select! { + new_st = self.worker.wait_for_work(&mut self.stop_signal_worker) => { + self.status = new_st; + } + _ = tokio::time::sleep(Duration::from_secs(1)) => { + // stay in Idle state + } } - Err(e) => { - error!("Error in worker {}: {}", self.worker.name(), e); + } else { + select! { + new_st = self.worker.wait_for_work(&mut self.stop_signal_worker) => { + self.status = new_st; + } + _ = self.stop_signal.changed() => { + // stay in Idle state + } } } } - WorkerStatus::Idle => { - self.status = self.worker.wait_for_work(&mut self.stop_signal).await; - } - WorkerStatus::Done => unreachable!() + WorkerStatus::Done => unreachable!(), } } } -- 2.43.0 From 269f996fd0e7a856be43348279b5ecb3de111817 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 14:07:10 +0200 Subject: [PATCH 03/33] Block manager with new worker --- src/block/manager.rs | 124 +++++++++++++++++++++++++------------------ 1 file changed, 73 insertions(+), 51 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 32ba0431..8a131270 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -9,9 +9,9 @@ use async_trait::async_trait; use serde::{Deserialize, Serialize}; use futures::future::*; -use futures::select; use tokio::fs; use tokio::io::{AsyncReadExt, AsyncWriteExt}; +use tokio::select; use tokio::sync::{watch, Mutex, Notify}; use opentelemetry::{ @@ -22,6 +22,7 @@ use opentelemetry::{ 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; @@ -110,6 +111,12 @@ pub struct BlockManager { // it INSIDE a Mutex. struct BlockManagerLocked(); +enum BlockIterResult { + BusyDidSomething, + BusyDidNothing, + IdleFor(Duration), +} + impl BlockManager { pub fn new( db: &db::Db, @@ -557,11 +564,14 @@ impl BlockManager { fn spawn_background_worker(self: Arc) { // Launch a background workers for background resync loop processing let background = self.system.background.clone(); + let worker = BlockResyncWorker { + manager: self, + tranquilizer: Tranquilizer::new(30), + next_delay: Duration::from_secs(10), + }; tokio::spawn(async move { tokio::time::sleep(Duration::from_secs(10)).await; - background.spawn_worker("block resync worker".into(), move |must_exit| { - self.resync_loop(must_exit) - }); + background.spawn_worker(worker); }); } @@ -579,37 +589,7 @@ impl BlockManager { Ok(()) } - async fn resync_loop(self: Arc, mut must_exit: watch::Receiver) { - let mut tranquilizer = Tranquilizer::new(30); - - while !*must_exit.borrow() { - match self.resync_iter(&mut must_exit).await { - Ok(true) => { - tranquilizer.tranquilize(self.background_tranquility).await; - } - Ok(false) => { - tranquilizer.reset(); - } - 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?) - error!( - "Could not do a resync iteration: {} (this is a very bad error)", - e - ); - tranquilizer.reset(); - } - } - } - } - - // The result of resync_iter is: - // - Ok(true) -> a block was processed (successfully or not) - // - Ok(false) -> no block was processed, but we are ready for the next iteration - // - Err(_) -> a Sled error occurred when reading/writing from resync_queue/resync_errors - async fn resync_iter(&self, must_exit: &mut watch::Receiver) -> Result { + 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(); @@ -629,7 +609,7 @@ impl BlockManager { // (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(false); + return Ok(BlockIterResult::BusyDidNothing); } } @@ -676,15 +656,11 @@ impl BlockManager { self.resync_queue.remove(time_bytes)?; } - Ok(true) + Ok(BlockIterResult::BusyDidSomething) } else { - let delay = tokio::time::sleep(Duration::from_millis(time_msec - now)); - select! { - _ = delay.fuse() => {}, - _ = self.resync_notify.notified().fuse() => {}, - _ = must_exit.changed().fuse() => {}, - } - Ok(false) + Ok(BlockIterResult::IdleFor(Duration::from_millis( + time_msec - now, + ))) } } else { // Here we wait either for a notification that an item has been @@ -693,13 +669,7 @@ impl BlockManager { // 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. - let delay = tokio::time::sleep(Duration::from_secs(10)); - select! { - _ = delay.fuse() => {}, - _ = self.resync_notify.notified().fuse() => {}, - _ = must_exit.changed().fuse() => {}, - } - Ok(false) + Ok(BlockIterResult::IdleFor(Duration::from_secs(10))) } } @@ -898,6 +868,58 @@ impl EndpointHandler for BlockManager { } } +struct BlockResyncWorker { + manager: Arc, + tranquilizer: Tranquilizer, + next_delay: Duration, +} + +#[async_trait] +impl Worker for BlockResyncWorker { + fn name(&self) -> String { + "Block resync worker".into() + } + + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + self.tranquilizer.reset(); + match self.manager.resync_iter().await { + Ok(BlockIterResult::BusyDidSomething) => { + self.tranquilizer + .tranquilize(self.manager.background_tranquility) + .await; + Ok(WorkerStatus::Busy) + } + Ok(BlockIterResult::BusyDidNothing) => Ok(WorkerStatus::Busy), + Ok(BlockIterResult::IdleFor(delay)) => { + self.next_delay = delay; + Ok(WorkerStatus::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?) + error!( + "Could not do a resync iteration: {} (this is a very bad error)", + e + ); + Err(e.into()) + } + } + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + select! { + _ = tokio::time::sleep(self.next_delay) => (), + _ = self.manager.resync_notify.notified() => (), + }; + WorkerStatus::Busy + } +} + struct BlockStatus { exists: bool, needed: RcEntry, -- 2.43.0 From b8338dea56c3e6b2e88dfe0d593a5408bb236962 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 14:23:54 +0200 Subject: [PATCH 04/33] New worker for index counter propagator --- src/model/index_counter.rs | 156 +++++++++++++++++++++---------------- 1 file changed, 89 insertions(+), 67 deletions(-) diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index 36e8172b..4318a064 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -4,6 +4,7 @@ use std::marker::PhantomData; use std::sync::Arc; use std::time::Duration; +use async_trait::async_trait; use serde::{Deserialize, Serialize}; use tokio::sync::{mpsc, watch}; @@ -11,6 +12,7 @@ use garage_db as db; use garage_rpc::ring::Ring; use garage_rpc::system::System; +use garage_util::background::*; use garage_util::data::*; use garage_util::error::*; use garage_util::time::*; @@ -171,11 +173,13 @@ impl IndexCounter { ), }); - let this2 = this.clone(); - background.spawn_worker( - format!("{} index counter propagator", T::COUNTER_TABLE_NAME), - move |must_exit| this2.clone().propagate_loop(propagate_rx, must_exit), - ); + background.spawn_worker(IndexPropagatorWorker { + index_counter: this.clone(), + propagate_rx, + buf: HashMap::new(), + errors: 0, + }); + this } @@ -239,68 +243,6 @@ impl IndexCounter { Ok(()) } - async fn propagate_loop( - self: Arc, - mut propagate_rx: mpsc::UnboundedReceiver<(T::CP, T::CS, LocalCounterEntry)>, - must_exit: watch::Receiver, - ) { - // This loop batches updates to counters to be sent all at once. - // They are sent once the propagate_rx channel has been emptied (or is closed). - let mut buf = HashMap::new(); - let mut errors = 0; - - loop { - let (ent, closed) = match propagate_rx.try_recv() { - Ok(ent) => (Some(ent), false), - Err(mpsc::error::TryRecvError::Empty) if buf.is_empty() => { - match propagate_rx.recv().await { - Some(ent) => (Some(ent), false), - None => (None, true), - } - } - Err(mpsc::error::TryRecvError::Empty) => (None, false), - Err(mpsc::error::TryRecvError::Disconnected) => (None, true), - }; - - if let Some((pk, sk, counters)) = ent { - let tree_key = self.table.data.tree_key(&pk, &sk); - let dist_entry = counters.into_counter_entry(self.this_node); - match buf.entry(tree_key) { - hash_map::Entry::Vacant(e) => { - e.insert(dist_entry); - } - hash_map::Entry::Occupied(mut e) => { - e.get_mut().merge(&dist_entry); - } - } - // As long as we can add entries, loop back and add them to batch - // before sending batch to other nodes - continue; - } - - if !buf.is_empty() { - let entries = buf.iter().map(|(_k, v)| v); - if let Err(e) = self.table.insert_many(entries).await { - errors += 1; - if errors >= 2 && *must_exit.borrow() { - error!("({}) Could not propagate {} counter values: {}, these counters will not be updated correctly.", T::COUNTER_TABLE_NAME, buf.len(), e); - break; - } - warn!("({}) Could not propagate {} counter values: {}, retrying in 5 seconds (retry #{})", T::COUNTER_TABLE_NAME, buf.len(), e, errors); - tokio::time::sleep(Duration::from_secs(5)).await; - continue; - } - - buf.clear(); - errors = 0; - } - - if closed || *must_exit.borrow() { - break; - } - } - } - pub fn offline_recount_all( &self, counted_table: &Arc>, @@ -437,6 +379,86 @@ impl IndexCounter { } } +struct IndexPropagatorWorker { + index_counter: Arc>, + propagate_rx: mpsc::UnboundedReceiver<(T::CP, T::CS, LocalCounterEntry)>, + + buf: HashMap, CounterEntry>, + errors: usize, +} + +impl IndexPropagatorWorker { + fn add_ent(&mut self, pk: T::CP, sk: T::CS, counters: LocalCounterEntry) { + let tree_key = self.index_counter.table.data.tree_key(&pk, &sk); + let dist_entry = counters.into_counter_entry(self.index_counter.this_node); + match self.buf.entry(tree_key) { + hash_map::Entry::Vacant(e) => { + e.insert(dist_entry); + } + hash_map::Entry::Occupied(mut e) => { + e.get_mut().merge(&dist_entry); + } + } + } +} + +#[async_trait] +impl Worker for IndexPropagatorWorker { + fn name(&self) -> String { + format!("{} index counter propagator", T::COUNTER_TABLE_NAME) + } + + async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { + // This loop batches updates to counters to be sent all at once. + // They are sent once the propagate_rx channel has been emptied (or is closed). + let closed = loop { + match self.propagate_rx.try_recv() { + Ok((pk, sk, counters)) => { + self.add_ent(pk, sk, counters); + } + Err(mpsc::error::TryRecvError::Empty) => break false, + Err(mpsc::error::TryRecvError::Disconnected) => break true, + } + }; + + if !self.buf.is_empty() { + let entries = self.buf.iter().map(|(_k, v)| v); + if let Err(e) = self.index_counter.table.insert_many(entries).await { + self.errors += 1; + if self.errors >= 2 && *must_exit.borrow() { + error!("({}) Could not propagate {} counter values: {}, these counters will not be updated correctly.", T::COUNTER_TABLE_NAME, self.buf.len(), e); + return Ok(WorkerStatus::Done); + } + warn!("({}) Could not propagate {} counter values: {}, retrying in 5 seconds (retry #{})", T::COUNTER_TABLE_NAME, self.buf.len(), e, self.errors); + tokio::time::sleep(Duration::from_secs(5)).await; + return Ok(WorkerStatus::Busy); + } else { + self.buf.clear(); + self.errors = 0; + } + + return Ok(WorkerStatus::Busy); + } else if closed { + return Ok(WorkerStatus::Done); + } else { + return Ok(WorkerStatus::Idle); + } + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + match self.propagate_rx.recv().await { + Some((pk, sk, counters)) => { + self.add_ent(pk, sk, counters); + WorkerStatus::Busy + } + None => match self.buf.is_empty() { + false => WorkerStatus::Busy, + true => WorkerStatus::Done, + }, + } + } +} + #[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] struct LocalCounterEntry { pk: T::CP, -- 2.43.0 From a855c54bdb1a6912e99a6d64ee97bc63c700f29f Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 15:27:58 +0200 Subject: [PATCH 05/33] Online repair new workers, except blocks and scrub --- src/garage/admin.rs | 12 +- src/garage/repair/online.rs | 226 ++++++++++++++++++++++-------------- 2 files changed, 142 insertions(+), 96 deletions(-) diff --git a/src/garage/admin.rs b/src/garage/admin.rs index 48914655..c9783e54 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -24,7 +24,7 @@ use garage_model::migrate::Migrate; use garage_model::permission::*; use crate::cli::*; -use crate::repair::online::OnlineRepair; +use crate::repair::online::launch_online_repair; pub const ADMIN_RPC_PATH: &str = "garage/admin_rpc.rs/Rpc"; @@ -693,15 +693,7 @@ impl AdminRpcHandler { ))) } } else { - let repair = OnlineRepair { - garage: self.garage.clone(), - }; - self.garage - .system - .background - .spawn_worker("Repair worker".into(), move |must_exit| async move { - repair.repair_worker(opt, must_exit).await - }); + launch_online_repair(self.garage.clone(), opt)?; Ok(AdminRpc::Ok(format!( "Repair launched on {:?}", self.garage.system.id diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index d6a71742..e6fcd705 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -1,5 +1,6 @@ use std::sync::Arc; +use async_trait::async_trait; use tokio::sync::watch; use garage_model::garage::Garage; @@ -7,83 +8,103 @@ use garage_model::s3::block_ref_table::*; use garage_model::s3::object_table::*; use garage_model::s3::version_table::*; use garage_table::*; +use garage_util::background::*; use garage_util::error::Error; use crate::*; -pub struct OnlineRepair { - pub garage: Arc, +pub fn launch_online_repair(garage: Arc, opt: RepairOpt) -> Result<(), Error> { + match opt.what { + RepairWhat::Tables => { + info!("Launching a full sync of tables"); + garage.bucket_table.syncer.add_full_sync(); + garage.object_table.syncer.add_full_sync(); + garage.version_table.syncer.add_full_sync(); + garage.block_ref_table.syncer.add_full_sync(); + garage.key_table.syncer.add_full_sync(); + } + RepairWhat::Versions => { + info!("Repairing the versions table"); + garage + .background + .spawn_worker(RepairVersionsWorker::new(garage.clone())); + } + RepairWhat::BlockRefs => { + info!("Repairing the block refs table"); + garage + .background + .spawn_worker(RepairBlockrefsWorker::new(garage.clone())); + } + RepairWhat::Blocks => { + unimplemented!() + /* + info!("Repairing the stored blocks"); + self.garage + .block_manager + .repair_data_store(&must_exit) + .await?; + */ + } + RepairWhat::Scrub { tranquility } => { + unimplemented!() + /* + info!("Verifying integrity of stored blocks"); + self.garage + .block_manager + .scrub_data_store(&must_exit, tranquility) + .await?; + */ + } + } + Ok(()) } -impl OnlineRepair { - pub async fn repair_worker(&self, opt: RepairOpt, must_exit: watch::Receiver) { - if let Err(e) = self.repair_worker_aux(opt, must_exit).await { - warn!("Repair worker failed with error: {}", e); +// ---- + +struct RepairVersionsWorker { + garage: Arc, + pos: Vec, + iter: usize, +} + +impl RepairVersionsWorker { + fn new(garage: Arc) -> Self { + Self { + garage, + pos: vec![], + iter: 0, } } +} - async fn repair_worker_aux( - &self, - opt: RepairOpt, - must_exit: watch::Receiver, - ) -> Result<(), Error> { - match opt.what { - RepairWhat::Tables => { - info!("Launching a full sync of tables"); - self.garage.bucket_table.syncer.add_full_sync(); - self.garage.object_table.syncer.add_full_sync(); - self.garage.version_table.syncer.add_full_sync(); - self.garage.block_ref_table.syncer.add_full_sync(); - self.garage.key_table.syncer.add_full_sync(); - } - RepairWhat::Versions => { - info!("Repairing the versions table"); - self.repair_versions(&must_exit).await?; - } - RepairWhat::BlockRefs => { - info!("Repairing the block refs table"); - self.repair_block_ref(&must_exit).await?; - } - RepairWhat::Blocks => { - info!("Repairing the stored blocks"); - self.garage - .block_manager - .repair_data_store(&must_exit) - .await?; - } - RepairWhat::Scrub { tranquility } => { - info!("Verifying integrity of stored blocks"); - self.garage - .block_manager - .scrub_data_store(&must_exit, tranquility) - .await?; - } - } - Ok(()) +#[async_trait] +impl Worker for RepairVersionsWorker { + fn name(&self) -> String { + "Version repair worker".into() } - async fn repair_versions(&self, must_exit: &watch::Receiver) -> Result<(), Error> { - let mut pos = vec![]; - let mut i = 0; - - while !*must_exit.borrow() { - let item_bytes = match self.garage.version_table.data.store.get_gt(pos)? { - Some((k, v)) => { - pos = k; - v - } - None => break, - }; - - i += 1; - if i % 1000 == 0 { - info!("repair_versions: {}", i); + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + let item_bytes = match self.garage.version_table.data.store.get_gt(&self.pos)? { + Some((k, v)) => { + self.pos = k; + v } - - let version = rmp_serde::decode::from_read_ref::<_, Version>(&item_bytes)?; - if version.deleted.get() { - continue; + None => { + info!("repair_versions: finished, done {}", self.iter); + return Ok(WorkerStatus::Done); } + }; + + self.iter += 1; + if self.iter % 1000 == 0 { + info!("repair_versions: {}", self.iter); + } + + let version = rmp_serde::decode::from_read_ref::<_, Version>(&item_bytes)?; + if !version.deleted.get() { let object = self .garage .object_table @@ -109,32 +130,61 @@ impl OnlineRepair { .await?; } } - info!("repair_versions: finished, done {}", i); - Ok(()) + + Ok(WorkerStatus::Busy) } - async fn repair_block_ref(&self, must_exit: &watch::Receiver) -> Result<(), Error> { - let mut pos = vec![]; - let mut i = 0; + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + unreachable!() + } +} - while !*must_exit.borrow() { - let item_bytes = match self.garage.block_ref_table.data.store.get_gt(pos)? { - Some((k, v)) => { - pos = k; - v - } - None => break, - }; +// ---- - i += 1; - if i % 1000 == 0 { - info!("repair_block_ref: {}", i); +struct RepairBlockrefsWorker { + garage: Arc, + pos: Vec, + iter: usize, +} + +impl RepairBlockrefsWorker { + fn new(garage: Arc) -> Self { + Self { + garage, + pos: vec![], + iter: 0, + } + } +} + +#[async_trait] +impl Worker for RepairBlockrefsWorker { + fn name(&self) -> String { + "Block refs repair worker".into() + } + + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + let item_bytes = match self.garage.block_ref_table.data.store.get_gt(&self.pos)? { + Some((k, v)) => { + self.pos = k; + v } - - let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(&item_bytes)?; - if block_ref.deleted.get() { - continue; + None => { + info!("repair_block_ref: finished, done {}", self.iter); + return Ok(WorkerStatus::Done); } + }; + + self.iter += 1; + if self.iter % 1000 == 0 { + info!("repair_block_ref: {}", self.iter); + } + + let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(&item_bytes)?; + if !block_ref.deleted.get() { let version = self .garage .version_table @@ -157,7 +207,11 @@ impl OnlineRepair { .await?; } } - info!("repair_block_ref: finished, done {}", i); - Ok(()) + + Ok(WorkerStatus::Busy) + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + unreachable!() } } -- 2.43.0 From ba1ace6cf6edcea58aa904ccf6190155a6ac7c5e Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 16:00:08 +0200 Subject: [PATCH 06/33] Block repair with new worker semantics --- Cargo.lock | 1 + src/block/lib.rs | 1 + src/block/manager.rs | 160 +------------------------- src/block/repair.rs | 204 ++++++++++++++++++++++++++++++++++ src/garage/Cargo.toml | 1 + src/garage/admin.rs | 2 +- src/garage/repair/online.rs | 49 ++++---- src/util/background/worker.rs | 3 + 8 files changed, 237 insertions(+), 184 deletions(-) create mode 100644 src/block/repair.rs diff --git a/Cargo.lock b/Cargo.lock index 486ad4a1..007e3b4e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -959,6 +959,7 @@ dependencies = [ "futures", "futures-util", "garage_api", + "garage_block", "garage_db", "garage_model 0.7.0", "garage_rpc 0.7.0", diff --git a/src/block/lib.rs b/src/block/lib.rs index dc685657..ebdb95d8 100644 --- a/src/block/lib.rs +++ b/src/block/lib.rs @@ -2,6 +2,7 @@ extern crate tracing; pub mod manager; +pub mod repair; mod block; mod metrics; diff --git a/src/block/manager.rs b/src/block/manager.rs index 8a131270..54368faf 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -1,7 +1,5 @@ -use core::ops::Bound; - use std::convert::TryInto; -use std::path::{Path, PathBuf}; +use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; @@ -94,7 +92,7 @@ pub struct BlockManager { mutation_lock: Mutex, - rc: BlockRc, + pub(crate) rc: BlockRc, resync_queue: CountedTree, resync_notify: Notify, @@ -225,90 +223,6 @@ impl BlockManager { Ok(()) } - /// Launch the repair procedure on the data store - /// - /// This will list all blocks locally present, as well as those - /// that are required because of refcount > 0, and will try - /// to fix any mismatch between the two. - pub async fn repair_data_store(&self, must_exit: &watch::Receiver) -> Result<(), Error> { - // 1. Repair blocks from RC table. - let mut next_start: Option = None; - loop { - // We have to do this complicated two-step process where we first read a bunch - // of hashes from the RC table, and then insert them in the to-resync queue, - // because of SQLite. Basically, as long as we have an iterator on a DB table, - // we can't do anything else on the DB. The naive approach (which we had previously) - // of just iterating on the RC table and inserting items one to one in the resync - // queue can't work here, it would just provoke a deadlock in the SQLite adapter code. - // This is mostly because the Rust bindings for SQLite assume a worst-case scenario - // where SQLite is not compiled in thread-safe mode, so we have to wrap everything - // in a mutex (see db/sqlite_adapter.rs and discussion in PR #322). - let mut batch_of_hashes = vec![]; - let start_bound = match next_start.as_ref() { - None => Bound::Unbounded, - Some(x) => Bound::Excluded(x.as_slice()), - }; - for entry in self - .rc - .rc - .range::<&[u8], _>((start_bound, Bound::Unbounded))? - { - let (hash, _) = entry?; - let hash = Hash::try_from(&hash[..]).unwrap(); - batch_of_hashes.push(hash); - if batch_of_hashes.len() >= 1000 { - break; - } - } - if batch_of_hashes.is_empty() { - break; - } - - for hash in batch_of_hashes.into_iter() { - self.put_to_resync(&hash, Duration::from_secs(0))?; - next_start = Some(hash) - } - - if *must_exit.borrow() { - return Ok(()); - } - } - - // 2. Repair blocks actually on disk - // Lists all blocks on disk and adds them to the resync queue. - // 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. - self.for_each_file( - (), - move |_, hash| async move { - self.put_to_resync(&hash, Duration::from_secs(0)) - .map_err(Into::into) - }, - must_exit, - ) - .await - } - - /// Verify integrity of each block on disk. Use `speed_limit` to limit the load generated by - /// this function. - pub async fn scrub_data_store( - &self, - must_exit: &watch::Receiver, - tranquility: u32, - ) -> Result<(), Error> { - let tranquilizer = Tranquilizer::new(30); - self.for_each_file( - tranquilizer, - move |mut tranquilizer, hash| async move { - let _ = self.read_block(&hash).await; - tranquilizer.tranquilize(tranquility).await; - Ok(tranquilizer) - }, - must_exit, - ) - .await - } - /// Get lenght of resync queue pub fn resync_queue_len(&self) -> Result { // This currently can't return an error because the CountedTree hack @@ -397,7 +311,7 @@ impl BlockManager { } /// Read block from disk, verifying it's integrity - async fn read_block(&self, hash: &Hash) -> Result { + pub(crate) async fn read_block(&self, hash: &Hash) -> Result { let data = self .read_block_internal(hash) .bound_record_duration(&self.metrics.block_read_duration) @@ -575,7 +489,7 @@ impl BlockManager { }); } - fn put_to_resync(&self, hash: &Hash, delay: Duration) -> db::Result<()> { + 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) } @@ -784,72 +698,6 @@ impl BlockManager { Ok(()) } - - // ---- Utility: iteration on files in the data directory ---- - - async fn for_each_file( - &self, - state: State, - mut f: F, - must_exit: &watch::Receiver, - ) -> Result<(), Error> - where - F: FnMut(State, Hash) -> Fut + Send, - Fut: Future> + Send, - State: Send, - { - self.for_each_file_rec(&self.data_dir, state, &mut f, must_exit) - .await - .map(|_| ()) - } - - fn for_each_file_rec<'a, F, Fut, State>( - &'a self, - path: &'a Path, - mut state: State, - f: &'a mut F, - must_exit: &'a watch::Receiver, - ) -> BoxFuture<'a, Result> - where - F: FnMut(State, Hash) -> Fut + Send, - Fut: Future> + Send, - State: Send + 'a, - { - async move { - let mut ls_data_dir = fs::read_dir(path).await?; - while let Some(data_dir_ent) = ls_data_dir.next_entry().await? { - if *must_exit.borrow() { - break; - } - - let name = data_dir_ent.file_name(); - let name = if let Ok(n) = name.into_string() { - n - } else { - continue; - }; - let ent_type = data_dir_ent.file_type().await?; - - let name = name.strip_suffix(".zst").unwrap_or(&name); - if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() { - state = self - .for_each_file_rec(&data_dir_ent.path(), state, f, must_exit) - .await?; - } else if name.len() == 64 { - let hash_bytes = if let Ok(h) = hex::decode(&name) { - h - } else { - continue; - }; - let mut hash = [0u8; 32]; - hash.copy_from_slice(&hash_bytes[..]); - state = f(state, hash.into()).await?; - } - } - Ok(state) - } - .boxed() - } } #[async_trait] diff --git a/src/block/repair.rs b/src/block/repair.rs new file mode 100644 index 00000000..0445527c --- /dev/null +++ b/src/block/repair.rs @@ -0,0 +1,204 @@ +use core::ops::Bound; + +use std::sync::Arc; +use std::time::Duration; + +use async_trait::async_trait; +use tokio::fs; +use tokio::sync::watch; + +use garage_util::background::*; +use garage_util::data::*; +use garage_util::error::*; +use garage_util::tranquilizer::Tranquilizer; + +use crate::manager::*; + +pub struct RepairWorker { + manager: Arc, + next_start: Option, + block_iter: Option, +} + +impl RepairWorker { + pub fn new(manager: Arc) -> Self { + Self { + manager, + next_start: None, + block_iter: None, + } + } +} + +#[async_trait] +impl Worker for RepairWorker { + fn name(&self) -> String { + "Block repair worker".into() + } + + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + match self.block_iter.as_mut() { + None => { + // Phase 1: Repair blocks from RC table. + + // We have to do this complicated two-step process where we first read a bunch + // of hashes from the RC table, and then insert them in the to-resync queue, + // because of SQLite. Basically, as long as we have an iterator on a DB table, + // we can't do anything else on the DB. The naive approach (which we had previously) + // of just iterating on the RC table and inserting items one to one in the resync + // queue can't work here, it would just provoke a deadlock in the SQLite adapter code. + // This is mostly because the Rust bindings for SQLite assume a worst-case scenario + // where SQLite is not compiled in thread-safe mode, so we have to wrap everything + // in a mutex (see db/sqlite_adapter.rs and discussion in PR #322). + let mut batch_of_hashes = vec![]; + let start_bound = match self.next_start.as_ref() { + None => Bound::Unbounded, + Some(x) => Bound::Excluded(x.as_slice()), + }; + for entry in self + .manager + .rc + .rc + .range::<&[u8], _>((start_bound, Bound::Unbounded))? + { + let (hash, _) = entry?; + let hash = Hash::try_from(&hash[..]).unwrap(); + batch_of_hashes.push(hash); + if batch_of_hashes.len() >= 1000 { + break; + } + } + if batch_of_hashes.is_empty() { + // move on to phase 2 + self.block_iter = Some(BlockStoreIterator::new(&self.manager).await?); + return Ok(WorkerStatus::Busy); + } + + for hash in batch_of_hashes.into_iter() { + self.manager.put_to_resync(&hash, Duration::from_secs(0))?; + self.next_start = Some(hash) + } + + Ok(WorkerStatus::Busy) + } + Some(bi) => { + // Phase 2: Repair blocks actually on disk + // Lists all blocks on disk and adds them to the resync queue. + // 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))?; + Ok(WorkerStatus::Busy) + } else { + Ok(WorkerStatus::Done) + } + } + } + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + unreachable!() + } +} + +// ---- + +pub struct ScrubWorker { + manager: Arc, + iterator: BlockStoreIterator, + tranquilizer: Tranquilizer, + tranquility: u32, +} + +impl ScrubWorker { + pub async fn new(manager: Arc, tranquility: u32) -> Result { + let iterator = BlockStoreIterator::new(&manager).await?; + Ok(Self { + manager, + iterator, + tranquilizer: Tranquilizer::new(30), + tranquility, + }) + } +} + +#[async_trait] +impl Worker for ScrubWorker { + fn name(&self) -> String { + "Block scrub worker".into() + } + + async fn work( + &mut self, + _must_exit: &mut watch::Receiver, + ) -> Result { + self.tranquilizer.reset(); + if let Some(hash) = self.iterator.next().await? { + let _ = self.manager.read_block(&hash).await; + self.tranquilizer.tranquilize(self.tranquility).await; + Ok(WorkerStatus::Busy) + } else { + Ok(WorkerStatus::Done) + } + } + + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + unreachable!() + } +} + +// ---- + +struct BlockStoreIterator { + path: Vec, +} + +impl BlockStoreIterator { + async fn new(manager: &BlockManager) -> Result { + let root_dir = manager.data_dir.clone(); + let read_root_dir = fs::read_dir(&root_dir).await?; + Ok(Self { + path: vec![read_root_dir], + }) + } + + async fn next(&mut self) -> Result, Error> { + loop { + if let Some(reader) = self.path.last_mut() { + if let Some(data_dir_ent) = reader.next_entry().await? { + let name = data_dir_ent.file_name(); + let name = if let Ok(n) = name.into_string() { + n + } else { + continue; + }; + let ent_type = data_dir_ent.file_type().await?; + + let name = name.strip_suffix(".zst").unwrap_or(&name); + if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() { + let read_child_dir = fs::read_dir(&data_dir_ent.path()).await?; + self.path.push(read_child_dir); + continue; + } else if name.len() == 64 { + let hash_bytes = if let Ok(h) = hex::decode(&name) { + h + } else { + continue; + }; + let mut hash = [0u8; 32]; + hash.copy_from_slice(&hash_bytes[..]); + return Ok(Some(hash.into())); + } + } else { + self.path.pop(); + continue; + } + } else { + return Ok(None); + } + } + } +} diff --git a/src/garage/Cargo.toml b/src/garage/Cargo.toml index 640e6975..7678ea26 100644 --- a/src/garage/Cargo.toml +++ b/src/garage/Cargo.toml @@ -23,6 +23,7 @@ path = "tests/lib.rs" [dependencies] garage_db = { version = "0.8.0", path = "../db" } garage_api = { version = "0.7.0", path = "../api" } +garage_block = { version = "0.7.0", path = "../block" } garage_model = { version = "0.7.0", path = "../model" } garage_rpc = { version = "0.7.0", path = "../rpc" } garage_table = { version = "0.7.0", path = "../table" } diff --git a/src/garage/admin.rs b/src/garage/admin.rs index c9783e54..8a984cfb 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -693,7 +693,7 @@ impl AdminRpcHandler { ))) } } else { - launch_online_repair(self.garage.clone(), opt)?; + launch_online_repair(self.garage.clone(), opt).await?; Ok(AdminRpc::Ok(format!( "Repair launched on {:?}", self.garage.system.id diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index e6fcd705..a5ccfa02 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -13,7 +13,7 @@ use garage_util::error::Error; use crate::*; -pub fn launch_online_repair(garage: Arc, opt: RepairOpt) -> Result<(), Error> { +pub async fn launch_online_repair(garage: Arc, opt: RepairOpt) -> Result<(), Error> { match opt.what { RepairWhat::Tables => { info!("Launching a full sync of tables"); @@ -36,24 +36,19 @@ pub fn launch_online_repair(garage: Arc, opt: RepairOpt) -> Result<(), E .spawn_worker(RepairBlockrefsWorker::new(garage.clone())); } RepairWhat::Blocks => { - unimplemented!() - /* info!("Repairing the stored blocks"); - self.garage - .block_manager - .repair_data_store(&must_exit) - .await?; - */ + garage + .background + .spawn_worker(garage_block::repair::RepairWorker::new( + garage.block_manager.clone(), + )); } RepairWhat::Scrub { tranquility } => { - unimplemented!() - /* info!("Verifying integrity of stored blocks"); - self.garage - .block_manager - .scrub_data_store(&must_exit, tranquility) - .await?; - */ + garage.background.spawn_worker( + garage_block::repair::ScrubWorker::new(garage.block_manager.clone(), tranquility) + .await?, + ); } } Ok(()) @@ -64,7 +59,7 @@ pub fn launch_online_repair(garage: Arc, opt: RepairOpt) -> Result<(), E struct RepairVersionsWorker { garage: Arc, pos: Vec, - iter: usize, + counter: usize, } impl RepairVersionsWorker { @@ -72,7 +67,7 @@ impl RepairVersionsWorker { Self { garage, pos: vec![], - iter: 0, + counter: 0, } } } @@ -93,14 +88,14 @@ impl Worker for RepairVersionsWorker { v } None => { - info!("repair_versions: finished, done {}", self.iter); + info!("repair_versions: finished, done {}", self.counter); return Ok(WorkerStatus::Done); } }; - self.iter += 1; - if self.iter % 1000 == 0 { - info!("repair_versions: {}", self.iter); + self.counter += 1; + if self.counter % 1000 == 0 { + info!("repair_versions: {}", self.counter); } let version = rmp_serde::decode::from_read_ref::<_, Version>(&item_bytes)?; @@ -144,7 +139,7 @@ impl Worker for RepairVersionsWorker { struct RepairBlockrefsWorker { garage: Arc, pos: Vec, - iter: usize, + counter: usize, } impl RepairBlockrefsWorker { @@ -152,7 +147,7 @@ impl RepairBlockrefsWorker { Self { garage, pos: vec![], - iter: 0, + counter: 0, } } } @@ -173,14 +168,14 @@ impl Worker for RepairBlockrefsWorker { v } None => { - info!("repair_block_ref: finished, done {}", self.iter); + info!("repair_block_ref: finished, done {}", self.counter); return Ok(WorkerStatus::Done); } }; - self.iter += 1; - if self.iter % 1000 == 0 { - info!("repair_block_ref: {}", self.iter); + self.counter += 1; + if self.counter % 1000 == 0 { + info!("repair_block_ref: {}", self.counter); } let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(&item_bytes)?; diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index 92f7990c..e30fecd7 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -159,6 +159,9 @@ impl WorkerHandler { self.task_id, e ); + // Sleep a bit so that error won't repeat immediately + // (TODO good way to handle errors) + tokio::time::sleep(Duration::from_secs(10)).await; } }, WorkerStatus::Idle => { -- 2.43.0 From 14337d2a561887b94ea165727dd49263717cb478 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 16:27:12 +0200 Subject: [PATCH 07/33] rename things --- src/block/manager.rs | 24 ++++++++++++------------ src/util/background/worker.rs | 1 + 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 54368faf..30d6e792 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -109,7 +109,7 @@ pub struct BlockManager { // it INSIDE a Mutex. struct BlockManagerLocked(); -enum BlockIterResult { +enum ResyncIterResult { BusyDidSomething, BusyDidNothing, IdleFor(Duration), @@ -478,7 +478,7 @@ impl BlockManager { fn spawn_background_worker(self: Arc) { // Launch a background workers for background resync loop processing let background = self.system.background.clone(); - let worker = BlockResyncWorker { + let worker = ResyncWorker { manager: self, tranquilizer: Tranquilizer::new(30), next_delay: Duration::from_secs(10), @@ -503,7 +503,7 @@ impl BlockManager { Ok(()) } - async fn resync_iter(&self) -> Result { + 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(); @@ -523,7 +523,7 @@ impl BlockManager { // (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(BlockIterResult::BusyDidNothing); + return Ok(ResyncIterResult::BusyDidNothing); } } @@ -570,9 +570,9 @@ impl BlockManager { self.resync_queue.remove(time_bytes)?; } - Ok(BlockIterResult::BusyDidSomething) + Ok(ResyncIterResult::BusyDidSomething) } else { - Ok(BlockIterResult::IdleFor(Duration::from_millis( + Ok(ResyncIterResult::IdleFor(Duration::from_millis( time_msec - now, ))) } @@ -583,7 +583,7 @@ impl BlockManager { // 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(BlockIterResult::IdleFor(Duration::from_secs(10))) + Ok(ResyncIterResult::IdleFor(Duration::from_secs(10))) } } @@ -716,14 +716,14 @@ impl EndpointHandler for BlockManager { } } -struct BlockResyncWorker { +struct ResyncWorker { manager: Arc, tranquilizer: Tranquilizer, next_delay: Duration, } #[async_trait] -impl Worker for BlockResyncWorker { +impl Worker for ResyncWorker { fn name(&self) -> String { "Block resync worker".into() } @@ -734,14 +734,14 @@ impl Worker for BlockResyncWorker { ) -> Result { self.tranquilizer.reset(); match self.manager.resync_iter().await { - Ok(BlockIterResult::BusyDidSomething) => { + Ok(ResyncIterResult::BusyDidSomething) => { self.tranquilizer .tranquilize(self.manager.background_tranquility) .await; Ok(WorkerStatus::Busy) } - Ok(BlockIterResult::BusyDidNothing) => Ok(WorkerStatus::Busy), - Ok(BlockIterResult::IdleFor(delay)) => { + Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerStatus::Busy), + Ok(ResyncIterResult::IdleFor(delay)) => { self.next_delay = delay; Ok(WorkerStatus::Idle) } diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index e30fecd7..f916692d 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -86,6 +86,7 @@ impl WorkerProcessor { } worker = await_next_worker => { if let Some(mut worker) = worker { + trace!("{} (TID {}): {:?}", worker.worker.name(), worker.task_id, worker.status); // TODO save new worker status somewhere if worker.status == WorkerStatus::Done { info!("Worker {} (TID {}) exited", worker.worker.name(), worker.task_id); -- 2.43.0 From 10c886111e304a53be24b29194f2127d91121929 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 17:18:16 +0200 Subject: [PATCH 08/33] Rename things, garage worker list cmd --- src/garage/admin.rs | 23 +++++++++++++++++++++++ src/garage/cli/cmd.rs | 4 ++++ src/garage/cli/structs.rs | 21 +++++++++++++++++++++ src/garage/cli/util.rs | 24 ++++++++++++++++++++++++ src/table/gc.rs | 11 ++++++++++- src/table/merkle.rs | 11 ++++++++++- src/table/sync.rs | 11 ++++++++++- src/util/background/mod.rs | 25 +++++++++++++++++++++++-- src/util/background/worker.rs | 27 ++++++++++++++++++++++++++- 9 files changed, 151 insertions(+), 6 deletions(-) diff --git a/src/garage/admin.rs b/src/garage/admin.rs index 8a984cfb..f307bea1 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -5,6 +5,7 @@ use std::sync::Arc; use async_trait::async_trait; use serde::{Deserialize, Serialize}; +use garage_util::background::*; use garage_util::crdt::*; use garage_util::data::*; use garage_util::error::Error as GarageError; @@ -36,6 +37,7 @@ pub enum AdminRpc { LaunchRepair(RepairOpt), Migrate(MigrateOpt), Stats(StatsOpt), + Worker(WorkerOpt), // Replies Ok(String), @@ -47,6 +49,7 @@ pub enum AdminRpc { }, KeyList(Vec<(String, String)>), KeyInfo(Key, HashMap), + WorkerList(HashMap), } impl Rpc for AdminRpc { @@ -822,6 +825,25 @@ impl AdminRpcHandler { Ok(()) } + + // ---- + + async fn handle_worker_cmd(&self, opt: WorkerOpt) -> Result { + match opt.cmd { + WorkerCmd::List { busy } => { + let workers = self.garage.background.get_worker_info(); + let workers = if busy { + workers + .into_iter() + .filter(|(_, w)| w.status == WorkerStatus::Busy) + .collect() + } else { + workers + }; + Ok(AdminRpc::WorkerList(workers)) + } + } + } } #[async_trait] @@ -837,6 +859,7 @@ impl EndpointHandler for AdminRpcHandler { AdminRpc::Migrate(opt) => self.handle_migrate(opt.clone()).await, AdminRpc::LaunchRepair(opt) => self.handle_launch_repair(opt.clone()).await, AdminRpc::Stats(opt) => self.handle_stats(opt.clone()).await, + AdminRpc::Worker(opt) => self.handle_worker_cmd(opt.clone()).await, m => Err(GarageError::unexpected_rpc_message(m).into()), } } diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 3a0bd956..38a58b76 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -39,6 +39,7 @@ pub async fn cli_command_dispatch( cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::LaunchRepair(ro)).await } Command::Stats(so) => cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::Stats(so)).await, + Command::Worker(wo) => cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::Worker(wo)).await, _ => unreachable!(), } } @@ -182,6 +183,9 @@ pub async fn cmd_admin( AdminRpc::KeyInfo(key, rb) => { print_key_info(&key, &rb); } + AdminRpc::WorkerList(wi) => { + print_worker_info(wi); + } r => { error!("Unexpected response: {:?}", r); } diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index 4f2efe19..f6b2d197 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -45,6 +45,10 @@ pub enum Command { /// Gather node statistics #[structopt(name = "stats")] Stats(StatsOpt), + + /// Manage background workers + #[structopt(name = "worker")] + Worker(WorkerOpt), } #[derive(StructOpt, Debug)] @@ -460,3 +464,20 @@ pub struct StatsOpt { #[structopt(short = "d", long = "detailed")] pub detailed: bool, } + +#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)] +pub struct WorkerOpt { + #[structopt(subcommand)] + pub cmd: WorkerCmd, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)] +pub enum WorkerCmd { + /// List all workers on Garage node + #[structopt(name = "list")] + List { + /// Show only busy workers + #[structopt(short = "b", long = "busy")] + busy: bool, + }, +} diff --git a/src/garage/cli/util.rs b/src/garage/cli/util.rs index 329e8a3e..81361864 100644 --- a/src/garage/cli/util.rs +++ b/src/garage/cli/util.rs @@ -1,5 +1,6 @@ use std::collections::HashMap; +use garage_util::background::*; use garage_util::crdt::*; use garage_util::data::Uuid; use garage_util::error::*; @@ -235,3 +236,26 @@ pub fn find_matching_node( Ok(candidates[0]) } } + +pub fn print_worker_info(wi: HashMap) { + let mut wi = wi.into_iter().collect::>(); + wi.sort_by_key(|(tid, info)| { + ( + match info.status { + WorkerStatus::Busy => 0, + WorkerStatus::Idle => 1, + WorkerStatus::Done => 2, + }, + *tid, + ) + }); + + let mut table = vec![]; + for (tid, info) in wi.iter() { + table.push(format!("{}\t{:?}\t{}", tid, info.status, info.name)); + if let Some(i) = &info.info { + table.push(format!("\t\t{}", i)); + } + } + format_table(table); +} diff --git a/src/table/gc.rs b/src/table/gc.rs index 36124c2f..d088a11c 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -332,7 +332,16 @@ where R: TableReplication + 'static, { fn name(&self) -> String { - format!("Table GC: {}", F::TABLE_NAME) + format!("{} GC", F::TABLE_NAME) + } + + fn info(&self) -> Option { + let l = self.gc.data.gc_todo_len().unwrap_or(0); + if l > 0 { + Some(format!("{} items in queue", l)) + } else { + None + } } async fn work( diff --git a/src/table/merkle.rs b/src/table/merkle.rs index d4d2717f..06d131cb 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -310,7 +310,16 @@ where R: TableReplication + 'static, { fn name(&self) -> String { - format!("Merkle tree updater: {}", F::TABLE_NAME) + format!("{} Merkle tree updater", F::TABLE_NAME) + } + + fn info(&self) -> Option { + let l = self.0.todo_len().unwrap_or(0); + if l > 0 { + Some(format!("{} items in queue", l)) + } else { + None + } } async fn work( diff --git a/src/table/sync.rs b/src/table/sync.rs index be081d96..bdf88782 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -574,7 +574,16 @@ impl SyncWorker { #[async_trait] impl Worker for SyncWorker { fn name(&self) -> String { - format!("Table sync worker for {}", F::TABLE_NAME) + format!("{} sync", F::TABLE_NAME) + } + + fn info(&self) -> Option { + let l = self.todo.len(); + if l > 0 { + Some(format!("{} partitions remaining", l)) + } else { + None + } } async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index c06e2225..92090a1a 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -4,9 +4,12 @@ pub mod job_worker; pub mod worker; use core::future::Future; + +use std::collections::HashMap; use std::pin::Pin; use std::sync::Arc; +use serde::{Deserialize, Serialize}; use tokio::sync::{mpsc, watch, Mutex}; use crate::error::Error; @@ -20,6 +23,14 @@ pub(crate) type Job = Pin + Send>>; pub struct BackgroundRunner { send_job: mpsc::UnboundedSender<(Job, bool)>, send_worker: mpsc::UnboundedSender>, + worker_info: Arc>>, +} + +#[derive(Clone, Serialize, Deserialize, Debug)] +pub struct WorkerInfo { + pub name: String, + pub info: Option, + pub status: WorkerStatus, } impl BackgroundRunner { @@ -30,8 +41,13 @@ impl BackgroundRunner { ) -> (Arc, tokio::task::JoinHandle<()>) { let (send_worker, worker_out) = mpsc::unbounded_channel::>(); - let await_all_done = - tokio::spawn(async move { WorkerProcessor::new(worker_out, stop_signal).run().await }); + let worker_info = Arc::new(std::sync::Mutex::new(HashMap::new())); + let mut worker_processor = + WorkerProcessor::new(worker_out, stop_signal, worker_info.clone()); + + let await_all_done = tokio::spawn(async move { + worker_processor.run().await; + }); let (send_job, queue_out) = mpsc::unbounded_channel(); let queue_out = Arc::new(Mutex::new(queue_out)); @@ -52,10 +68,15 @@ impl BackgroundRunner { let bgrunner = Arc::new(Self { send_job, send_worker, + worker_info, }); (bgrunner, await_all_done) } + pub fn get_worker_info(&self) -> HashMap { + self.worker_info.lock().unwrap().clone() + } + /// Spawn a task to be run in background pub fn spawn(&self, job: T) where diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index f916692d..1d4eb3ea 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -1,16 +1,20 @@ +use std::collections::HashMap; +use std::sync::Arc; use std::time::{Duration, Instant}; use async_trait::async_trait; use futures::future::*; use futures::stream::FuturesUnordered; use futures::StreamExt; +use serde::{Deserialize, Serialize}; use tokio::select; use tokio::sync::{mpsc, watch}; use tracing::*; +use crate::background::WorkerInfo; use crate::error::Error; -#[derive(PartialEq, Copy, Clone, Debug)] +#[derive(PartialEq, Copy, Clone, Debug, Serialize, Deserialize)] pub enum WorkerStatus { Busy, Idle, @@ -21,6 +25,10 @@ pub enum WorkerStatus { pub trait Worker: Send { fn name(&self) -> String; + fn info(&self) -> Option { + None + } + /// Work: do a basic unit of work, if one is available (otherwise, should return /// WorkerStatus::Idle immediately). We will do our best to not interrupt this future in the /// middle of processing, it will only be interrupted at the last minute when Garage is trying @@ -39,16 +47,19 @@ pub trait Worker: Send { pub(crate) struct WorkerProcessor { stop_signal: watch::Receiver, worker_chan: mpsc::UnboundedReceiver>, + worker_info: Arc>>, } impl WorkerProcessor { pub(crate) fn new( worker_chan: mpsc::UnboundedReceiver>, stop_signal: watch::Receiver, + worker_info: Arc>>, ) -> Self { Self { stop_signal, worker_chan, + worker_info, } } @@ -87,6 +98,20 @@ impl WorkerProcessor { worker = await_next_worker => { if let Some(mut worker) = worker { trace!("{} (TID {}): {:?}", worker.worker.name(), worker.task_id, worker.status); + let mut wi = self.worker_info.lock().unwrap(); + match wi.get_mut(&worker.task_id) { + Some(i) => { + i.status = worker.status; + i.info = worker.worker.info(); + } + None => { + wi.insert(worker.task_id, WorkerInfo { + name: worker.worker.name(), + status: worker.status, + info: worker.worker.info(), + }); + } + } // TODO save new worker status somewhere if worker.status == WorkerStatus::Done { info!("Worker {} (TID {}) exited", worker.worker.name(), worker.task_id); -- 2.43.0 From 708dab6e0ff8c2dab5cac39a1f64a5e391c32137 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 17:22:20 +0200 Subject: [PATCH 09/33] fix clippy --- src/util/background/worker.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index 1d4eb3ea..f933fc06 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -193,7 +193,7 @@ impl WorkerHandler { WorkerStatus::Idle => { if *self.stop_signal.borrow() { select! { - new_st = self.worker.wait_for_work(&mut self.stop_signal_worker) => { + new_st = self.worker.wait_for_work(&self.stop_signal_worker) => { self.status = new_st; } _ = tokio::time::sleep(Duration::from_secs(1)) => { @@ -202,7 +202,7 @@ impl WorkerHandler { } } else { select! { - new_st = self.worker.wait_for_work(&mut self.stop_signal_worker) => { + new_st = self.worker.wait_for_work(&self.stop_signal_worker) => { self.status = new_st; } _ = self.stop_signal.changed() => { -- 2.43.0 From 8fb8569912745ea90e8d0dd8313b431774780ad2 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 18:04:03 +0200 Subject: [PATCH 10/33] fix clippy --- src/table/sync.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/table/sync.rs b/src/table/sync.rs index bdf88782..a331ec75 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -597,11 +597,10 @@ impl Worker for SyncWor async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { select! { - s = self.add_full_sync_rx.recv() => match s { - Some(()) => { + s = self.add_full_sync_rx.recv() => { + if let Some(()) = s { self.add_full_sync(); } - None => (), }, _ = self.ring_recv.changed() => { let new_ring = self.ring_recv.borrow(); -- 2.43.0 From f82cf164f5ab525560415355503c04315901a739 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 21 Jun 2022 18:06:59 +0200 Subject: [PATCH 11/33] Update Cargo.nix --- Cargo.nix | 32 +++++++++++++++++--------------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/Cargo.nix b/Cargo.nix index e5155e61..6427edf9 100644 --- a/Cargo.nix +++ b/Cargo.nix @@ -739,7 +739,7 @@ in registry = "registry+https://github.com/rust-lang/crates.io-index"; src = fetchCratesIo { inherit name version; sha256 = "59a6001667ab124aebae2a495118e11d30984c3a653e99d86d58971708cf5e4b"; }; dependencies = { - ${ if hostPlatform.config == "aarch64-linux-android" || hostPlatform.parsed.cpu.name == "aarch64" && hostPlatform.parsed.kernel.name == "linux" || hostPlatform.config == "aarch64-apple-darwin" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + ${ if hostPlatform.config == "aarch64-apple-darwin" || hostPlatform.config == "aarch64-linux-android" || hostPlatform.parsed.cpu.name == "aarch64" && hostPlatform.parsed.kernel.name == "linux" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; }; }); @@ -1364,6 +1364,7 @@ in futures = rustPackages."registry+https://github.com/rust-lang/crates.io-index".futures."0.3.21" { inherit profileName; }; futures_util = rustPackages."registry+https://github.com/rust-lang/crates.io-index".futures-util."0.3.21" { inherit profileName; }; garage_api = rustPackages."unknown".garage_api."0.7.0" { inherit profileName; }; + garage_block = rustPackages."unknown".garage_block."0.7.0" { inherit profileName; }; garage_db = rustPackages."unknown".garage_db."0.8.0" { inherit profileName; }; garage_model = rustPackages."unknown".garage_model."0.7.0" { inherit profileName; }; garage_rpc = rustPackages."unknown".garage_rpc."0.7.0" { inherit profileName; }; @@ -1493,10 +1494,10 @@ in err_derive = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".err-derive."0.3.1" { profileName = "__noProfile"; }; heed = rustPackages."registry+https://github.com/rust-lang/crates.io-index".heed."0.11.0" { inherit profileName; }; hexdump = rustPackages."registry+https://github.com/rust-lang/crates.io-index".hexdump."0.1.1" { inherit profileName; }; - log = rustPackages."registry+https://github.com/rust-lang/crates.io-index".log."0.4.16" { inherit profileName; }; ${ if rootFeatures' ? "garage_db" then "pretty_env_logger" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".pretty_env_logger."0.4.0" { inherit profileName; }; rusqlite = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rusqlite."0.27.0" { inherit profileName; }; sled = rustPackages."registry+https://github.com/rust-lang/crates.io-index".sled."0.34.7" { inherit profileName; }; + tracing = rustPackages."registry+https://github.com/rust-lang/crates.io-index".tracing."0.1.32" { inherit profileName; }; }; devDependencies = { mktemp = rustPackages."registry+https://github.com/rust-lang/crates.io-index".mktemp."0.4.1" { inherit profileName; }; @@ -1717,6 +1718,7 @@ in (lib.optional (rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_util") "k2v") ]; dependencies = { + ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "async_trait" else null } = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".async-trait."0.1.52" { profileName = "__noProfile"; }; ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "blake2" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".blake2."0.9.2" { inherit profileName; }; ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "chrono" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".chrono."0.4.19" { inherit profileName; }; ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "err_derive" else null } = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".err-derive."0.3.1" { profileName = "__noProfile"; }; @@ -4327,7 +4329,7 @@ in ]; dependencies = { bitflags = rustPackages."registry+https://github.com/rust-lang/crates.io-index".bitflags."1.3.2" { inherit profileName; }; - ${ if hostPlatform.parsed.kernel.name == "linux" || hostPlatform.parsed.kernel.name == "android" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + ${ if hostPlatform.parsed.kernel.name == "android" || hostPlatform.parsed.kernel.name == "linux" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; ${ if !(hostPlatform.parsed.kernel.name == "linux" || hostPlatform.parsed.kernel.name == "android") then "parking_lot" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".parking_lot."0.11.2" { inherit profileName; }; ${ if !(hostPlatform.parsed.kernel.name == "linux" || hostPlatform.parsed.kernel.name == "android") then "parking_lot_core" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".parking_lot_core."0.8.5" { inherit profileName; }; static_init_macro = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".static_init_macro."1.0.2" { profileName = "__noProfile"; }; @@ -4410,7 +4412,7 @@ in [ "proc-macro" ] [ "quote" ] [ "visit" ] - (lib.optional (rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client") "visit-mut") + [ "visit-mut" ] ]; dependencies = { proc_macro2 = rustPackages."registry+https://github.com/rust-lang/crates.io-index".proc-macro2."1.0.36" { inherit profileName; }; @@ -4883,19 +4885,19 @@ in registry = "registry+https://github.com/rust-lang/crates.io-index"; src = fetchCratesIo { inherit name version; sha256 = "4a1bdf54a7c28a2bbf701e1d2233f6c77f473486b94bee4f9678da5a148dca7f"; }; features = builtins.concatLists [ - (lib.optional (rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client") "attributes") - (lib.optional (rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client") "default") + [ "attributes" ] + [ "default" ] (lib.optional (rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_web") "log") (lib.optional (rootFeatures' ? "garage") "log-always") - (lib.optional (rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client") "std") - (lib.optional (rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client") "tracing-attributes") + [ "std" ] + [ "tracing-attributes" ] ]; dependencies = { - ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "cfg_if" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".cfg-if."1.0.0" { inherit profileName; }; + cfg_if = rustPackages."registry+https://github.com/rust-lang/crates.io-index".cfg-if."1.0.0" { inherit profileName; }; ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_web" then "log" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".log."0.4.16" { inherit profileName; }; - ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "pin_project_lite" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".pin-project-lite."0.2.8" { inherit profileName; }; - ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "tracing_attributes" else null } = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".tracing-attributes."0.1.20" { profileName = "__noProfile"; }; - ${ if rootFeatures' ? "garage" || rootFeatures' ? "garage_api" || rootFeatures' ? "garage_block" || rootFeatures' ? "garage_model" || rootFeatures' ? "garage_rpc" || rootFeatures' ? "garage_table" || rootFeatures' ? "garage_util" || rootFeatures' ? "garage_web" || rootFeatures' ? "k2v-client" then "tracing_core" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".tracing-core."0.1.23" { inherit profileName; }; + pin_project_lite = rustPackages."registry+https://github.com/rust-lang/crates.io-index".pin-project-lite."0.2.8" { inherit profileName; }; + tracing_attributes = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".tracing-attributes."0.1.20" { profileName = "__noProfile"; }; + tracing_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".tracing-core."0.1.23" { inherit profileName; }; }; }); @@ -5330,10 +5332,10 @@ in [ "default" ] ]; dependencies = { - ${ if hostPlatform.config == "aarch64-uwp-windows-msvc" || hostPlatform.config == "aarch64-pc-windows-msvc" then "windows_aarch64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_aarch64_msvc."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "i686-pc-windows-gnu" || hostPlatform.config == "i686-uwp-windows-gnu" then "windows_i686_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_gnu."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "aarch64-pc-windows-msvc" || hostPlatform.config == "aarch64-uwp-windows-msvc" then "windows_aarch64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_aarch64_msvc."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "i686-uwp-windows-gnu" || hostPlatform.config == "i686-pc-windows-gnu" then "windows_i686_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_gnu."0.32.0" { inherit profileName; }; ${ if hostPlatform.config == "i686-uwp-windows-msvc" || hostPlatform.config == "i686-pc-windows-msvc" then "windows_i686_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_msvc."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "x86_64-uwp-windows-gnu" || hostPlatform.config == "x86_64-pc-windows-gnu" then "windows_x86_64_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_gnu."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "x86_64-pc-windows-gnu" || hostPlatform.config == "x86_64-uwp-windows-gnu" then "windows_x86_64_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_gnu."0.32.0" { inherit profileName; }; ${ if hostPlatform.config == "x86_64-uwp-windows-msvc" || hostPlatform.config == "x86_64-pc-windows-msvc" then "windows_x86_64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_msvc."0.32.0" { inherit profileName; }; }; }); -- 2.43.0 From 59b43914d4a9ae9a50ae79fee61b1a46bff941f9 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 23 Jun 2022 17:05:11 +0200 Subject: [PATCH 12/33] (to test) error reporting and throttling at higher layer --- src/block/manager.rs | 15 +++++-------- src/block/repair.rs | 3 +-- src/garage/admin.rs | 4 +++- src/garage/cli/util.rs | 16 +++++++++++++- src/model/index_counter.rs | 12 +++++++---- src/util/background/mod.rs | 3 +++ src/util/background/worker.rs | 40 +++++++++++++++++++++++++++++------ src/util/tranquilizer.rs | 23 +++++++++++++++++--- 8 files changed, 89 insertions(+), 27 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 30d6e792..db73ecbc 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -734,12 +734,9 @@ impl Worker for ResyncWorker { ) -> Result { self.tranquilizer.reset(); match self.manager.resync_iter().await { - Ok(ResyncIterResult::BusyDidSomething) => { - self.tranquilizer - .tranquilize(self.manager.background_tranquility) - .await; - Ok(WorkerStatus::Busy) - } + Ok(ResyncIterResult::BusyDidSomething) => Ok(self + .tranquilizer + .tranquilize_worker(self.manager.background_tranquility)), Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerStatus::Busy), Ok(ResyncIterResult::IdleFor(delay)) => { self.next_delay = delay; @@ -750,10 +747,8 @@ impl Worker for ResyncWorker { // 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?) - error!( - "Could not do a resync iteration: {} (this is a very bad error)", - e - ); + // Here we just give the error to the worker manager, + // it will print it to the logs and increment a counter Err(e.into()) } } diff --git a/src/block/repair.rs b/src/block/repair.rs index 0445527c..a5c01629 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -138,8 +138,7 @@ impl Worker for ScrubWorker { self.tranquilizer.reset(); if let Some(hash) = self.iterator.next().await? { let _ = self.manager.read_block(&hash).await; - self.tranquilizer.tranquilize(self.tranquility).await; - Ok(WorkerStatus::Busy) + Ok(self.tranquilizer.tranquilize_worker(self.tranquility)) } else { Ok(WorkerStatus::Done) } diff --git a/src/garage/admin.rs b/src/garage/admin.rs index f307bea1..c98f5142 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -835,7 +835,9 @@ impl AdminRpcHandler { let workers = if busy { workers .into_iter() - .filter(|(_, w)| w.status == WorkerStatus::Busy) + .filter(|(_, w)| { + matches!(w.status, WorkerStatus::Busy | WorkerStatus::Throttled(_)) + }) .collect() } else { workers diff --git a/src/garage/cli/util.rs b/src/garage/cli/util.rs index 81361864..ddb353f9 100644 --- a/src/garage/cli/util.rs +++ b/src/garage/cli/util.rs @@ -242,7 +242,7 @@ pub fn print_worker_info(wi: HashMap) { wi.sort_by_key(|(tid, info)| { ( match info.status { - WorkerStatus::Busy => 0, + WorkerStatus::Busy | WorkerStatus::Throttled(_) => 0, WorkerStatus::Idle => 1, WorkerStatus::Done => 2, }, @@ -256,6 +256,20 @@ pub fn print_worker_info(wi: HashMap) { if let Some(i) = &info.info { table.push(format!("\t\t{}", i)); } + if info.consecutive_errors > 0 { + table.push(format!( + "\t\t{} CONSECUTIVE ERRORS ({} total), last: {}", + info.consecutive_errors, + info.errors, + info.last_error.as_deref().unwrap_or("(?)") + )); + } else if info.errors > 0 { + table.push(format!( + "\t\t{} errors, last: {}", + info.errors, + info.last_error.as_deref().unwrap_or("(?)") + )); + } } format_table(table); } diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index 4318a064..9e29b421 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -2,7 +2,6 @@ use core::ops::Bound; use std::collections::{hash_map, BTreeMap, HashMap}; use std::marker::PhantomData; use std::sync::Arc; -use std::time::Duration; use async_trait::async_trait; use serde::{Deserialize, Serialize}; @@ -408,6 +407,10 @@ impl Worker for IndexPropagatorWorker { format!("{} index counter propagator", T::COUNTER_TABLE_NAME) } + fn info(&self) -> Option { + Some(format!("{} items in queue", self.buf.len())) + } + async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { // This loop batches updates to counters to be sent all at once. // They are sent once the propagate_rx channel has been emptied (or is closed). @@ -429,9 +432,10 @@ impl Worker for IndexPropagatorWorker { error!("({}) Could not propagate {} counter values: {}, these counters will not be updated correctly.", T::COUNTER_TABLE_NAME, self.buf.len(), e); return Ok(WorkerStatus::Done); } - warn!("({}) Could not propagate {} counter values: {}, retrying in 5 seconds (retry #{})", T::COUNTER_TABLE_NAME, self.buf.len(), e, self.errors); - tokio::time::sleep(Duration::from_secs(5)).await; - return Ok(WorkerStatus::Busy); + // Propagate error up to worker manager, it will log it, increment a counter, + // and sleep for a certain delay (with exponential backoff), waiting for + // things to go back to normal + return Err(e); } else { self.buf.clear(); self.errors = 0; diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index 92090a1a..f7e15b80 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -31,6 +31,9 @@ pub struct WorkerInfo { pub name: String, pub info: Option, pub status: WorkerStatus, + pub errors: usize, + pub consecutive_errors: usize, + pub last_error: Option, } impl BackgroundRunner { diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index f933fc06..e4a04250 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -17,6 +17,7 @@ use crate::error::Error; #[derive(PartialEq, Copy, Clone, Debug, Serialize, Deserialize)] pub enum WorkerStatus { Busy, + Throttled(f32), Idle, Done, } @@ -82,14 +83,17 @@ impl WorkerProcessor { next_task_id += 1; let stop_signal = self.stop_signal.clone(); let stop_signal_worker = self.stop_signal.clone(); - workers.push(async move { - let mut worker = WorkerHandler { + let mut worker = WorkerHandler { task_id, stop_signal, stop_signal_worker, worker: new_worker, status: WorkerStatus::Busy, + errors: 0, + consecutive_errors: 0, + last_error: None, }; + workers.push(async move { worker.step().await; worker }.boxed()); @@ -98,21 +102,31 @@ impl WorkerProcessor { worker = await_next_worker => { if let Some(mut worker) = worker { trace!("{} (TID {}): {:?}", worker.worker.name(), worker.task_id, worker.status); + + // Save worker info let mut wi = self.worker_info.lock().unwrap(); match wi.get_mut(&worker.task_id) { Some(i) => { i.status = worker.status; i.info = worker.worker.info(); + i.errors = worker.errors; + i.consecutive_errors = worker.consecutive_errors; + if worker.last_error.is_some() { + i.last_error = worker.last_error.take(); + } } None => { wi.insert(worker.task_id, WorkerInfo { name: worker.worker.name(), status: worker.status, info: worker.worker.info(), + errors: worker.errors, + consecutive_errors: worker.consecutive_errors, + last_error: worker.last_error.take(), }); } } - // TODO save new worker status somewhere + if worker.status == WorkerStatus::Done { info!("Worker {} (TID {}) exited", worker.worker.name(), worker.task_id); } else { @@ -169,6 +183,9 @@ struct WorkerHandler { stop_signal_worker: watch::Receiver, worker: Box, status: WorkerStatus, + errors: usize, + consecutive_errors: usize, + last_error: Option, } impl WorkerHandler { @@ -177,6 +194,7 @@ impl WorkerHandler { WorkerStatus::Busy => match self.worker.work(&mut self.stop_signal).await { Ok(s) => { self.status = s; + self.consecutive_errors = 0; } Err(e) => { error!( @@ -185,11 +203,21 @@ impl WorkerHandler { self.task_id, e ); - // Sleep a bit so that error won't repeat immediately - // (TODO good way to handle errors) - tokio::time::sleep(Duration::from_secs(10)).await; + self.errors += 1; + self.consecutive_errors += 1; + self.last_error = Some(format!("{}", e)); + // Sleep a bit so that error won't repeat immediately, exponential backoff + // strategy (min 1sec, max ~60sec) + self.status = WorkerStatus::Throttled( + (1.5f32).powf(std::cmp::min(10, self.consecutive_errors - 1) as f32), + ); } }, + WorkerStatus::Throttled(delay) => { + // Sleep for given delay and go back to busy state + tokio::time::sleep(Duration::from_secs_f32(delay)).await; + self.status = WorkerStatus::Busy; + } WorkerStatus::Idle => { if *self.stop_signal.borrow() { select! { diff --git a/src/util/tranquilizer.rs b/src/util/tranquilizer.rs index 28711387..f0c2b410 100644 --- a/src/util/tranquilizer.rs +++ b/src/util/tranquilizer.rs @@ -3,6 +3,8 @@ use std::time::{Duration, Instant}; use tokio::time::sleep; +use crate::background::WorkerStatus; + /// A tranquilizer is a helper object that is used to make /// background operations not take up too much time. /// @@ -33,7 +35,7 @@ impl Tranquilizer { } } - pub async fn tranquilize(&mut self, tranquility: u32) { + fn tranquilize_internal(&mut self, tranquility: u32) -> Option { let observation = Instant::now() - self.last_step_begin; self.observations.push_back(observation); @@ -45,10 +47,25 @@ impl Tranquilizer { if !self.observations.is_empty() { let delay = (tranquility * self.sum_observations) / (self.observations.len() as u32); - sleep(delay).await; + Some(delay) + } else { + None } + } - self.reset(); + pub async fn tranquilize(&mut self, tranquility: u32) { + if let Some(delay) = self.tranquilize_internal(tranquility) { + sleep(delay).await; + self.reset(); + } + } + + #[must_use] + pub fn tranquilize_worker(&mut self, tranquility: u32) -> WorkerStatus { + match self.tranquilize_internal(tranquility) { + Some(delay) => WorkerStatus::Throttled(delay.as_secs_f32()), + None => WorkerStatus::Busy, + } } pub fn reset(&mut self) { -- 2.43.0 From 95ffba343f14d7274e08099b9aca5a85da2259ed Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 24 Jun 2022 10:31:11 +0200 Subject: [PATCH 13/33] Error reporting --- Cargo.lock | 188 ++++++++++++++++++++++++++++++++-- Makefile | 11 +- src/garage/Cargo.toml | 1 + src/garage/admin.rs | 20 ++-- src/garage/cli/cmd.rs | 8 +- src/garage/cli/structs.rs | 15 ++- src/garage/cli/util.rs | 42 ++++++-- src/model/index_counter.rs | 6 +- src/table/gc.rs | 5 +- src/table/merkle.rs | 5 +- src/table/sync.rs | 5 +- src/util/background/mod.rs | 2 +- src/util/background/worker.rs | 15 ++- 13 files changed, 274 insertions(+), 49 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 007e3b4e..8de73002 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -93,6 +93,15 @@ dependencies = [ "winapi", ] +[[package]] +name = "autocfg" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0dde43e75fd43e8a1bf86103336bc699aa8d17ad1be60c76c0bdfd4828e19b78" +dependencies = [ + "autocfg 1.1.0", +] + [[package]] name = "autocfg" version = "1.1.0" @@ -543,7 +552,7 @@ version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1145cf131a2c6ba0615079ab6a638f7e1973ac9c2634fcbeaaad6114246efe8c" dependencies = [ - "autocfg", + "autocfg 1.1.0", "cfg-if 1.0.0", "crossbeam-utils 0.8.8", "lazy_static", @@ -985,6 +994,7 @@ dependencies = [ "sha2", "static_init", "structopt", + "timeago", "tokio", "toml", "tracing", @@ -1631,7 +1641,7 @@ version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282a6247722caba404c065016bbfa522806e51714c34f5dfc3e4a3a46fcb4223" dependencies = [ - "autocfg", + "autocfg 1.1.0", "hashbrown", ] @@ -1653,6 +1663,16 @@ dependencies = [ "serde", ] +[[package]] +name = "isolang" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "265ef164908329e47e753c769b14cbb27434abf0c41984dca201484022f09ce5" +dependencies = [ + "phf", + "phf_codegen", +] + [[package]] name = "itertools" version = "0.4.19" @@ -1977,7 +1997,7 @@ version = "0.6.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5aa361d4faea93603064a027415f07bd8e1d5c88c9fbf68bf56a285428fd79ce" dependencies = [ - "autocfg", + "autocfg 1.1.0", ] [[package]] @@ -2139,7 +2159,7 @@ version = "0.1.44" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2cc698a63b549a70bc047073d2949cce27cd1c7b0a4a862d08a8031bc2801db" dependencies = [ - "autocfg", + "autocfg 1.1.0", "num-traits", ] @@ -2149,7 +2169,7 @@ version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9a64b1ec5cda2586e284722486d802acf1f7dbdc623e2bfc57e65ca1cd099290" dependencies = [ - "autocfg", + "autocfg 1.1.0", ] [[package]] @@ -2218,7 +2238,7 @@ version = "0.9.72" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7e46109c383602735fa0a2e48dd2b7c892b048e1bf69e5c3b1d804b7d9c203cb" dependencies = [ - "autocfg", + "autocfg 1.1.0", "cc", "libc", "openssl-src", @@ -2388,6 +2408,44 @@ dependencies = [ "indexmap", ] +[[package]] +name = "phf" +version = "0.7.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b3da44b85f8e8dfaec21adae67f95d93244b2ecf6ad2a692320598dcc8e6dd18" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_codegen" +version = "0.7.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b03e85129e324ad4166b06b2c7491ae27fe3ec353af72e72cd1654c7225d517e" +dependencies = [ + "phf_generator", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.7.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09364cc93c159b8b06b1f4dd8a4398984503483891b0c26b867cf431fb132662" +dependencies = [ + "phf_shared", + "rand 0.6.5", +] + +[[package]] +name = "phf_shared" +version = "0.7.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "234f71a15de2288bcb7e3b6515828d22af7ec8598ee6d24c3b526fa0a80b67a0" +dependencies = [ + "siphasher", +] + [[package]] name = "pin-project" version = "0.4.29" @@ -2642,6 +2700,25 @@ dependencies = [ "winapi", ] +[[package]] +name = "rand" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d71dacdc3c88c1fde3885a3be3fbab9f35724e6ce99467f7d9c5026132184ca" +dependencies = [ + "autocfg 0.1.8", + "libc", + "rand_chacha 0.1.1", + "rand_core 0.4.2", + "rand_hc", + "rand_isaac", + "rand_jitter", + "rand_os", + "rand_pcg", + "rand_xorshift", + "winapi", +] + [[package]] name = "rand" version = "0.8.5" @@ -2649,10 +2726,20 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" dependencies = [ "libc", - "rand_chacha", + "rand_chacha 0.3.1", "rand_core 0.6.3", ] +[[package]] +name = "rand_chacha" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "556d3a1ca6600bfcbab7c7c91ccb085ac7fbbcd70e008a98742e7847f4f7bcef" +dependencies = [ + "autocfg 0.1.8", + "rand_core 0.3.1", +] + [[package]] name = "rand_chacha" version = "0.3.1" @@ -2687,6 +2774,77 @@ dependencies = [ "getrandom", ] +[[package]] +name = "rand_hc" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b40677c7be09ae76218dc623efbf7b18e34bced3f38883af07bb75630a21bc4" +dependencies = [ + "rand_core 0.3.1", +] + +[[package]] +name = "rand_isaac" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ded997c9d5f13925be2a6fd7e66bf1872597f759fd9dd93513dd7e92e5a5ee08" +dependencies = [ + "rand_core 0.3.1", +] + +[[package]] +name = "rand_jitter" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1166d5c91dc97b88d1decc3285bb0a99ed84b05cfd0bc2341bdf2d43fc41e39b" +dependencies = [ + "libc", + "rand_core 0.4.2", + "winapi", +] + +[[package]] +name = "rand_os" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b75f676a1e053fc562eafbb47838d67c84801e38fc1ba459e8f180deabd5071" +dependencies = [ + "cloudabi", + "fuchsia-cprng", + "libc", + "rand_core 0.4.2", + "rdrand", + "winapi", +] + +[[package]] +name = "rand_pcg" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "abf9b09b01790cfe0364f52bf32995ea3c39f4d2dd011eac241d2914146d0b44" +dependencies = [ + "autocfg 0.1.8", + "rand_core 0.4.2", +] + +[[package]] +name = "rand_xorshift" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cbf7e9e623549b0e21f6e97cf8ecf247c1a8fd2e8a992ae265314300b2455d5c" +dependencies = [ + "rand_core 0.3.1", +] + +[[package]] +name = "rdrand" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "678054eb77286b51581ba43620cc911abf02758c91f93f479767aed0f90458b2" +dependencies = [ + "rand_core 0.3.1", +] + [[package]] name = "redox_syscall" version = "0.2.11" @@ -3109,6 +3267,12 @@ dependencies = [ "libc", ] +[[package]] +name = "siphasher" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b8de496cf83d4ed58b6be86c3a275b8602f6ffe98d3024a869e124147a9a3ac" + [[package]] name = "slab" version = "0.4.5" @@ -3357,6 +3521,16 @@ dependencies = [ "num_threads", ] +[[package]] +name = "timeago" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ec32dde57efb15c035ac074118d7f32820451395f28cb0524a01d4e94983b26" +dependencies = [ + "chrono", + "isolang", +] + [[package]] name = "tinyvec" version = "1.5.1" diff --git a/Makefile b/Makefile index eeeffedb..406230c9 100644 --- a/Makefile +++ b/Makefile @@ -1,4 +1,4 @@ -.PHONY: doc all release shell +.PHONY: doc all release shell run1 run2 run3 all: clear; cargo build --all-features @@ -11,3 +11,12 @@ release: shell: nix-shell + +run1: + RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.1.toml server + +run2: + RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.2.toml server + +run3: + RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.3.toml server diff --git a/src/garage/Cargo.toml b/src/garage/Cargo.toml index 7678ea26..8948e750 100644 --- a/src/garage/Cargo.toml +++ b/src/garage/Cargo.toml @@ -32,6 +32,7 @@ garage_web = { version = "0.7.0", path = "../web" } bytes = "1.0" bytesize = "1.1" +timeago = "0.3" hex = "0.4" tracing = { version = "0.1.30", features = ["log-always"] } pretty_env_logger = "0.4" diff --git a/src/garage/admin.rs b/src/garage/admin.rs index c98f5142..9c6a0c57 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -5,7 +5,6 @@ use std::sync::Arc; use async_trait::async_trait; use serde::{Deserialize, Serialize}; -use garage_util::background::*; use garage_util::crdt::*; use garage_util::data::*; use garage_util::error::Error as GarageError; @@ -49,7 +48,10 @@ pub enum AdminRpc { }, KeyList(Vec<(String, String)>), KeyInfo(Key, HashMap), - WorkerList(HashMap), + WorkerList( + HashMap, + WorkerListOpt, + ), } impl Rpc for AdminRpc { @@ -830,19 +832,9 @@ impl AdminRpcHandler { async fn handle_worker_cmd(&self, opt: WorkerOpt) -> Result { match opt.cmd { - WorkerCmd::List { busy } => { + WorkerCmd::List { opt } => { let workers = self.garage.background.get_worker_info(); - let workers = if busy { - workers - .into_iter() - .filter(|(_, w)| { - matches!(w.status, WorkerStatus::Busy | WorkerStatus::Throttled(_)) - }) - .collect() - } else { - workers - }; - Ok(AdminRpc::WorkerList(workers)) + Ok(AdminRpc::WorkerList(workers, opt)) } } } diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 38a58b76..1aa2c2ff 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -1,4 +1,5 @@ use std::collections::HashSet; +use std::time::Duration; use garage_util::error::*; use garage_util::formater::format_table; @@ -101,6 +102,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tLast seen".to_string()]; for adv in status.iter().filter(|adv| !adv.is_up) { if let Some(NodeRoleV(Some(cfg))) = layout.roles.get(&adv.id) { + let tf = timeago::Formatter::new(); failed_nodes.push(format!( "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{last_seen}", id = adv.id, @@ -111,7 +113,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> capacity = cfg.capacity_string(), last_seen = adv .last_seen_secs_ago - .map(|s| format!("{}s ago", s)) + .map(|s| tf.convert(Duration::from_secs(s))) .unwrap_or_else(|| "never seen".into()), )); } @@ -183,8 +185,8 @@ pub async fn cmd_admin( AdminRpc::KeyInfo(key, rb) => { print_key_info(&key, &rb); } - AdminRpc::WorkerList(wi) => { - print_worker_info(wi); + AdminRpc::WorkerList(wi, wlo) => { + print_worker_info(wi, wlo); } r => { error!("Unexpected response: {:?}", r); diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index f6b2d197..c1ee32ab 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -476,8 +476,17 @@ pub enum WorkerCmd { /// List all workers on Garage node #[structopt(name = "list")] List { - /// Show only busy workers - #[structopt(short = "b", long = "busy")] - busy: bool, + #[structopt(flatten)] + opt: WorkerListOpt, }, } + +#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone, Copy)] +pub struct WorkerListOpt { + /// Show only busy workers + #[structopt(short = "b", long = "busy")] + pub busy: bool, + /// Show only workers with errors + #[structopt(short = "e", long = "errors")] + pub errors: bool, +} diff --git a/src/garage/cli/util.rs b/src/garage/cli/util.rs index ddb353f9..fc5a9932 100644 --- a/src/garage/cli/util.rs +++ b/src/garage/cli/util.rs @@ -1,15 +1,19 @@ use std::collections::HashMap; +use std::time::Duration; use garage_util::background::*; use garage_util::crdt::*; use garage_util::data::Uuid; use garage_util::error::*; use garage_util::formater::format_table; +use garage_util::time::*; use garage_model::bucket_table::*; use garage_model::key_table::*; use garage_model::s3::object_table::{BYTES, OBJECTS, UNFINISHED_UPLOADS}; +use crate::cli::structs::WorkerListOpt; + pub fn print_bucket_list(bl: Vec) { println!("List of buckets:"); @@ -237,7 +241,7 @@ pub fn find_matching_node( } } -pub fn print_worker_info(wi: HashMap) { +pub fn print_worker_info(wi: HashMap, wlo: WorkerListOpt) { let mut wi = wi.into_iter().collect::>(); wi.sort_by_key(|(tid, info)| { ( @@ -252,23 +256,39 @@ pub fn print_worker_info(wi: HashMap) { let mut table = vec![]; for (tid, info) in wi.iter() { + if wlo.busy && !matches!(info.status, WorkerStatus::Busy | WorkerStatus::Throttled(_)) { + continue; + } + if wlo.errors && info.errors == 0 { + continue; + } + table.push(format!("{}\t{:?}\t{}", tid, info.status, info.name)); if let Some(i) = &info.info { - table.push(format!("\t\t{}", i)); + table.push(format!("\t\t {}", i)); } + let tf = timeago::Formatter::new(); + let (err_ago, err_msg) = info + .last_error + .as_ref() + .map(|(m, t)| { + ( + tf.convert(Duration::from_millis(now_msec() - t)), + m.as_str(), + ) + }) + .unwrap_or(("(?) ago".into(), "(?)")); if info.consecutive_errors > 0 { table.push(format!( - "\t\t{} CONSECUTIVE ERRORS ({} total), last: {}", - info.consecutive_errors, - info.errors, - info.last_error.as_deref().unwrap_or("(?)") + "\t\t {} consecutive errors ({} total), last {}", + info.consecutive_errors, info.errors, err_ago, )); + table.push(format!("\t\t {}", err_msg)); } else if info.errors > 0 { - table.push(format!( - "\t\t{} errors, last: {}", - info.errors, - info.last_error.as_deref().unwrap_or("(?)") - )); + table.push(format!("\t\t ({} errors, last {})", info.errors, err_ago,)); + if wlo.errors { + table.push(format!("\t\t {}", err_msg)); + } } } format_table(table); diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index 9e29b421..474ec12c 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -408,7 +408,11 @@ impl Worker for IndexPropagatorWorker { } fn info(&self) -> Option { - Some(format!("{} items in queue", self.buf.len())) + if !self.buf.is_empty() { + Some(format!("{} items in queue", self.buf.len())) + } else { + None + } } async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { diff --git a/src/table/gc.rs b/src/table/gc.rs index d088a11c..0899d5e5 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -357,7 +357,10 @@ where } } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { + if *must_exit.borrow() { + return WorkerStatus::Done; + } tokio::time::sleep(self.wait_delay).await; WorkerStatus::Busy } diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 06d131cb..4c84933a 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -329,7 +329,10 @@ where self.0.updater_loop_iter() } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { + if *must_exit.borrow() { + return WorkerStatus::Done; + } tokio::time::sleep(Duration::from_secs(10)).await; WorkerStatus::Busy } diff --git a/src/table/sync.rs b/src/table/sync.rs index a331ec75..a7e1994c 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -595,7 +595,10 @@ impl Worker for SyncWor } } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { + if *must_exit.borrow() { + return WorkerStatus::Done; + } select! { s = self.add_full_sync_rx.recv() => { if let Some(()) = s { diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index f7e15b80..636b9c13 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -33,7 +33,7 @@ pub struct WorkerInfo { pub status: WorkerStatus, pub errors: usize, pub consecutive_errors: usize, - pub last_error: Option, + pub last_error: Option<(String, u64)>, } impl BackgroundRunner { diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index e4a04250..c08a0aaa 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -13,6 +13,7 @@ use tracing::*; use crate::background::WorkerInfo; use crate::error::Error; +use crate::time::now_msec; #[derive(PartialEq, Copy, Clone, Debug, Serialize, Deserialize)] pub enum WorkerStatus { @@ -167,7 +168,7 @@ impl WorkerProcessor { select! { _ = drain_everything => { - info!("All workers exited in time \\o/"); + info!("All workers exited peacefully \\o/"); } _ = tokio::time::sleep(Duration::from_secs(9)) => { error!("Some workers could not exit in time, we are cancelling some things in the middle"); @@ -176,7 +177,6 @@ impl WorkerProcessor { } } -// TODO add tranquilizer struct WorkerHandler { task_id: usize, stop_signal: watch::Receiver, @@ -185,7 +185,7 @@ struct WorkerHandler { status: WorkerStatus, errors: usize, consecutive_errors: usize, - last_error: Option, + last_error: Option<(String, u64)>, } impl WorkerHandler { @@ -205,7 +205,7 @@ impl WorkerHandler { ); self.errors += 1; self.consecutive_errors += 1; - self.last_error = Some(format!("{}", e)); + self.last_error = Some((format!("{}", e), now_msec())); // Sleep a bit so that error won't repeat immediately, exponential backoff // strategy (min 1sec, max ~60sec) self.status = WorkerStatus::Throttled( @@ -215,7 +215,12 @@ impl WorkerHandler { }, WorkerStatus::Throttled(delay) => { // Sleep for given delay and go back to busy state - tokio::time::sleep(Duration::from_secs_f32(delay)).await; + if !*self.stop_signal.borrow() { + select! { + _ = tokio::time::sleep(Duration::from_secs_f32(delay)) => (), + _ = self.stop_signal.changed() => (), + } + } self.status = WorkerStatus::Busy; } WorkerStatus::Idle => { -- 2.43.0 From 0837b3dacd994b35dbccbfefdd68be044eb4b5ac Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 24 Jun 2022 10:49:52 +0200 Subject: [PATCH 14/33] Slightly improve blocking code, add info to resync worker --- src/block/manager.rs | 17 +++++++++++++++++ src/table/merkle.rs | 13 ++++++++++++- 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index db73ecbc..4a595cc8 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -728,6 +728,23 @@ impl Worker for ResyncWorker { "Block resync worker".into() } + fn info(&self) -> Option { + let mut ret = vec![]; + 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)); + } + if elen > 0 { + ret.push(format!("{} blocks in error state", elen)); + } + if ret.len() > 0 { + Some(ret.join(", ")) + } else { + None + } + } + async fn work( &mut self, _must_exit: &mut watch::Receiver, diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 4c84933a..21186220 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -326,7 +326,18 @@ where &mut self, _must_exit: &mut watch::Receiver, ) -> Result { - self.0.updater_loop_iter() + let updater = self.0.clone(); + tokio::task::spawn_blocking(move || { + for _i in 0..100 { + let s = updater.updater_loop_iter(); + if !matches!(s, Ok(WorkerStatus::Busy)) { + return s; + } + } + Ok(WorkerStatus::Busy) + }) + .await + .unwrap() } async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { -- 2.43.0 From e7810e9cb3cdbe6aaecddddd1146bf15e5b50c7c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 24 Jun 2022 11:04:55 +0200 Subject: [PATCH 15/33] Smaller batches for index counter propagation --- src/block/repair.rs | 1 + src/model/index_counter.rs | 7 +++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/block/repair.rs b/src/block/repair.rs index a5c01629..97989780 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -53,6 +53,7 @@ impl Worker for RepairWorker { // This is mostly because the Rust bindings for SQLite assume a worst-case scenario // where SQLite is not compiled in thread-safe mode, so we have to wrap everything // in a mutex (see db/sqlite_adapter.rs and discussion in PR #322). + // TODO: maybe do this with tokio::task::spawn_blocking ? let mut batch_of_hashes = vec![]; let start_bound = match self.next_start.as_ref() { None => Bound::Unbounded, diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index 474ec12c..9d5aa955 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -429,7 +429,8 @@ impl Worker for IndexPropagatorWorker { }; if !self.buf.is_empty() { - let entries = self.buf.iter().map(|(_k, v)| v); + let entries_k = self.buf.keys().take(100).cloned().collect::>(); + let entries = entries_k.iter().map(|k| self.buf.get(k).unwrap()); if let Err(e) = self.index_counter.table.insert_many(entries).await { self.errors += 1; if self.errors >= 2 && *must_exit.borrow() { @@ -441,7 +442,9 @@ impl Worker for IndexPropagatorWorker { // things to go back to normal return Err(e); } else { - self.buf.clear(); + for k in entries_k { + self.buf.remove(&k); + } self.errors = 0; } -- 2.43.0 From 928394cc326a2ce79d057b484c944943a379229f Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Jun 2022 11:58:14 +0200 Subject: [PATCH 16/33] (makefile with run release) --- Makefile | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/Makefile b/Makefile index 406230c9..33b63521 100644 --- a/Makefile +++ b/Makefile @@ -14,9 +14,15 @@ shell: run1: RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.1.toml server +run1rel: + RUST_LOG=garage=debug ./target/release/garage -c tmp/config.1.toml server run2: RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.2.toml server +run2rel: + RUST_LOG=garage=debug ./target/release/garage -c tmp/config.2.toml server run3: RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.3.toml server +run3rel: + RUST_LOG=garage=debug ./target/release/garage -c tmp/config.3.toml server -- 2.43.0 From fdfe7dd60dc78ec63d109d2c0fd3e5a36c5409b2 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Jun 2022 12:01:23 +0200 Subject: [PATCH 17/33] Fix clippy lint --- src/block/manager.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 4a595cc8..27f51ff8 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -738,7 +738,7 @@ impl Worker for ResyncWorker { if elen > 0 { ret.push(format!("{} blocks in error state", elen)); } - if ret.len() > 0 { + if !ret.is_empty() { Some(ret.join(", ")) } else { None -- 2.43.0 From fc507242569a91ee638485848f0fd7def6e026f7 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Jun 2022 12:06:17 +0200 Subject: [PATCH 18/33] Update cargo.nix --- Cargo.nix | 235 ++++++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 228 insertions(+), 7 deletions(-) diff --git a/Cargo.nix b/Cargo.nix index 6427edf9..c242710e 100644 --- a/Cargo.nix +++ b/Cargo.nix @@ -172,6 +172,16 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".autocfg."0.1.8" = overridableMkRustCrate (profileName: rec { + name = "autocfg"; + version = "0.1.8"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "0dde43e75fd43e8a1bf86103336bc699aa8d17ad1be60c76c0bdfd4828e19b78"; }; + dependencies = { + autocfg = rustPackages."registry+https://github.com/rust-lang/crates.io-index".autocfg."1.1.0" { inherit profileName; }; + }; + }); + "registry+https://github.com/rust-lang/crates.io-index".autocfg."1.1.0" = overridableMkRustCrate (profileName: rec { name = "autocfg"; version = "1.1.0"; @@ -739,7 +749,7 @@ in registry = "registry+https://github.com/rust-lang/crates.io-index"; src = fetchCratesIo { inherit name version; sha256 = "59a6001667ab124aebae2a495118e11d30984c3a653e99d86d58971708cf5e4b"; }; dependencies = { - ${ if hostPlatform.config == "aarch64-apple-darwin" || hostPlatform.config == "aarch64-linux-android" || hostPlatform.parsed.cpu.name == "aarch64" && hostPlatform.parsed.kernel.name == "linux" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + ${ if hostPlatform.config == "aarch64-apple-darwin" || hostPlatform.parsed.cpu.name == "aarch64" && hostPlatform.parsed.kernel.name == "linux" || hostPlatform.config == "aarch64-linux-android" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; }; }); @@ -1384,6 +1394,7 @@ in serde = rustPackages."registry+https://github.com/rust-lang/crates.io-index".serde."1.0.137" { inherit profileName; }; serde_bytes = rustPackages."registry+https://github.com/rust-lang/crates.io-index".serde_bytes."0.11.5" { inherit profileName; }; structopt = rustPackages."registry+https://github.com/rust-lang/crates.io-index".structopt."0.3.26" { inherit profileName; }; + timeago = rustPackages."registry+https://github.com/rust-lang/crates.io-index".timeago."0.3.1" { inherit profileName; }; tokio = rustPackages."registry+https://github.com/rust-lang/crates.io-index".tokio."1.17.0" { inherit profileName; }; toml = rustPackages."registry+https://github.com/rust-lang/crates.io-index".toml."0.5.8" { inherit profileName; }; tracing = rustPackages."registry+https://github.com/rust-lang/crates.io-index".tracing."0.1.32" { inherit profileName; }; @@ -2221,6 +2232,22 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".isolang."1.0.0" = overridableMkRustCrate (profileName: rec { + name = "isolang"; + version = "1.0.0"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "265ef164908329e47e753c769b14cbb27434abf0c41984dca201484022f09ce5"; }; + features = builtins.concatLists [ + [ "default" ] + ]; + dependencies = { + phf = rustPackages."registry+https://github.com/rust-lang/crates.io-index".phf."0.7.24" { inherit profileName; }; + }; + buildDependencies = { + phf_codegen = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".phf_codegen."0.7.24" { profileName = "__noProfile"; }; + }; + }); + "registry+https://github.com/rust-lang/crates.io-index".itertools."0.4.19" = overridableMkRustCrate (profileName: rec { name = "itertools"; version = "0.4.19"; @@ -2728,7 +2755,7 @@ in [ "os-poll" ] ]; dependencies = { - ${ if hostPlatform.isUnix || hostPlatform.parsed.kernel.name == "wasi" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + ${ if hostPlatform.parsed.kernel.name == "wasi" || hostPlatform.isUnix then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; log = rustPackages."registry+https://github.com/rust-lang/crates.io-index".log."0.4.16" { inherit profileName; }; ${ if hostPlatform.isWindows then "miow" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".miow."0.3.7" { inherit profileName; }; ${ if hostPlatform.isWindows then "ntapi" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".ntapi."0.3.7" { inherit profileName; }; @@ -3244,6 +3271,48 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".phf."0.7.24" = overridableMkRustCrate (profileName: rec { + name = "phf"; + version = "0.7.24"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "b3da44b85f8e8dfaec21adae67f95d93244b2ecf6ad2a692320598dcc8e6dd18"; }; + dependencies = { + phf_shared = rustPackages."registry+https://github.com/rust-lang/crates.io-index".phf_shared."0.7.24" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".phf_codegen."0.7.24" = overridableMkRustCrate (profileName: rec { + name = "phf_codegen"; + version = "0.7.24"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "b03e85129e324ad4166b06b2c7491ae27fe3ec353af72e72cd1654c7225d517e"; }; + dependencies = { + phf_generator = rustPackages."registry+https://github.com/rust-lang/crates.io-index".phf_generator."0.7.24" { inherit profileName; }; + phf_shared = rustPackages."registry+https://github.com/rust-lang/crates.io-index".phf_shared."0.7.24" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".phf_generator."0.7.24" = overridableMkRustCrate (profileName: rec { + name = "phf_generator"; + version = "0.7.24"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "09364cc93c159b8b06b1f4dd8a4398984503483891b0c26b867cf431fb132662"; }; + dependencies = { + phf_shared = rustPackages."registry+https://github.com/rust-lang/crates.io-index".phf_shared."0.7.24" { inherit profileName; }; + rand = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand."0.6.5" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".phf_shared."0.7.24" = overridableMkRustCrate (profileName: rec { + name = "phf_shared"; + version = "0.7.24"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "234f71a15de2288bcb7e3b6515828d22af7ec8598ee6d24c3b526fa0a80b67a0"; }; + dependencies = { + siphasher = rustPackages."registry+https://github.com/rust-lang/crates.io-index".siphasher."0.2.3" { inherit profileName; }; + }; + }); + "registry+https://github.com/rust-lang/crates.io-index".pin-project."0.4.29" = overridableMkRustCrate (profileName: rec { name = "pin-project"; version = "0.4.29"; @@ -3570,6 +3639,34 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".rand."0.6.5" = overridableMkRustCrate (profileName: rec { + name = "rand"; + version = "0.6.5"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "6d71dacdc3c88c1fde3885a3be3fbab9f35724e6ce99467f7d9c5026132184ca"; }; + features = builtins.concatLists [ + [ "alloc" ] + [ "default" ] + [ "rand_os" ] + [ "std" ] + ]; + dependencies = { + ${ if hostPlatform.isUnix then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + rand_chacha = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_chacha."0.1.1" { inherit profileName; }; + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.4.2" { inherit profileName; }; + rand_hc = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_hc."0.1.0" { inherit profileName; }; + rand_isaac = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_isaac."0.1.1" { inherit profileName; }; + rand_jitter = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_jitter."0.1.4" { inherit profileName; }; + rand_os = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_os."0.1.3" { inherit profileName; }; + rand_pcg = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_pcg."0.1.2" { inherit profileName; }; + rand_xorshift = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_xorshift."0.1.1" { inherit profileName; }; + ${ if hostPlatform.isWindows then "winapi" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".winapi."0.3.9" { inherit profileName; }; + }; + buildDependencies = { + autocfg = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".autocfg."0.1.8" { profileName = "__noProfile"; }; + }; + }); + "registry+https://github.com/rust-lang/crates.io-index".rand."0.8.5" = overridableMkRustCrate (profileName: rec { name = "rand"; version = "0.8.5"; @@ -3592,6 +3689,19 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".rand_chacha."0.1.1" = overridableMkRustCrate (profileName: rec { + name = "rand_chacha"; + version = "0.1.1"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "556d3a1ca6600bfcbab7c7c91ccb085ac7fbbcd70e008a98742e7847f4f7bcef"; }; + dependencies = { + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.3.1" { inherit profileName; }; + }; + buildDependencies = { + autocfg = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".autocfg."0.1.8" { profileName = "__noProfile"; }; + }; + }); + "registry+https://github.com/rust-lang/crates.io-index".rand_chacha."0.3.1" = overridableMkRustCrate (profileName: rec { name = "rand_chacha"; version = "0.3.1"; @@ -3646,6 +3756,93 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".rand_hc."0.1.0" = overridableMkRustCrate (profileName: rec { + name = "rand_hc"; + version = "0.1.0"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "7b40677c7be09ae76218dc623efbf7b18e34bced3f38883af07bb75630a21bc4"; }; + dependencies = { + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.3.1" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".rand_isaac."0.1.1" = overridableMkRustCrate (profileName: rec { + name = "rand_isaac"; + version = "0.1.1"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "ded997c9d5f13925be2a6fd7e66bf1872597f759fd9dd93513dd7e92e5a5ee08"; }; + dependencies = { + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.3.1" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".rand_jitter."0.1.4" = overridableMkRustCrate (profileName: rec { + name = "rand_jitter"; + version = "0.1.4"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "1166d5c91dc97b88d1decc3285bb0a99ed84b05cfd0bc2341bdf2d43fc41e39b"; }; + features = builtins.concatLists [ + [ "std" ] + ]; + dependencies = { + ${ if hostPlatform.parsed.kernel.name == "darwin" || hostPlatform.parsed.kernel.name == "ios" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.4.2" { inherit profileName; }; + ${ if hostPlatform.parsed.kernel.name == "windows" then "winapi" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".winapi."0.3.9" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".rand_os."0.1.3" = overridableMkRustCrate (profileName: rec { + name = "rand_os"; + version = "0.1.3"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "7b75f676a1e053fc562eafbb47838d67c84801e38fc1ba459e8f180deabd5071"; }; + dependencies = { + ${ if hostPlatform.parsed.kernel.name == "cloudabi" then "cloudabi" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".cloudabi."0.0.3" { inherit profileName; }; + ${ if hostPlatform.parsed.kernel.name == "fuchsia" then "fuchsia_cprng" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".fuchsia-cprng."0.1.1" { inherit profileName; }; + ${ if hostPlatform.isUnix then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.4.2" { inherit profileName; }; + ${ if hostPlatform.parsed.abi.name == "sgx" then "rdrand" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rdrand."0.4.0" { inherit profileName; }; + ${ if hostPlatform.isWindows then "winapi" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".winapi."0.3.9" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".rand_pcg."0.1.2" = overridableMkRustCrate (profileName: rec { + name = "rand_pcg"; + version = "0.1.2"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "abf9b09b01790cfe0364f52bf32995ea3c39f4d2dd011eac241d2914146d0b44"; }; + dependencies = { + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.4.2" { inherit profileName; }; + }; + buildDependencies = { + autocfg = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".autocfg."0.1.8" { profileName = "__noProfile"; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".rand_xorshift."0.1.1" = overridableMkRustCrate (profileName: rec { + name = "rand_xorshift"; + version = "0.1.1"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "cbf7e9e623549b0e21f6e97cf8ecf247c1a8fd2e8a992ae265314300b2455d5c"; }; + dependencies = { + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.3.1" { inherit profileName; }; + }; + }); + + "registry+https://github.com/rust-lang/crates.io-index".rdrand."0.4.0" = overridableMkRustCrate (profileName: rec { + name = "rdrand"; + version = "0.4.0"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "678054eb77286b51581ba43620cc911abf02758c91f93f479767aed0f90458b2"; }; + features = builtins.concatLists [ + [ "default" ] + [ "std" ] + ]; + dependencies = { + rand_core = rustPackages."registry+https://github.com/rust-lang/crates.io-index".rand_core."0.3.1" { inherit profileName; }; + }; + }); + "registry+https://github.com/rust-lang/crates.io-index".redox_syscall."0.2.11" = overridableMkRustCrate (profileName: rec { name = "redox_syscall"; version = "0.2.11"; @@ -3740,7 +3937,7 @@ in ]; dependencies = { ${ if hostPlatform.parsed.kernel.name == "android" || hostPlatform.parsed.kernel.name == "linux" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; - ${ if hostPlatform.parsed.kernel.name == "dragonfly" || hostPlatform.parsed.kernel.name == "freebsd" || hostPlatform.parsed.kernel.name == "illumos" || hostPlatform.parsed.kernel.name == "netbsd" || hostPlatform.parsed.kernel.name == "openbsd" || hostPlatform.parsed.kernel.name == "solaris" || hostPlatform.parsed.kernel.name == "android" || hostPlatform.parsed.kernel.name == "linux" then "once_cell" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".once_cell."1.10.0" { inherit profileName; }; + ${ if hostPlatform.parsed.kernel.name == "android" || hostPlatform.parsed.kernel.name == "linux" || hostPlatform.parsed.kernel.name == "dragonfly" || hostPlatform.parsed.kernel.name == "freebsd" || hostPlatform.parsed.kernel.name == "illumos" || hostPlatform.parsed.kernel.name == "netbsd" || hostPlatform.parsed.kernel.name == "openbsd" || hostPlatform.parsed.kernel.name == "solaris" then "once_cell" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".once_cell."1.10.0" { inherit profileName; }; ${ if hostPlatform.parsed.cpu.name == "i686" || hostPlatform.parsed.cpu.name == "x86_64" || (hostPlatform.parsed.cpu.name == "aarch64" || hostPlatform.parsed.cpu.name == "armv6l" || hostPlatform.parsed.cpu.name == "armv7l") && (hostPlatform.parsed.kernel.name == "android" || hostPlatform.parsed.kernel.name == "fuchsia" || hostPlatform.parsed.kernel.name == "linux") then "spin" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".spin."0.5.2" { inherit profileName; }; untrusted = rustPackages."registry+https://github.com/rust-lang/crates.io-index".untrusted."0.7.1" { inherit profileName; }; ${ if hostPlatform.parsed.cpu.name == "wasm32" && hostPlatform.parsed.vendor.name == "unknown" && hostPlatform.parsed.kernel.name == "unknown" && hostPlatform.parsed.abi.name == "" then "web_sys" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".web-sys."0.3.56" { inherit profileName; }; @@ -4215,6 +4412,13 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".siphasher."0.2.3" = overridableMkRustCrate (profileName: rec { + name = "siphasher"; + version = "0.2.3"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "0b8de496cf83d4ed58b6be86c3a275b8602f6ffe98d3024a869e124147a9a3ac"; }; + }); + "registry+https://github.com/rust-lang/crates.io-index".slab."0.4.5" = overridableMkRustCrate (profileName: rec { name = "slab"; version = "0.4.5"; @@ -4541,6 +4745,23 @@ in }; }); + "registry+https://github.com/rust-lang/crates.io-index".timeago."0.3.1" = overridableMkRustCrate (profileName: rec { + name = "timeago"; + version = "0.3.1"; + registry = "registry+https://github.com/rust-lang/crates.io-index"; + src = fetchCratesIo { inherit name version; sha256 = "6ec32dde57efb15c035ac074118d7f32820451395f28cb0524a01d4e94983b26"; }; + features = builtins.concatLists [ + [ "chrono" ] + [ "default" ] + [ "isolang" ] + [ "translations" ] + ]; + dependencies = { + chrono = rustPackages."registry+https://github.com/rust-lang/crates.io-index".chrono."0.4.19" { inherit profileName; }; + isolang = rustPackages."registry+https://github.com/rust-lang/crates.io-index".isolang."1.0.0" { inherit profileName; }; + }; + }); + "registry+https://github.com/rust-lang/crates.io-index".tinyvec."1.5.1" = overridableMkRustCrate (profileName: rec { name = "tinyvec"; version = "1.5.1"; @@ -5332,11 +5553,11 @@ in [ "default" ] ]; dependencies = { - ${ if hostPlatform.config == "aarch64-pc-windows-msvc" || hostPlatform.config == "aarch64-uwp-windows-msvc" then "windows_aarch64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_aarch64_msvc."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "i686-uwp-windows-gnu" || hostPlatform.config == "i686-pc-windows-gnu" then "windows_i686_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_gnu."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "aarch64-uwp-windows-msvc" || hostPlatform.config == "aarch64-pc-windows-msvc" then "windows_aarch64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_aarch64_msvc."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "i686-pc-windows-gnu" || hostPlatform.config == "i686-uwp-windows-gnu" then "windows_i686_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_gnu."0.32.0" { inherit profileName; }; ${ if hostPlatform.config == "i686-uwp-windows-msvc" || hostPlatform.config == "i686-pc-windows-msvc" then "windows_i686_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_msvc."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "x86_64-pc-windows-gnu" || hostPlatform.config == "x86_64-uwp-windows-gnu" then "windows_x86_64_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_gnu."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "x86_64-uwp-windows-msvc" || hostPlatform.config == "x86_64-pc-windows-msvc" then "windows_x86_64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_msvc."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "x86_64-uwp-windows-gnu" || hostPlatform.config == "x86_64-pc-windows-gnu" then "windows_x86_64_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_gnu."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "x86_64-pc-windows-msvc" || hostPlatform.config == "x86_64-uwp-windows-msvc" then "windows_x86_64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_msvc."0.32.0" { inherit profileName; }; }; }); -- 2.43.0 From 0e5175abeeb1b2d9cfe27603005b7feb3cf040de Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Jun 2022 16:52:46 +0200 Subject: [PATCH 19/33] Report progress of scrub and block repair --- Makefile | 17 ++-- src/block/repair.rs | 154 +++++++++++++++++++++++++++--------- src/garage/admin.rs | 2 +- src/garage/repair/online.rs | 13 +-- 4 files changed, 133 insertions(+), 53 deletions(-) diff --git a/Makefile b/Makefile index 33b63521..1f0f3644 100644 --- a/Makefile +++ b/Makefile @@ -3,26 +3,25 @@ all: clear; cargo build --all-features -doc: - cd doc/book; mdbook build - release: nix-build --arg release true shell: nix-shell +# ---- + run1: - RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.1.toml server + RUST_LOG=garage=debug ./target/debug/garage -c tmp/config1.toml server run1rel: - RUST_LOG=garage=debug ./target/release/garage -c tmp/config.1.toml server + RUST_LOG=garage=debug ./target/release/garage -c tmp/config1.toml server run2: - RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.2.toml server + RUST_LOG=garage=debug ./target/debug/garage -c tmp/config2.toml server run2rel: - RUST_LOG=garage=debug ./target/release/garage -c tmp/config.2.toml server + RUST_LOG=garage=debug ./target/release/garage -c tmp/config2.toml server run3: - RUST_LOG=garage=debug ./target/debug/garage -c tmp/config.3.toml server + RUST_LOG=garage=debug ./target/debug/garage -c tmp/config3.toml server run3rel: - RUST_LOG=garage=debug ./target/release/garage -c tmp/config.3.toml server + RUST_LOG=garage=debug ./target/release/garage -c tmp/config3.toml server diff --git a/src/block/repair.rs b/src/block/repair.rs index 97989780..a2a8443e 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -1,4 +1,5 @@ use core::ops::Bound; +use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; @@ -36,6 +37,25 @@ impl Worker for RepairWorker { "Block repair worker".into() } + fn info(&self) -> Option { + match self.block_iter.as_ref() { + None => { + let idx_bytes = self + .next_start + .as_ref() + .map(|x| x.as_slice()) + .unwrap_or(&[]); + let idx_bytes = if idx_bytes.len() > 4 { + &idx_bytes[..4] + } else { + idx_bytes + }; + Some(format!("Phase 1: {}", hex::encode(idx_bytes))) + } + Some(bi) => Some(format!("Phase 2: {:.2}% done", bi.progress() * 100.)), + } + } + async fn work( &mut self, _must_exit: &mut watch::Receiver, @@ -74,7 +94,7 @@ impl Worker for RepairWorker { } if batch_of_hashes.is_empty() { // move on to phase 2 - self.block_iter = Some(BlockStoreIterator::new(&self.manager).await?); + self.block_iter = Some(BlockStoreIterator::new(&self.manager)); return Ok(WorkerStatus::Busy); } @@ -115,14 +135,14 @@ pub struct ScrubWorker { } impl ScrubWorker { - pub async fn new(manager: Arc, tranquility: u32) -> Result { - let iterator = BlockStoreIterator::new(&manager).await?; - Ok(Self { + pub fn new(manager: Arc, tranquility: u32) -> Self { + let iterator = BlockStoreIterator::new(&manager); + Self { manager, iterator, tranquilizer: Tranquilizer::new(30), tranquility, - }) + } } } @@ -132,6 +152,10 @@ impl Worker for ScrubWorker { "Block scrub worker".into() } + fn info(&self) -> Option { + Some(format!("{:.2}% done", self.iterator.progress() * 100.)) + } + async fn work( &mut self, _must_exit: &mut watch::Receiver, @@ -153,51 +177,107 @@ impl Worker for ScrubWorker { // ---- struct BlockStoreIterator { - path: Vec, + path: Vec, +} + +enum ReadingDir { + Pending(PathBuf), + Read { + subpaths: Vec, + pos: usize, + }, } impl BlockStoreIterator { - async fn new(manager: &BlockManager) -> Result { + fn new(manager: &BlockManager) -> Self { let root_dir = manager.data_dir.clone(); - let read_root_dir = fs::read_dir(&root_dir).await?; - Ok(Self { - path: vec![read_root_dir], - }) + Self { + path: vec![ReadingDir::Pending(root_dir)], + } + } + + /// Returns progress done, between 0% and 1% + fn progress(&self) -> f32 { + if self.path.is_empty() { + 1.0 + } else { + let mut ret = 0.0; + let mut next_div = 1; + for p in self.path.iter() { + match p { + ReadingDir::Pending(_) => break, + ReadingDir::Read { subpaths, pos } => { + next_div *= subpaths.len(); + ret += ((*pos - 1) as f32) / (next_div as f32); + } + } + } + ret + } } async fn next(&mut self) -> Result, Error> { loop { - if let Some(reader) = self.path.last_mut() { - if let Some(data_dir_ent) = reader.next_entry().await? { - let name = data_dir_ent.file_name(); - let name = if let Ok(n) = name.into_string() { - n - } else { - continue; - }; - let ent_type = data_dir_ent.file_type().await?; + let last_path = match self.path.last_mut() { + None => return Ok(None), + Some(lp) => lp, + }; - let name = name.strip_suffix(".zst").unwrap_or(&name); - if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() { - let read_child_dir = fs::read_dir(&data_dir_ent.path()).await?; - self.path.push(read_child_dir); - continue; - } else if name.len() == 64 { - let hash_bytes = if let Ok(h) = hex::decode(&name) { - h - } else { - continue; - }; - let mut hash = [0u8; 32]; - hash.copy_from_slice(&hash_bytes[..]); - return Ok(Some(hash.into())); - } - } else { + if let ReadingDir::Pending(path) = last_path { + let mut reader = fs::read_dir(&path).await?; + let mut subpaths = vec![]; + while let Some(ent) = reader.next_entry().await? { + subpaths.push(ent); + } + *last_path = ReadingDir::Read { subpaths, pos: 0 }; + } + + let (subpaths, pos) = match *last_path { + ReadingDir::Read { + ref subpaths, + ref mut pos, + } => (subpaths, pos), + ReadingDir::Pending(_) => unreachable!(), + }; + + if *pos >= subpaths.len() { + self.path.pop(); + continue; + } + + let data_dir_ent = match subpaths.get(*pos) { + None => { self.path.pop(); continue; } + Some(ent) => { + *pos += 1; + ent + } + }; + + let name = data_dir_ent.file_name(); + let name = if let Ok(n) = name.into_string() { + n } else { - return Ok(None); + continue; + }; + let ent_type = data_dir_ent.file_type().await?; + + let name = name.strip_suffix(".zst").unwrap_or(&name); + if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() { + let path = data_dir_ent.path(); + self.path.push(ReadingDir::Pending(path)); + continue; + } else if name.len() == 64 { + let hash_bytes = if let Ok(h) = hex::decode(&name) { + h + } else { + continue; + }; + let mut hash = [0u8; 32]; + hash.copy_from_slice(&hash_bytes[..]); + return Ok(Some(hash.into())); } } } diff --git a/src/garage/admin.rs b/src/garage/admin.rs index 9c6a0c57..de49331e 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -698,7 +698,7 @@ impl AdminRpcHandler { ))) } } else { - launch_online_repair(self.garage.clone(), opt).await?; + launch_online_repair(self.garage.clone(), opt); Ok(AdminRpc::Ok(format!( "Repair launched on {:?}", self.garage.system.id diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index a5ccfa02..b0437c5e 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -13,7 +13,7 @@ use garage_util::error::Error; use crate::*; -pub async fn launch_online_repair(garage: Arc, opt: RepairOpt) -> Result<(), Error> { +pub fn launch_online_repair(garage: Arc, opt: RepairOpt) { match opt.what { RepairWhat::Tables => { info!("Launching a full sync of tables"); @@ -45,13 +45,14 @@ pub async fn launch_online_repair(garage: Arc, opt: RepairOpt) -> Result } RepairWhat::Scrub { tranquility } => { info!("Verifying integrity of stored blocks"); - garage.background.spawn_worker( - garage_block::repair::ScrubWorker::new(garage.block_manager.clone(), tranquility) - .await?, - ); + garage + .background + .spawn_worker(garage_block::repair::ScrubWorker::new( + garage.block_manager.clone(), + tranquility, + )); } } - Ok(()) } // ---- -- 2.43.0 From 247dbcd5980e6a0158fe209d85788d3167dceab0 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Jun 2022 17:57:48 +0200 Subject: [PATCH 20/33] Only one scrub worker (wip) --- Cargo.lock | 1 + src/block/Cargo.toml | 1 + src/block/manager.rs | 37 +++++++- src/block/repair.rs | 176 +++++++++++++++++++++++++++++++++--- src/garage/admin.rs | 2 +- src/garage/cli/structs.rs | 25 ++++- src/garage/repair/online.rs | 22 +++-- src/util/tranquilizer.rs | 4 + 8 files changed, 240 insertions(+), 28 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8de73002..c45ee015 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1049,6 +1049,7 @@ dependencies = [ name = "garage_block" version = "0.7.0" dependencies = [ + "arc-swap", "async-trait", "bytes 1.1.0", "futures", diff --git a/src/block/Cargo.toml b/src/block/Cargo.toml index 80346aca..2555a44a 100644 --- a/src/block/Cargo.toml +++ b/src/block/Cargo.toml @@ -21,6 +21,7 @@ garage_table = { version = "0.7.0", path = "../table" } opentelemetry = "0.17" +arc-swap = "1.5" async-trait = "0.1.7" bytes = "1.0" hex = "0.4" diff --git a/src/block/manager.rs b/src/block/manager.rs index 27f51ff8..015ac71b 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -3,6 +3,7 @@ use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; +use arc_swap::ArcSwapOption; use async_trait::async_trait; use serde::{Deserialize, Serialize}; @@ -10,7 +11,7 @@ use futures::future::*; use tokio::fs; use tokio::io::{AsyncReadExt, AsyncWriteExt}; use tokio::select; -use tokio::sync::{watch, Mutex, Notify}; +use tokio::sync::{mpsc, watch, Mutex, Notify}; use opentelemetry::{ trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer}, @@ -35,6 +36,7 @@ use garage_table::replication::{TableReplication, TableShardedReplication}; use crate::block::*; use crate::metrics::*; use crate::rc::*; +use crate::repair::*; /// Size under which data will be stored inlined in database instead of as files pub const INLINE_THRESHOLD: usize = 3072; @@ -86,6 +88,8 @@ pub struct BlockManager { pub replication: TableShardedReplication, /// Directory in which block are stored pub data_dir: PathBuf, + /// State store (only used by scrub worker to store time of last scrub) + pub(crate) state_variables_store: db::Tree, compression_level: Option, background_tranquility: u32, @@ -102,6 +106,8 @@ pub struct BlockManager { endpoint: Arc>, metrics: BlockManagerMetrics, + + tx_scrub_command: ArcSwapOption>, } // This custom struct contains functions that must only be ran @@ -141,6 +147,10 @@ impl BlockManager { let resync_errors = CountedTree::new(resync_errors).expect("Could not count block_local_resync_errors"); + let state_variables_store = db + .open_tree("state_variables") + .expect("Unable to open state_variables tree"); + let endpoint = system .netapp .endpoint("garage_block/manager.rs/Rpc".to_string()); @@ -159,13 +169,15 @@ impl BlockManager { resync_queue, resync_notify: Notify::new(), resync_errors, + state_variables_store, system, endpoint, metrics, + tx_scrub_command: ArcSwapOption::new(None), }); block_manager.endpoint.set_handler(block_manager.clone()); - block_manager.clone().spawn_background_worker(); + block_manager.clone().spawn_background_workers(); block_manager } @@ -242,6 +254,17 @@ impl BlockManager { Ok(self.rc.rc.len()?) } + /// 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; + } + //// ----- Managing the reference counter ---- /// Increment the number of time a block is used, putting it to resynchronization if it is @@ -475,11 +498,11 @@ impl BlockManager { // for times that are earlier than the exponential back-off delay // is a natural condition that is handled properly). - fn spawn_background_worker(self: Arc) { + fn spawn_background_workers(self: Arc) { // Launch a background workers for background resync loop processing let background = self.system.background.clone(); let worker = ResyncWorker { - manager: self, + manager: self.clone(), tranquilizer: Tranquilizer::new(30), next_delay: Duration::from_secs(10), }; @@ -487,6 +510,12 @@ impl BlockManager { 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, 4); + self.system.background.spawn_worker(scrub_worker); } pub(crate) fn put_to_resync(&self, hash: &Hash, delay: Duration) -> db::Result<()> { diff --git a/src/block/repair.rs b/src/block/repair.rs index a2a8443e..8335de51 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -1,20 +1,26 @@ use core::ops::Bound; +use std::convert::TryInto; use std::path::PathBuf; - use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; use tokio::fs; +use tokio::select; +use tokio::sync::mpsc; use tokio::sync::watch; use garage_util::background::*; use garage_util::data::*; use garage_util::error::*; +use garage_util::time::*; use garage_util::tranquilizer::Tranquilizer; use crate::manager::*; +const SCRUB_INTERVAL: Duration = Duration::from_secs(3600 * 24 * 30); // full scrub every 30 days +const TIME_LAST_COMPLETE_SCRUB: &[u8] = b"time_last_complete_scrub"; + pub struct RepairWorker { manager: Arc, next_start: Option, @@ -129,19 +135,107 @@ impl Worker for RepairWorker { pub struct ScrubWorker { manager: Arc, - iterator: BlockStoreIterator, + rx_cmd: mpsc::Receiver, + + work: ScrubWorkerState, tranquilizer: Tranquilizer, tranquility: u32, + + time_last_complete_scrub: u64, +} + +enum ScrubWorkerState { + Running(BlockStoreIterator), + Paused(BlockStoreIterator, u64), // u64 = time when to resume scrub + Finished, +} + +impl Default for ScrubWorkerState { + fn default() -> Self { + ScrubWorkerState::Finished + } +} + +pub enum ScrubWorkerCommand { + Start, + Pause(Duration), + Resume, + Cancel, + SetTranquility(u32), } impl ScrubWorker { - pub fn new(manager: Arc, tranquility: u32) -> Self { - let iterator = BlockStoreIterator::new(&manager); + pub fn new( + manager: Arc, + rx_cmd: mpsc::Receiver, + tranquility: u32, + ) -> Self { + let time_last_complete_scrub = match manager + .state_variables_store + .get(TIME_LAST_COMPLETE_SCRUB) + .expect("DB error when initializing scrub worker") + { + Some(v) => u64::from_be_bytes(v.try_into().unwrap()), + None => 0, + }; Self { manager, - iterator, + rx_cmd, + work: ScrubWorkerState::Finished, tranquilizer: Tranquilizer::new(30), tranquility, + time_last_complete_scrub, + } + } + + fn handle_cmd(&mut self, cmd: ScrubWorkerCommand) { + match cmd { + ScrubWorkerCommand::Start => { + self.work = match std::mem::take(&mut self.work) { + ScrubWorkerState::Finished => { + let iterator = BlockStoreIterator::new(&self.manager); + ScrubWorkerState::Running(iterator) + } + work => { + error!("Cannot start scrub worker: already running!"); + work + } + }; + } + ScrubWorkerCommand::Pause(dur) => { + self.work = match std::mem::take(&mut self.work) { + ScrubWorkerState::Running(it) | ScrubWorkerState::Paused(it, _) => { + ScrubWorkerState::Paused(it, now_msec() + dur.as_millis() as u64) + } + work => { + error!("Cannot pause scrub worker: not running!"); + work + } + }; + } + ScrubWorkerCommand::Resume => { + self.work = match std::mem::take(&mut self.work) { + ScrubWorkerState::Paused(it, _) => ScrubWorkerState::Running(it), + work => { + error!("Cannot resume scrub worker: not paused!"); + work + } + }; + } + ScrubWorkerCommand::Cancel => { + self.work = match std::mem::take(&mut self.work) { + ScrubWorkerState::Running(_) | ScrubWorkerState::Paused(_, _) => { + ScrubWorkerState::Finished + } + work => { + error!("Cannot cancel scrub worker: not running!"); + work + } + } + } + ScrubWorkerCommand::SetTranquility(t) => { + self.tranquility = t; + } } } } @@ -153,24 +247,80 @@ impl Worker for ScrubWorker { } fn info(&self) -> Option { - Some(format!("{:.2}% done", self.iterator.progress() * 100.)) + match &self.work { + ScrubWorkerState::Running(bsi) => Some(format!("{:.2}% done", bsi.progress() * 100.)), + ScrubWorkerState::Paused(_bsi, rt) => { + Some(format!("Paused, resumes at {}", msec_to_rfc3339(*rt))) + } + ScrubWorkerState::Finished => Some(format!( + "Last completed scrub: {}", + msec_to_rfc3339(self.time_last_complete_scrub) + )), + } } async fn work( &mut self, _must_exit: &mut watch::Receiver, ) -> Result { - self.tranquilizer.reset(); - if let Some(hash) = self.iterator.next().await? { - let _ = self.manager.read_block(&hash).await; - Ok(self.tranquilizer.tranquilize_worker(self.tranquility)) - } else { - Ok(WorkerStatus::Done) + match self.rx_cmd.try_recv() { + Ok(cmd) => self.handle_cmd(cmd), + Err(mpsc::error::TryRecvError::Disconnected) => return Ok(WorkerStatus::Done), + Err(mpsc::error::TryRecvError::Empty) => (), + }; + + match &mut self.work { + ScrubWorkerState::Running(bsi) => { + self.tranquilizer.reset(); + if let Some(hash) = bsi.next().await? { + let _ = self.manager.read_block(&hash).await; + Ok(self.tranquilizer.tranquilize_worker(self.tranquility)) + } else { + self.time_last_complete_scrub = now_msec(); // TODO save to file + self.manager.state_variables_store.insert( + TIME_LAST_COMPLETE_SCRUB, + u64::to_be_bytes(self.time_last_complete_scrub), + )?; + self.work = ScrubWorkerState::Finished; + self.tranquilizer.clear(); + Ok(WorkerStatus::Idle) + } + } + _ => Ok(WorkerStatus::Idle), } } async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { - unreachable!() + match &self.work { + ScrubWorkerState::Running(_) => return WorkerStatus::Busy, + ScrubWorkerState::Paused(_, resume_time) => { + let delay = Duration::from_millis(resume_time - now_msec()); + select! { + _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Resume), + cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { + self.handle_cmd(cmd); + } else { + return WorkerStatus::Done; + } + } + } + ScrubWorkerState::Finished => { + let delay = SCRUB_INTERVAL + - Duration::from_secs(now_msec() - self.time_last_complete_scrub); + select! { + _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Start), + cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { + self.handle_cmd(cmd); + } else { + return WorkerStatus::Done; + } + } + } + } + match &self.work { + ScrubWorkerState::Running(_) => WorkerStatus::Busy, + _ => WorkerStatus::Idle, + } } } diff --git a/src/garage/admin.rs b/src/garage/admin.rs index de49331e..71ee608c 100644 --- a/src/garage/admin.rs +++ b/src/garage/admin.rs @@ -698,7 +698,7 @@ impl AdminRpcHandler { ))) } } else { - launch_online_repair(self.garage.clone(), opt); + launch_online_repair(self.garage.clone(), opt).await; Ok(AdminRpc::Ok(format!( "Repair launched on {:?}", self.garage.system.id diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index c1ee32ab..bc44b5ef 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -427,8 +427,29 @@ pub enum RepairWhat { /// Verify integrity of all blocks on disc (extremely slow, i/o intensive) #[structopt(name = "scrub")] Scrub { - /// Tranquility factor (see tranquilizer documentation) - #[structopt(name = "tranquility", default_value = "2")] + #[structopt(subcommand)] + cmd: ScrubCmd, + }, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)] +pub enum ScrubCmd { + /// Start scrub + #[structopt(name = "start")] + Start, + /// Pause scrub (it will resume automatically after 24 hours) + #[structopt(name = "pause")] + Pause, + /// Resume paused scrub + #[structopt(name = "resume")] + Resume, + /// Cancel scrub in progress + #[structopt(name = "cancel")] + Cancel, + /// Set tranquility level for in-progress and future scrubs + #[structopt(name = "set-tranquility")] + SetTranquility { + #[structopt()] tranquility: u32, }, } diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index b0437c5e..8207a8b4 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -1,8 +1,10 @@ use std::sync::Arc; +use std::time::Duration; use async_trait::async_trait; use tokio::sync::watch; +use garage_block::repair::ScrubWorkerCommand; use garage_model::garage::Garage; use garage_model::s3::block_ref_table::*; use garage_model::s3::object_table::*; @@ -13,7 +15,7 @@ use garage_util::error::Error; use crate::*; -pub fn launch_online_repair(garage: Arc, opt: RepairOpt) { +pub async fn launch_online_repair(garage: Arc, opt: RepairOpt) { match opt.what { RepairWhat::Tables => { info!("Launching a full sync of tables"); @@ -43,14 +45,18 @@ pub fn launch_online_repair(garage: Arc, opt: RepairOpt) { garage.block_manager.clone(), )); } - RepairWhat::Scrub { tranquility } => { + RepairWhat::Scrub { cmd } => { info!("Verifying integrity of stored blocks"); - garage - .background - .spawn_worker(garage_block::repair::ScrubWorker::new( - garage.block_manager.clone(), - tranquility, - )); + let cmd = match cmd { + ScrubCmd::Start => ScrubWorkerCommand::Start, + ScrubCmd::Pause => ScrubWorkerCommand::Pause(Duration::from_secs(3600 * 24)), + ScrubCmd::Resume => ScrubWorkerCommand::Resume, + ScrubCmd::Cancel => ScrubWorkerCommand::Cancel, + ScrubCmd::SetTranquility { tranquility } => { + ScrubWorkerCommand::SetTranquility(tranquility) + } + }; + garage.block_manager.send_scrub_command(cmd).await; } } } diff --git a/src/util/tranquilizer.rs b/src/util/tranquilizer.rs index f0c2b410..9c796f8b 100644 --- a/src/util/tranquilizer.rs +++ b/src/util/tranquilizer.rs @@ -71,4 +71,8 @@ impl Tranquilizer { pub fn reset(&mut self) { self.last_step_begin = Instant::now(); } + + pub fn clear(&mut self) { + self.observations.clear(); + } } -- 2.43.0 From f1c972289d53e956a5837dcae2a53cc13f61ec7b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Jun 2022 16:09:26 +0200 Subject: [PATCH 21/33] update cargo.nix --- Cargo.nix | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/Cargo.nix b/Cargo.nix index c242710e..37bf3186 100644 --- a/Cargo.nix +++ b/Cargo.nix @@ -749,7 +749,7 @@ in registry = "registry+https://github.com/rust-lang/crates.io-index"; src = fetchCratesIo { inherit name version; sha256 = "59a6001667ab124aebae2a495118e11d30984c3a653e99d86d58971708cf5e4b"; }; dependencies = { - ${ if hostPlatform.config == "aarch64-apple-darwin" || hostPlatform.parsed.cpu.name == "aarch64" && hostPlatform.parsed.kernel.name == "linux" || hostPlatform.config == "aarch64-linux-android" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + ${ if hostPlatform.config == "aarch64-linux-android" || hostPlatform.parsed.cpu.name == "aarch64" && hostPlatform.parsed.kernel.name == "linux" || hostPlatform.config == "aarch64-apple-darwin" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; }; }); @@ -1470,6 +1470,7 @@ in registry = "unknown"; src = fetchCrateLocal (workspaceSrc + "/src/block"); dependencies = { + arc_swap = rustPackages."registry+https://github.com/rust-lang/crates.io-index".arc-swap."1.5.0" { inherit profileName; }; async_trait = buildRustPackages."registry+https://github.com/rust-lang/crates.io-index".async-trait."0.1.52" { profileName = "__noProfile"; }; bytes = rustPackages."registry+https://github.com/rust-lang/crates.io-index".bytes."1.1.0" { inherit profileName; }; futures = rustPackages."registry+https://github.com/rust-lang/crates.io-index".futures."0.3.21" { inherit profileName; }; @@ -2755,7 +2756,7 @@ in [ "os-poll" ] ]; dependencies = { - ${ if hostPlatform.parsed.kernel.name == "wasi" || hostPlatform.isUnix then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; + ${ if hostPlatform.isUnix || hostPlatform.parsed.kernel.name == "wasi" then "libc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".libc."0.2.121" { inherit profileName; }; log = rustPackages."registry+https://github.com/rust-lang/crates.io-index".log."0.4.16" { inherit profileName; }; ${ if hostPlatform.isWindows then "miow" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".miow."0.3.7" { inherit profileName; }; ${ if hostPlatform.isWindows then "ntapi" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".ntapi."0.3.7" { inherit profileName; }; @@ -5553,11 +5554,11 @@ in [ "default" ] ]; dependencies = { - ${ if hostPlatform.config == "aarch64-uwp-windows-msvc" || hostPlatform.config == "aarch64-pc-windows-msvc" then "windows_aarch64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_aarch64_msvc."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "i686-pc-windows-gnu" || hostPlatform.config == "i686-uwp-windows-gnu" then "windows_i686_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_gnu."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "aarch64-pc-windows-msvc" || hostPlatform.config == "aarch64-uwp-windows-msvc" then "windows_aarch64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_aarch64_msvc."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "i686-uwp-windows-gnu" || hostPlatform.config == "i686-pc-windows-gnu" then "windows_i686_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_gnu."0.32.0" { inherit profileName; }; ${ if hostPlatform.config == "i686-uwp-windows-msvc" || hostPlatform.config == "i686-pc-windows-msvc" then "windows_i686_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_i686_msvc."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "x86_64-uwp-windows-gnu" || hostPlatform.config == "x86_64-pc-windows-gnu" then "windows_x86_64_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_gnu."0.32.0" { inherit profileName; }; - ${ if hostPlatform.config == "x86_64-pc-windows-msvc" || hostPlatform.config == "x86_64-uwp-windows-msvc" then "windows_x86_64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_msvc."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "x86_64-pc-windows-gnu" || hostPlatform.config == "x86_64-uwp-windows-gnu" then "windows_x86_64_gnu" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_gnu."0.32.0" { inherit profileName; }; + ${ if hostPlatform.config == "x86_64-uwp-windows-msvc" || hostPlatform.config == "x86_64-pc-windows-msvc" then "windows_x86_64_msvc" else null } = rustPackages."registry+https://github.com/rust-lang/crates.io-index".windows_x86_64_msvc."0.32.0" { inherit profileName; }; }; }); -- 2.43.0 From b053fc051842677dbabe1cab294af74ed26932a4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Jun 2022 16:59:19 +0200 Subject: [PATCH 22/33] Persist scrub worker thing in Persister --- src/block/manager.rs | 11 +---- src/block/repair.rs | 89 ++++++++++++++++++++++------------- src/garage/cli/util.rs | 2 +- src/garage/repair/online.rs | 2 +- src/rpc/system.rs | 6 ++- src/util/background/worker.rs | 13 ++++- 6 files changed, 76 insertions(+), 47 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 015ac71b..36166ae3 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -88,8 +88,6 @@ pub struct BlockManager { pub replication: TableShardedReplication, /// Directory in which block are stored pub data_dir: PathBuf, - /// State store (only used by scrub worker to store time of last scrub) - pub(crate) state_variables_store: db::Tree, compression_level: Option, background_tranquility: u32, @@ -102,7 +100,7 @@ pub struct BlockManager { resync_notify: Notify, resync_errors: CountedTree, - system: Arc, + pub(crate) system: Arc, endpoint: Arc>, metrics: BlockManagerMetrics, @@ -147,10 +145,6 @@ impl BlockManager { let resync_errors = CountedTree::new(resync_errors).expect("Could not count block_local_resync_errors"); - let state_variables_store = db - .open_tree("state_variables") - .expect("Unable to open state_variables tree"); - let endpoint = system .netapp .endpoint("garage_block/manager.rs/Rpc".to_string()); @@ -169,7 +163,6 @@ impl BlockManager { resync_queue, resync_notify: Notify::new(), resync_errors, - state_variables_store, system, endpoint, metrics, @@ -514,7 +507,7 @@ impl BlockManager { // 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, 4); + let scrub_worker = ScrubWorker::new(self.clone(), scrub_rx); self.system.background.spawn_worker(scrub_worker); } diff --git a/src/block/repair.rs b/src/block/repair.rs index 8335de51..27ed05c2 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -1,10 +1,10 @@ use core::ops::Bound; -use std::convert::TryInto; use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; +use serde::{Deserialize, Serialize}; use tokio::fs; use tokio::select; use tokio::sync::mpsc; @@ -13,13 +13,13 @@ use tokio::sync::watch; use garage_util::background::*; use garage_util::data::*; use garage_util::error::*; +use garage_util::persister::Persister; use garage_util::time::*; use garage_util::tranquilizer::Tranquilizer; use crate::manager::*; const SCRUB_INTERVAL: Duration = Duration::from_secs(3600 * 24 * 30); // full scrub every 30 days -const TIME_LAST_COMPLETE_SCRUB: &[u8] = b"time_last_complete_scrub"; pub struct RepairWorker { manager: Arc, @@ -139,8 +139,14 @@ pub struct ScrubWorker { work: ScrubWorkerState, tranquilizer: Tranquilizer, - tranquility: u32, + persister: Persister, + persisted: ScrubWorkerPersisted, +} + +#[derive(Serialize, Deserialize)] +struct ScrubWorkerPersisted { + tranquility: u32, time_last_complete_scrub: u64, } @@ -156,6 +162,7 @@ impl Default for ScrubWorkerState { } } +#[derive(Debug)] pub enum ScrubWorkerCommand { Start, Pause(Duration), @@ -165,30 +172,26 @@ pub enum ScrubWorkerCommand { } impl ScrubWorker { - pub fn new( - manager: Arc, - rx_cmd: mpsc::Receiver, - tranquility: u32, - ) -> Self { - let time_last_complete_scrub = match manager - .state_variables_store - .get(TIME_LAST_COMPLETE_SCRUB) - .expect("DB error when initializing scrub worker") - { - Some(v) => u64::from_be_bytes(v.try_into().unwrap()), - None => 0, + pub fn new(manager: Arc, rx_cmd: mpsc::Receiver) -> Self { + let persister = Persister::new(&manager.system.metadata_dir, "scrub_info"); + let persisted = match persister.load() { + Ok(v) => v, + Err(_) => ScrubWorkerPersisted { + time_last_complete_scrub: 0, + tranquility: 4, + }, }; Self { manager, rx_cmd, work: ScrubWorkerState::Finished, tranquilizer: Tranquilizer::new(30), - tranquility, - time_last_complete_scrub, + persister, + persisted, } } - fn handle_cmd(&mut self, cmd: ScrubWorkerCommand) { + async fn handle_cmd(&mut self, cmd: ScrubWorkerCommand) { match cmd { ScrubWorkerCommand::Start => { self.work = match std::mem::take(&mut self.work) { @@ -234,7 +237,10 @@ impl ScrubWorker { } } ScrubWorkerCommand::SetTranquility(t) => { - self.tranquility = t; + self.persisted.tranquility = t; + if let Err(e) = self.persister.save_async(&self.persisted).await { + error!("Could not save new tranquilitiy value: {}", e); + } } } } @@ -248,13 +254,17 @@ impl Worker for ScrubWorker { fn info(&self) -> Option { match &self.work { - ScrubWorkerState::Running(bsi) => Some(format!("{:.2}% done", bsi.progress() * 100.)), + ScrubWorkerState::Running(bsi) => Some(format!( + "{:.2}% done (tranquility = {})", + bsi.progress() * 100., + self.persisted.tranquility + )), ScrubWorkerState::Paused(_bsi, rt) => { Some(format!("Paused, resumes at {}", msec_to_rfc3339(*rt))) } ScrubWorkerState::Finished => Some(format!( "Last completed scrub: {}", - msec_to_rfc3339(self.time_last_complete_scrub) + msec_to_rfc3339(self.persisted.time_last_complete_scrub) )), } } @@ -264,7 +274,7 @@ impl Worker for ScrubWorker { _must_exit: &mut watch::Receiver, ) -> Result { match self.rx_cmd.try_recv() { - Ok(cmd) => self.handle_cmd(cmd), + Ok(cmd) => self.handle_cmd(cmd).await, Err(mpsc::error::TryRecvError::Disconnected) => return Ok(WorkerStatus::Done), Err(mpsc::error::TryRecvError::Empty) => (), }; @@ -274,13 +284,12 @@ impl Worker for ScrubWorker { self.tranquilizer.reset(); if let Some(hash) = bsi.next().await? { let _ = self.manager.read_block(&hash).await; - Ok(self.tranquilizer.tranquilize_worker(self.tranquility)) + Ok(self + .tranquilizer + .tranquilize_worker(self.persisted.tranquility)) } else { - self.time_last_complete_scrub = now_msec(); // TODO save to file - self.manager.state_variables_store.insert( - TIME_LAST_COMPLETE_SCRUB, - u64::to_be_bytes(self.time_last_complete_scrub), - )?; + self.persisted.time_last_complete_scrub = now_msec(); + self.persister.save_async(&self.persisted).await?; self.work = ScrubWorkerState::Finished; self.tranquilizer.clear(); Ok(WorkerStatus::Idle) @@ -294,23 +303,35 @@ impl Worker for ScrubWorker { match &self.work { ScrubWorkerState::Running(_) => return WorkerStatus::Busy, ScrubWorkerState::Paused(_, resume_time) => { - let delay = Duration::from_millis(resume_time - now_msec()); + let now = now_msec(); + if now >= *resume_time { + self.handle_cmd(ScrubWorkerCommand::Resume).await; + return WorkerStatus::Busy; + } + let delay = Duration::from_millis(*resume_time - now); select! { - _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Resume), + _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Resume).await, cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { - self.handle_cmd(cmd); + self.handle_cmd(cmd).await; } else { return WorkerStatus::Done; } } } ScrubWorkerState::Finished => { + let now = now_msec(); + if now - self.persisted.time_last_complete_scrub + >= SCRUB_INTERVAL.as_millis() as u64 + { + self.handle_cmd(ScrubWorkerCommand::Start).await; + return WorkerStatus::Busy; + } let delay = SCRUB_INTERVAL - - Duration::from_secs(now_msec() - self.time_last_complete_scrub); + - Duration::from_millis(now - self.persisted.time_last_complete_scrub); select! { - _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Start), + _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Start).await, cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { - self.handle_cmd(cmd); + self.handle_cmd(cmd).await; } else { return WorkerStatus::Done; } diff --git a/src/garage/cli/util.rs b/src/garage/cli/util.rs index fc5a9932..8be56138 100644 --- a/src/garage/cli/util.rs +++ b/src/garage/cli/util.rs @@ -263,7 +263,7 @@ pub fn print_worker_info(wi: HashMap, wlo: WorkerListOpt) { continue; } - table.push(format!("{}\t{:?}\t{}", tid, info.status, info.name)); + table.push(format!("{}\t{}\t{}", tid, info.status, info.name)); if let Some(i) = &info.info { table.push(format!("\t\t {}", i)); } diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index 8207a8b4..d4366486 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -46,7 +46,6 @@ pub async fn launch_online_repair(garage: Arc, opt: RepairOpt) { )); } RepairWhat::Scrub { cmd } => { - info!("Verifying integrity of stored blocks"); let cmd = match cmd { ScrubCmd::Start => ScrubWorkerCommand::Start, ScrubCmd::Pause => ScrubWorkerCommand::Pause(Duration::from_secs(3600 * 24)), @@ -56,6 +55,7 @@ pub async fn launch_online_repair(garage: Arc, opt: RepairOpt) { ScrubWorkerCommand::SetTranquility(tranquility) } }; + info!("Sending command to scrub worker: {:?}", cmd); garage.block_manager.send_scrub_command(cmd).await; } } diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 1d7c3ea4..77b79864 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -2,7 +2,7 @@ use std::collections::HashMap; use std::io::{Read, Write}; use std::net::{IpAddr, SocketAddr}; -use std::path::Path; +use std::path::{Path, PathBuf}; use std::sync::{Arc, RwLock}; use std::time::{Duration, Instant}; @@ -104,6 +104,9 @@ pub struct System { /// The job runner of this node pub background: Arc, + + /// Path to metadata directory (usefull) + pub metadata_dir: PathBuf, } #[derive(Debug, Clone, Serialize, Deserialize)] @@ -295,6 +298,7 @@ impl System { ring, update_ring: Mutex::new(update_ring), background, + metadata_dir: config.metadata_dir.clone(), }); sys.system_endpoint.set_handler(sys.clone()); sys diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index c08a0aaa..7fd63c2b 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -15,7 +15,7 @@ use crate::background::WorkerInfo; use crate::error::Error; use crate::time::now_msec; -#[derive(PartialEq, Copy, Clone, Debug, Serialize, Deserialize)] +#[derive(PartialEq, Copy, Clone, Serialize, Deserialize, Debug)] pub enum WorkerStatus { Busy, Throttled(f32), @@ -23,6 +23,17 @@ pub enum WorkerStatus { Done, } +impl std::fmt::Display for WorkerStatus { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + WorkerStatus::Busy => write!(f, "Busy"), + WorkerStatus::Throttled(t) => write!(f, "Thr:{:.3}", t), + WorkerStatus::Idle => write!(f, "Idle"), + WorkerStatus::Done => write!(f, "Done"), + } + } +} + #[async_trait] pub trait Worker: Send { fn name(&self) -> String; -- 2.43.0 From b0a181e17ede60db7a00bb42c22a4cb2c3eb3312 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Jun 2022 17:10:59 +0200 Subject: [PATCH 23/33] Count corruptions in scrub worker --- src/block/repair.rs | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/src/block/repair.rs b/src/block/repair.rs index 27ed05c2..136a4c0d 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -148,6 +148,7 @@ pub struct ScrubWorker { struct ScrubWorkerPersisted { tranquility: u32, time_last_complete_scrub: u64, + corruptions_detected: u64, } enum ScrubWorkerState { @@ -179,6 +180,7 @@ impl ScrubWorker { Err(_) => ScrubWorkerPersisted { time_last_complete_scrub: 0, tranquility: 4, + corruptions_detected: 0, }, }; Self { @@ -253,20 +255,21 @@ impl Worker for ScrubWorker { } fn info(&self) -> Option { - match &self.work { - ScrubWorkerState::Running(bsi) => Some(format!( + let s = match &self.work { + ScrubWorkerState::Running(bsi) => format!( "{:.2}% done (tranquility = {})", bsi.progress() * 100., self.persisted.tranquility - )), + ), ScrubWorkerState::Paused(_bsi, rt) => { - Some(format!("Paused, resumes at {}", msec_to_rfc3339(*rt))) + format!("Paused, resumes at {}", msec_to_rfc3339(*rt)) } - ScrubWorkerState::Finished => Some(format!( + ScrubWorkerState::Finished => format!( "Last completed scrub: {}", msec_to_rfc3339(self.persisted.time_last_complete_scrub) - )), - } + ), + }; + Some(format!("{} ; corruptions detected: {}", s, self.persisted.corruptions_detected)) } async fn work( @@ -283,7 +286,15 @@ impl Worker for ScrubWorker { ScrubWorkerState::Running(bsi) => { self.tranquilizer.reset(); if let Some(hash) = bsi.next().await? { - let _ = self.manager.read_block(&hash).await; + match self.manager.read_block(&hash).await { + Err(Error::CorruptData(_)) => { + error!("Found corrupt data block during scrub: {:?}", hash); + self.persisted.corruptions_detected += 1; + self.persister.save_async(&self.persisted).await?; + } + Err(e) => return Err(e), + _ => (), + }; Ok(self .tranquilizer .tranquilize_worker(self.persisted.tranquility)) -- 2.43.0 From fc2bc8b0cac3c407ea0aa4bb004ea8c0a15a9120 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Jun 2022 17:31:23 +0200 Subject: [PATCH 24/33] cargo fmt --- src/block/repair.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/block/repair.rs b/src/block/repair.rs index 136a4c0d..f1df49e0 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -269,7 +269,10 @@ impl Worker for ScrubWorker { msec_to_rfc3339(self.persisted.time_last_complete_scrub) ), }; - Some(format!("{} ; corruptions detected: {}", s, self.persisted.corruptions_detected)) + Some(format!( + "{} ; corruptions detected: {}", + s, self.persisted.corruptions_detected + )) } async fn work( -- 2.43.0 From 368414a261345088b30c57e87ee29a29026672a8 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Jun 2022 17:42:54 +0200 Subject: [PATCH 25/33] Small things --- src/block/repair.rs | 3 +-- src/garage/repair/online.rs | 8 ++++++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/block/repair.rs b/src/block/repair.rs index f1df49e0..284a8846 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -381,7 +381,7 @@ impl BlockStoreIterator { } } - /// Returns progress done, between 0% and 1% + /// Returns progress done, between 0 and 1 fn progress(&self) -> f32 { if self.path.is_empty() { 1.0 @@ -453,7 +453,6 @@ impl BlockStoreIterator { if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() { let path = data_dir_ent.path(); self.path.push(ReadingDir::Pending(path)); - continue; } else if name.len() == 64 { let hash_bytes = if let Ok(h) = hex::decode(&name) { h diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index d4366486..f0d4b847 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -85,6 +85,10 @@ impl Worker for RepairVersionsWorker { "Version repair worker".into() } + fn info(&self) -> Option { + Some(format!("{} items done", self.counter)) + } + async fn work( &mut self, _must_exit: &mut watch::Receiver, @@ -165,6 +169,10 @@ impl Worker for RepairBlockrefsWorker { "Block refs repair worker".into() } + fn info(&self) -> Option { + Some(format!("{} items done", self.counter)) + } + async fn work( &mut self, _must_exit: &mut watch::Receiver, -- 2.43.0 From c4be56704df81c257b07e8b83170f46346908013 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Jun 2022 17:43:56 +0200 Subject: [PATCH 26/33] remove useless info!() --- src/garage/repair/online.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index f0d4b847..eeb9cea3 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -105,9 +105,6 @@ impl Worker for RepairVersionsWorker { }; self.counter += 1; - if self.counter % 1000 == 0 { - info!("repair_versions: {}", self.counter); - } let version = rmp_serde::decode::from_read_ref::<_, Version>(&item_bytes)?; if !version.deleted.get() { @@ -189,9 +186,6 @@ impl Worker for RepairBlockrefsWorker { }; self.counter += 1; - if self.counter % 1000 == 0 { - info!("repair_block_ref: {}", self.counter); - } let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(&item_bytes)?; if !block_ref.deleted.get() { -- 2.43.0 From 5ee9cb7768f27182325ae9b943f39c12e69a95d6 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 4 Jul 2022 12:53:47 +0200 Subject: [PATCH 27/33] add delays in k2v test_items_and_indices --- src/garage/tests/k2v/item.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/garage/tests/k2v/item.rs b/src/garage/tests/k2v/item.rs index bf2b01f8..32537336 100644 --- a/src/garage/tests/k2v/item.rs +++ b/src/garage/tests/k2v/item.rs @@ -1,3 +1,5 @@ +use std::time::Duration; + use crate::common; use assert_json_diff::assert_json_eq; @@ -86,6 +88,7 @@ async fn test_items_and_indices() { assert_eq!(res_body, content); // ReadIndex -- now there should be some stuff + tokio::time::sleep(Duration::from_secs(1)).await; let res = ctx .k2v .request @@ -154,6 +157,7 @@ async fn test_items_and_indices() { assert_eq!(res_body, content2); // ReadIndex -- now there should be some stuff + tokio::time::sleep(Duration::from_secs(1)).await; let res = ctx .k2v .request @@ -222,6 +226,7 @@ async fn test_items_and_indices() { ); // ReadIndex -- now there should be some stuff + tokio::time::sleep(Duration::from_secs(1)).await; let res = ctx .k2v .request @@ -290,6 +295,7 @@ async fn test_items_and_indices() { assert_eq!(res.status(), 204); // ReadIndex -- now there should be some stuff + tokio::time::sleep(Duration::from_secs(1)).await; let res = ctx .k2v .request -- 2.43.0 From 0200eae679cbb04f846f173a9b7b256ed697f500 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Jul 2022 09:55:27 +0200 Subject: [PATCH 28/33] cleanup --- src/block/manager.rs | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 36166ae3..17d4a72d 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -494,11 +494,7 @@ impl BlockManager { fn spawn_background_workers(self: Arc) { // Launch a background workers for background resync loop processing let background = self.system.background.clone(); - let worker = ResyncWorker { - manager: self.clone(), - tranquilizer: Tranquilizer::new(30), - next_delay: Duration::from_secs(10), - }; + let worker = ResyncWorker::new(self.clone()); tokio::spawn(async move { tokio::time::sleep(Duration::from_secs(10)).await; background.spawn_worker(worker); @@ -744,6 +740,16 @@ struct ResyncWorker { 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 { -- 2.43.0 From 4cc9a648abbc5434aee17f568525499340b5df3d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Jul 2022 10:10:30 +0200 Subject: [PATCH 29/33] Small refactoring --- src/block/repair.rs | 64 +++++++++++++++++++-------------------------- 1 file changed, 27 insertions(+), 37 deletions(-) diff --git a/src/block/repair.rs b/src/block/repair.rs index 284a8846..2c21cfd6 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -261,8 +261,12 @@ impl Worker for ScrubWorker { bsi.progress() * 100., self.persisted.tranquility ), - ScrubWorkerState::Paused(_bsi, rt) => { - format!("Paused, resumes at {}", msec_to_rfc3339(*rt)) + ScrubWorkerState::Paused(bsi, rt) => { + format!( + "Paused, {:.2}% done, resumes at {}", + bsi.progress() * 100., + msec_to_rfc3339(*rt) + ) } ScrubWorkerState::Finished => format!( "Last completed scrub: {}", @@ -314,44 +318,30 @@ impl Worker for ScrubWorker { } async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { - match &self.work { + let (wait_until, command) = match &self.work { ScrubWorkerState::Running(_) => return WorkerStatus::Busy, - ScrubWorkerState::Paused(_, resume_time) => { - let now = now_msec(); - if now >= *resume_time { - self.handle_cmd(ScrubWorkerCommand::Resume).await; - return WorkerStatus::Busy; - } - let delay = Duration::from_millis(*resume_time - now); - select! { - _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Resume).await, - cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { - self.handle_cmd(cmd).await; - } else { - return WorkerStatus::Done; - } - } - } - ScrubWorkerState::Finished => { - let now = now_msec(); - if now - self.persisted.time_last_complete_scrub - >= SCRUB_INTERVAL.as_millis() as u64 - { - self.handle_cmd(ScrubWorkerCommand::Start).await; - return WorkerStatus::Busy; - } - let delay = SCRUB_INTERVAL - - Duration::from_millis(now - self.persisted.time_last_complete_scrub); - select! { - _ = tokio::time::sleep(delay) => self.handle_cmd(ScrubWorkerCommand::Start).await, - cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { - self.handle_cmd(cmd).await; - } else { - return WorkerStatus::Done; - } - } + ScrubWorkerState::Paused(_, resume_time) => (*resume_time, ScrubWorkerCommand::Resume), + ScrubWorkerState::Finished => ( + self.persisted.time_last_complete_scrub + SCRUB_INTERVAL.as_millis() as u64, + ScrubWorkerCommand::Start, + ), + }; + + let now = now_msec(); + if now >= wait_until { + self.handle_cmd(command).await; + return WorkerStatus::Busy; + } + let delay = Duration::from_millis(wait_until - now); + select! { + _ = tokio::time::sleep(delay) => self.handle_cmd(command).await, + cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { + self.handle_cmd(cmd).await; + } else { + return WorkerStatus::Done; } } + match &self.work { ScrubWorkerState::Running(_) => WorkerStatus::Busy, _ => WorkerStatus::Idle, -- 2.43.0 From 4312623930efda3099474569c05a086ef1d02998 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Jul 2022 10:16:46 +0200 Subject: [PATCH 30/33] whoops and small refactoring --- src/block/repair.rs | 18 +++++------------- 1 file changed, 5 insertions(+), 13 deletions(-) diff --git a/src/block/repair.rs b/src/block/repair.rs index 2c21cfd6..26d1bd8f 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -415,11 +415,6 @@ impl BlockStoreIterator { ReadingDir::Pending(_) => unreachable!(), }; - if *pos >= subpaths.len() { - self.path.pop(); - continue; - } - let data_dir_ent = match subpaths.get(*pos) { None => { self.path.pop(); @@ -444,14 +439,11 @@ impl BlockStoreIterator { let path = data_dir_ent.path(); self.path.push(ReadingDir::Pending(path)); } else if name.len() == 64 { - let hash_bytes = if let Ok(h) = hex::decode(&name) { - h - } else { - continue; - }; - let mut hash = [0u8; 32]; - hash.copy_from_slice(&hash_bytes[..]); - return Ok(Some(hash.into())); + if let Ok(h) = hex::decode(&name) { + let mut hash = [0u8; 32]; + hash.copy_from_slice(&h); + return Ok(Some(hash.into())); + } } } } -- 2.43.0 From 0f660b086c23d13c91b0c55fd4d43017a09c1f4b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Jul 2022 10:32:41 +0200 Subject: [PATCH 31/33] fix comments --- src/rpc/system.rs | 2 +- src/util/background/worker.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 77b79864..f9f2970b 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -105,7 +105,7 @@ pub struct System { /// The job runner of this node pub background: Arc, - /// Path to metadata directory (usefull) + /// Path to metadata directory pub metadata_dir: PathBuf, } diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index 7fd63c2b..aadc677f 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -47,7 +47,7 @@ pub trait Worker: Send { /// middle of processing, it will only be interrupted at the last minute when Garage is trying /// to exit and this hasn't returned yet. This function may return an error to indicate that /// its unit of work could not be processed due to an error: the error will be logged and - /// .work() will be called again immediately. + /// .work() will be called again after a short delay. async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result; /// Wait for work: await for some task to become available. This future can be interrupted in -- 2.43.0 From d1cf1a0fa6952e84874f36f1dc66e4a978959d8f Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Jul 2022 10:39:41 +0200 Subject: [PATCH 32/33] Rename WorkerStatus to WorkerState because it's a state in a state machine --- src/block/manager.rs | 10 +++--- src/block/repair.rs | 32 +++++++++--------- src/garage/cli/util.rs | 12 +++---- src/garage/repair/online.rs | 16 ++++----- src/model/index_counter.rs | 18 +++++------ src/table/gc.rs | 12 +++---- src/table/merkle.rs | 18 +++++------ src/table/sync.rs | 14 ++++---- src/util/background/job_worker.rs | 12 +++---- src/util/background/mod.rs | 4 +-- src/util/background/worker.rs | 54 +++++++++++++++---------------- src/util/tranquilizer.rs | 8 ++--- 12 files changed, 105 insertions(+), 105 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 17d4a72d..e54fb992 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -776,16 +776,16 @@ impl Worker for ResyncWorker { async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { self.tranquilizer.reset(); match self.manager.resync_iter().await { Ok(ResyncIterResult::BusyDidSomething) => Ok(self .tranquilizer .tranquilize_worker(self.manager.background_tranquility)), - Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerStatus::Busy), + Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerState::Busy), Ok(ResyncIterResult::IdleFor(delay)) => { self.next_delay = delay; - Ok(WorkerStatus::Idle) + Ok(WorkerState::Idle) } Err(e) => { // The errors that we have here are only Sled errors @@ -799,12 +799,12 @@ impl Worker for ResyncWorker { } } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { select! { _ = tokio::time::sleep(self.next_delay) => (), _ = self.manager.resync_notify.notified() => (), }; - WorkerStatus::Busy + WorkerState::Busy } } diff --git a/src/block/repair.rs b/src/block/repair.rs index 26d1bd8f..cd5afe44 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -65,7 +65,7 @@ impl Worker for RepairWorker { async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { match self.block_iter.as_mut() { None => { // Phase 1: Repair blocks from RC table. @@ -101,7 +101,7 @@ impl Worker for RepairWorker { if batch_of_hashes.is_empty() { // move on to phase 2 self.block_iter = Some(BlockStoreIterator::new(&self.manager)); - return Ok(WorkerStatus::Busy); + return Ok(WorkerState::Busy); } for hash in batch_of_hashes.into_iter() { @@ -109,7 +109,7 @@ impl Worker for RepairWorker { self.next_start = Some(hash) } - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) } Some(bi) => { // Phase 2: Repair blocks actually on disk @@ -118,15 +118,15 @@ impl Worker for RepairWorker { // 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))?; - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) } else { - Ok(WorkerStatus::Done) + Ok(WorkerState::Done) } } } } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { unreachable!() } } @@ -282,10 +282,10 @@ impl Worker for ScrubWorker { async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { match self.rx_cmd.try_recv() { Ok(cmd) => self.handle_cmd(cmd).await, - Err(mpsc::error::TryRecvError::Disconnected) => return Ok(WorkerStatus::Done), + Err(mpsc::error::TryRecvError::Disconnected) => return Ok(WorkerState::Done), Err(mpsc::error::TryRecvError::Empty) => (), }; @@ -310,16 +310,16 @@ impl Worker for ScrubWorker { self.persister.save_async(&self.persisted).await?; self.work = ScrubWorkerState::Finished; self.tranquilizer.clear(); - Ok(WorkerStatus::Idle) + Ok(WorkerState::Idle) } } - _ => Ok(WorkerStatus::Idle), + _ => Ok(WorkerState::Idle), } } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { let (wait_until, command) = match &self.work { - ScrubWorkerState::Running(_) => return WorkerStatus::Busy, + ScrubWorkerState::Running(_) => return WorkerState::Busy, ScrubWorkerState::Paused(_, resume_time) => (*resume_time, ScrubWorkerCommand::Resume), ScrubWorkerState::Finished => ( self.persisted.time_last_complete_scrub + SCRUB_INTERVAL.as_millis() as u64, @@ -330,7 +330,7 @@ impl Worker for ScrubWorker { let now = now_msec(); if now >= wait_until { self.handle_cmd(command).await; - return WorkerStatus::Busy; + return WorkerState::Busy; } let delay = Duration::from_millis(wait_until - now); select! { @@ -338,13 +338,13 @@ impl Worker for ScrubWorker { cmd = self.rx_cmd.recv() => if let Some(cmd) = cmd { self.handle_cmd(cmd).await; } else { - return WorkerStatus::Done; + return WorkerState::Done; } } match &self.work { - ScrubWorkerState::Running(_) => WorkerStatus::Busy, - _ => WorkerStatus::Idle, + ScrubWorkerState::Running(_) => WorkerState::Busy, + _ => WorkerState::Idle, } } } diff --git a/src/garage/cli/util.rs b/src/garage/cli/util.rs index 8be56138..396938ae 100644 --- a/src/garage/cli/util.rs +++ b/src/garage/cli/util.rs @@ -245,10 +245,10 @@ pub fn print_worker_info(wi: HashMap, wlo: WorkerListOpt) { let mut wi = wi.into_iter().collect::>(); wi.sort_by_key(|(tid, info)| { ( - match info.status { - WorkerStatus::Busy | WorkerStatus::Throttled(_) => 0, - WorkerStatus::Idle => 1, - WorkerStatus::Done => 2, + match info.state { + WorkerState::Busy | WorkerState::Throttled(_) => 0, + WorkerState::Idle => 1, + WorkerState::Done => 2, }, *tid, ) @@ -256,14 +256,14 @@ pub fn print_worker_info(wi: HashMap, wlo: WorkerListOpt) { let mut table = vec![]; for (tid, info) in wi.iter() { - if wlo.busy && !matches!(info.status, WorkerStatus::Busy | WorkerStatus::Throttled(_)) { + if wlo.busy && !matches!(info.state, WorkerState::Busy | WorkerState::Throttled(_)) { continue; } if wlo.errors && info.errors == 0 { continue; } - table.push(format!("{}\t{}\t{}", tid, info.status, info.name)); + table.push(format!("{}\t{}\t{}", tid, info.state, info.name)); if let Some(i) = &info.info { table.push(format!("\t\t {}", i)); } diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index eeb9cea3..160ce8f8 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -92,7 +92,7 @@ impl Worker for RepairVersionsWorker { async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { let item_bytes = match self.garage.version_table.data.store.get_gt(&self.pos)? { Some((k, v)) => { self.pos = k; @@ -100,7 +100,7 @@ impl Worker for RepairVersionsWorker { } None => { info!("repair_versions: finished, done {}", self.counter); - return Ok(WorkerStatus::Done); + return Ok(WorkerState::Done); } }; @@ -134,10 +134,10 @@ impl Worker for RepairVersionsWorker { } } - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { unreachable!() } } @@ -173,7 +173,7 @@ impl Worker for RepairBlockrefsWorker { async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { let item_bytes = match self.garage.block_ref_table.data.store.get_gt(&self.pos)? { Some((k, v)) => { self.pos = k; @@ -181,7 +181,7 @@ impl Worker for RepairBlockrefsWorker { } None => { info!("repair_block_ref: finished, done {}", self.counter); - return Ok(WorkerStatus::Done); + return Ok(WorkerState::Done); } }; @@ -212,10 +212,10 @@ impl Worker for RepairBlockrefsWorker { } } - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { unreachable!() } } diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index 9d5aa955..26833390 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -415,7 +415,7 @@ impl Worker for IndexPropagatorWorker { } } - async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { + async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { // This loop batches updates to counters to be sent all at once. // They are sent once the propagate_rx channel has been emptied (or is closed). let closed = loop { @@ -435,7 +435,7 @@ impl Worker for IndexPropagatorWorker { self.errors += 1; if self.errors >= 2 && *must_exit.borrow() { error!("({}) Could not propagate {} counter values: {}, these counters will not be updated correctly.", T::COUNTER_TABLE_NAME, self.buf.len(), e); - return Ok(WorkerStatus::Done); + return Ok(WorkerState::Done); } // Propagate error up to worker manager, it will log it, increment a counter, // and sleep for a certain delay (with exponential backoff), waiting for @@ -448,23 +448,23 @@ impl Worker for IndexPropagatorWorker { self.errors = 0; } - return Ok(WorkerStatus::Busy); + return Ok(WorkerState::Busy); } else if closed { - return Ok(WorkerStatus::Done); + return Ok(WorkerState::Done); } else { - return Ok(WorkerStatus::Idle); + return Ok(WorkerState::Idle); } } - async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, _must_exit: &watch::Receiver) -> WorkerState { match self.propagate_rx.recv().await { Some((pk, sk, counters)) => { self.add_ent(pk, sk, counters); - WorkerStatus::Busy + WorkerState::Busy } None => match self.buf.is_empty() { - false => WorkerStatus::Busy, - true => WorkerStatus::Done, + false => WorkerState::Busy, + true => WorkerState::Done, }, } } diff --git a/src/table/gc.rs b/src/table/gc.rs index 0899d5e5..9ffae184 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -347,22 +347,22 @@ where async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { match self.gc.gc_loop_iter().await? { - None => Ok(WorkerStatus::Busy), + None => Ok(WorkerState::Busy), Some(delay) => { self.wait_delay = delay; - Ok(WorkerStatus::Idle) + Ok(WorkerState::Idle) } } } - async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerState { if *must_exit.borrow() { - return WorkerStatus::Done; + return WorkerState::Done; } tokio::time::sleep(self.wait_delay).await; - WorkerStatus::Busy + WorkerState::Busy } } diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 21186220..ca5891a7 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -82,12 +82,12 @@ where ret } - fn updater_loop_iter(&self) -> Result { + fn updater_loop_iter(&self) -> Result { if let Some((key, valhash)) = self.data.merkle_todo.first()? { self.update_item(&key, &valhash)?; - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) } else { - Ok(WorkerStatus::Idle) + Ok(WorkerState::Idle) } } @@ -325,27 +325,27 @@ where async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { let updater = self.0.clone(); tokio::task::spawn_blocking(move || { for _i in 0..100 { let s = updater.updater_loop_iter(); - if !matches!(s, Ok(WorkerStatus::Busy)) { + if !matches!(s, Ok(WorkerState::Busy)) { return s; } } - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) }) .await .unwrap() } - async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerState { if *must_exit.borrow() { - return WorkerStatus::Done; + return WorkerState::Done; } tokio::time::sleep(Duration::from_secs(10)).await; - WorkerStatus::Busy + WorkerState::Busy } } diff --git a/src/table/sync.rs b/src/table/sync.rs index a7e1994c..b3756a5e 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -586,18 +586,18 @@ impl Worker for SyncWor } } - async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { + async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { if let Some(partition) = self.pop_task() { self.syncer.sync_partition(&partition, must_exit).await?; - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) } else { - Ok(WorkerStatus::Idle) + Ok(WorkerState::Idle) } } - async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerState { if *must_exit.borrow() { - return WorkerStatus::Done; + return WorkerState::Done; } select! { s = self.add_full_sync_rx.recv() => { @@ -619,8 +619,8 @@ impl Worker for SyncWor } } match self.todo.is_empty() { - false => WorkerStatus::Busy, - true => WorkerStatus::Idle, + false => WorkerState::Busy, + true => WorkerState::Idle, } } } diff --git a/src/util/background/job_worker.rs b/src/util/background/job_worker.rs index fcdac582..6754382a 100644 --- a/src/util/background/job_worker.rs +++ b/src/util/background/job_worker.rs @@ -24,17 +24,17 @@ impl Worker for JobWorker { async fn work( &mut self, _must_exit: &mut watch::Receiver, - ) -> Result { + ) -> Result { match self.next_job.take() { - None => return Ok(WorkerStatus::Idle), + None => return Ok(WorkerState::Idle), Some(job) => { job.await?; - Ok(WorkerStatus::Busy) + Ok(WorkerState::Busy) } } } - async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus { + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerState { loop { match self.job_chan.lock().await.recv().await { Some((job, cancellable)) => { @@ -42,9 +42,9 @@ impl Worker for JobWorker { continue; } self.next_job = Some(job); - return WorkerStatus::Busy; + return WorkerState::Busy; } - None => return WorkerStatus::Done, + None => return WorkerState::Done, } } } diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index 636b9c13..619f5068 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -14,7 +14,7 @@ use tokio::sync::{mpsc, watch, Mutex}; use crate::error::Error; use worker::WorkerProcessor; -pub use worker::{Worker, WorkerStatus}; +pub use worker::{Worker, WorkerState}; pub(crate) type JobOutput = Result<(), Error>; pub(crate) type Job = Pin + Send>>; @@ -30,7 +30,7 @@ pub struct BackgroundRunner { pub struct WorkerInfo { pub name: String, pub info: Option, - pub status: WorkerStatus, + pub state: WorkerState, pub errors: usize, pub consecutive_errors: usize, pub last_error: Option<(String, u64)>, diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index aadc677f..7f573a07 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -16,20 +16,20 @@ use crate::error::Error; use crate::time::now_msec; #[derive(PartialEq, Copy, Clone, Serialize, Deserialize, Debug)] -pub enum WorkerStatus { +pub enum WorkerState { Busy, Throttled(f32), Idle, Done, } -impl std::fmt::Display for WorkerStatus { +impl std::fmt::Display for WorkerState { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { - WorkerStatus::Busy => write!(f, "Busy"), - WorkerStatus::Throttled(t) => write!(f, "Thr:{:.3}", t), - WorkerStatus::Idle => write!(f, "Idle"), - WorkerStatus::Done => write!(f, "Done"), + WorkerState::Busy => write!(f, "Busy"), + WorkerState::Throttled(t) => write!(f, "Thr:{:.3}", t), + WorkerState::Idle => write!(f, "Idle"), + WorkerState::Done => write!(f, "Done"), } } } @@ -43,18 +43,18 @@ pub trait Worker: Send { } /// Work: do a basic unit of work, if one is available (otherwise, should return - /// WorkerStatus::Idle immediately). We will do our best to not interrupt this future in the + /// WorkerState::Idle immediately). We will do our best to not interrupt this future in the /// middle of processing, it will only be interrupted at the last minute when Garage is trying /// to exit and this hasn't returned yet. This function may return an error to indicate that /// its unit of work could not be processed due to an error: the error will be logged and /// .work() will be called again after a short delay. - async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result; + async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result; /// Wait for work: await for some task to become available. This future can be interrupted in /// the middle for any reason. This future doesn't have to await on must_exit.changed(), we /// are doing it for you. Therefore it only receives a read refernce to must_exit which allows /// it to check if we are exiting. - async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerStatus; + async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerState; } pub(crate) struct WorkerProcessor { @@ -100,7 +100,7 @@ impl WorkerProcessor { stop_signal, stop_signal_worker, worker: new_worker, - status: WorkerStatus::Busy, + state: WorkerState::Busy, errors: 0, consecutive_errors: 0, last_error: None, @@ -113,13 +113,13 @@ impl WorkerProcessor { } worker = await_next_worker => { if let Some(mut worker) = worker { - trace!("{} (TID {}): {:?}", worker.worker.name(), worker.task_id, worker.status); + trace!("{} (TID {}): {:?}", worker.worker.name(), worker.task_id, worker.state); // Save worker info let mut wi = self.worker_info.lock().unwrap(); match wi.get_mut(&worker.task_id) { Some(i) => { - i.status = worker.status; + i.state = worker.state; i.info = worker.worker.info(); i.errors = worker.errors; i.consecutive_errors = worker.consecutive_errors; @@ -130,7 +130,7 @@ impl WorkerProcessor { None => { wi.insert(worker.task_id, WorkerInfo { name: worker.worker.name(), - status: worker.status, + state: worker.state, info: worker.worker.info(), errors: worker.errors, consecutive_errors: worker.consecutive_errors, @@ -139,7 +139,7 @@ impl WorkerProcessor { } } - if worker.status == WorkerStatus::Done { + if worker.state == WorkerState::Done { info!("Worker {} (TID {}) exited", worker.worker.name(), worker.task_id); } else { workers.push(async move { @@ -157,14 +157,14 @@ impl WorkerProcessor { let drain_half_time = Instant::now() + Duration::from_secs(5); let drain_everything = async move { while let Some(mut worker) = workers.next().await { - if worker.status == WorkerStatus::Done { + if worker.state == WorkerState::Done { info!( "Worker {} (TID {}) exited", worker.worker.name(), worker.task_id ); } else if Instant::now() > drain_half_time { - warn!("Worker {} (TID {}) interrupted between two iterations in state {:?} (this should be fine)", worker.worker.name(), worker.task_id, worker.status); + warn!("Worker {} (TID {}) interrupted between two iterations in state {:?} (this should be fine)", worker.worker.name(), worker.task_id, worker.state); } else { workers.push( async move { @@ -193,7 +193,7 @@ struct WorkerHandler { stop_signal: watch::Receiver, stop_signal_worker: watch::Receiver, worker: Box, - status: WorkerStatus, + state: WorkerState, errors: usize, consecutive_errors: usize, last_error: Option<(String, u64)>, @@ -201,10 +201,10 @@ struct WorkerHandler { impl WorkerHandler { async fn step(&mut self) { - match self.status { - WorkerStatus::Busy => match self.worker.work(&mut self.stop_signal).await { + match self.state { + WorkerState::Busy => match self.worker.work(&mut self.stop_signal).await { Ok(s) => { - self.status = s; + self.state = s; self.consecutive_errors = 0; } Err(e) => { @@ -219,12 +219,12 @@ impl WorkerHandler { self.last_error = Some((format!("{}", e), now_msec())); // Sleep a bit so that error won't repeat immediately, exponential backoff // strategy (min 1sec, max ~60sec) - self.status = WorkerStatus::Throttled( + self.state = WorkerState::Throttled( (1.5f32).powf(std::cmp::min(10, self.consecutive_errors - 1) as f32), ); } }, - WorkerStatus::Throttled(delay) => { + WorkerState::Throttled(delay) => { // Sleep for given delay and go back to busy state if !*self.stop_signal.borrow() { select! { @@ -232,13 +232,13 @@ impl WorkerHandler { _ = self.stop_signal.changed() => (), } } - self.status = WorkerStatus::Busy; + self.state = WorkerState::Busy; } - WorkerStatus::Idle => { + WorkerState::Idle => { if *self.stop_signal.borrow() { select! { new_st = self.worker.wait_for_work(&self.stop_signal_worker) => { - self.status = new_st; + self.state = new_st; } _ = tokio::time::sleep(Duration::from_secs(1)) => { // stay in Idle state @@ -247,7 +247,7 @@ impl WorkerHandler { } else { select! { new_st = self.worker.wait_for_work(&self.stop_signal_worker) => { - self.status = new_st; + self.state = new_st; } _ = self.stop_signal.changed() => { // stay in Idle state @@ -255,7 +255,7 @@ impl WorkerHandler { } } } - WorkerStatus::Done => unreachable!(), + WorkerState::Done => unreachable!(), } } } diff --git a/src/util/tranquilizer.rs b/src/util/tranquilizer.rs index 9c796f8b..fdb2918b 100644 --- a/src/util/tranquilizer.rs +++ b/src/util/tranquilizer.rs @@ -3,7 +3,7 @@ use std::time::{Duration, Instant}; use tokio::time::sleep; -use crate::background::WorkerStatus; +use crate::background::WorkerState; /// A tranquilizer is a helper object that is used to make /// background operations not take up too much time. @@ -61,10 +61,10 @@ impl Tranquilizer { } #[must_use] - pub fn tranquilize_worker(&mut self, tranquility: u32) -> WorkerStatus { + pub fn tranquilize_worker(&mut self, tranquility: u32) -> WorkerState { match self.tranquilize_internal(tranquility) { - Some(delay) => WorkerStatus::Throttled(delay.as_secs_f32()), - None => WorkerStatus::Busy, + Some(delay) => WorkerState::Throttled(delay.as_secs_f32()), + None => WorkerState::Busy, } } -- 2.43.0 From 08cd5f2f1d4a42e63a256eb3a7811e874d222fe4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Jul 2022 10:45:20 +0200 Subject: [PATCH 33/33] cargo fmt --- Cargo.lock | 2 +- src/block/manager.rs | 5 +---- src/block/repair.rs | 10 ++-------- src/garage/repair/online.rs | 10 ++-------- src/table/gc.rs | 5 +---- src/table/merkle.rs | 5 +---- src/util/background/job_worker.rs | 5 +---- 7 files changed, 9 insertions(+), 33 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c45ee015..e1ccfc2d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1737,7 +1737,7 @@ dependencies = [ [[package]] name = "k2v-client" -version = "0.1.0" +version = "0.0.1" dependencies = [ "base64", "clap 3.1.18", diff --git a/src/block/manager.rs b/src/block/manager.rs index e54fb992..017ba9da 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -773,10 +773,7 @@ impl Worker for ResyncWorker { } } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + 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 diff --git a/src/block/repair.rs b/src/block/repair.rs index cd5afe44..07ff6772 100644 --- a/src/block/repair.rs +++ b/src/block/repair.rs @@ -62,10 +62,7 @@ impl Worker for RepairWorker { } } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { match self.block_iter.as_mut() { None => { // Phase 1: Repair blocks from RC table. @@ -279,10 +276,7 @@ impl Worker for ScrubWorker { )) } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { match self.rx_cmd.try_recv() { Ok(cmd) => self.handle_cmd(cmd).await, Err(mpsc::error::TryRecvError::Disconnected) => return Ok(WorkerState::Done), diff --git a/src/garage/repair/online.rs b/src/garage/repair/online.rs index 160ce8f8..e33cf097 100644 --- a/src/garage/repair/online.rs +++ b/src/garage/repair/online.rs @@ -89,10 +89,7 @@ impl Worker for RepairVersionsWorker { Some(format!("{} items done", self.counter)) } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { let item_bytes = match self.garage.version_table.data.store.get_gt(&self.pos)? { Some((k, v)) => { self.pos = k; @@ -170,10 +167,7 @@ impl Worker for RepairBlockrefsWorker { Some(format!("{} items done", self.counter)) } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { let item_bytes = match self.garage.block_ref_table.data.store.get_gt(&self.pos)? { Some((k, v)) => { self.pos = k; diff --git a/src/table/gc.rs b/src/table/gc.rs index 9ffae184..12218d97 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -344,10 +344,7 @@ where } } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { match self.gc.gc_loop_iter().await? { None => Ok(WorkerState::Busy), Some(delay) => { diff --git a/src/table/merkle.rs b/src/table/merkle.rs index ca5891a7..a5c29723 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -322,10 +322,7 @@ where } } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { let updater = self.0.clone(); tokio::task::spawn_blocking(move || { for _i in 0..100 { diff --git a/src/util/background/job_worker.rs b/src/util/background/job_worker.rs index 6754382a..2568ea11 100644 --- a/src/util/background/job_worker.rs +++ b/src/util/background/job_worker.rs @@ -21,10 +21,7 @@ impl Worker for JobWorker { format!("Job worker #{}", self.index) } - async fn work( - &mut self, - _must_exit: &mut watch::Receiver, - ) -> Result { + async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { match self.next_job.take() { None => return Ok(WorkerState::Idle), Some(job) => { -- 2.43.0