aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorAlex Auvolat <alex@adnab.me>2022-09-06 22:13:01 +0200
committerAlex Auvolat <alex@adnab.me>2022-09-06 22:13:01 +0200
commit6b958979bd898a576ee9c1021cc908b3ec75ffc7 (patch)
tree72e35a3b12c616965bcdf60bf18d1c0db29955ba
parent4024822585783368993ac26807d076d8c312bb35 (diff)
parentd23b3a14fc28de164080e762f0e97e6cbc868940 (diff)
downloadgarage-6b958979bd898a576ee9c1021cc908b3ec75ffc7.tar.gz
garage-6b958979bd898a576ee9c1021cc908b3ec75ffc7.zip
Merge branch 'lx-perf-improvements' into netapp-stream-body
-rw-r--r--Cargo.nix6
-rw-r--r--Makefile2
-rw-r--r--src/block/lib.rs1
-rw-r--r--src/block/manager.rs567
-rw-r--r--src/block/repair.rs34
-rw-r--r--src/block/resync.rs599
-rw-r--r--src/garage/admin.rs32
-rw-r--r--src/garage/cli/structs.rs19
-rw-r--r--src/model/garage.rs1
-rw-r--r--src/rpc/Cargo.toml2
-rw-r--r--src/util/config.rs7
11 files changed, 735 insertions, 535 deletions
diff --git a/Cargo.nix b/Cargo.nix
index 9877f729..1c5ac952 100644
--- a/Cargo.nix
+++ b/Cargo.nix
@@ -4649,7 +4649,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 == "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 "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"; };
@@ -5754,8 +5754,8 @@ in
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-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-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 == "i686-pc-windows-msvc" || hostPlatform.config == "i686-uwp-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-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; };
};
});
diff --git a/Makefile b/Makefile
index 1f0f3644..23e10f78 100644
--- a/Makefile
+++ b/Makefile
@@ -1,7 +1,7 @@
.PHONY: doc all release shell run1 run2 run3
all:
- clear; cargo build --all-features
+ clear; cargo build
release:
nix-build --arg release true
diff --git a/src/block/lib.rs b/src/block/lib.rs
index ebdb95d8..d2814f77 100644
--- a/src/block/lib.rs
+++ b/src/block/lib.rs
@@ -3,6 +3,7 @@ extern crate tracing;
pub mod manager;
pub mod repair;
+pub mod resync;
mod block;
mod metrics;
diff --git a/src/block/manager.rs b/src/block/manager.rs
index 66a454b0..b9cd09e7 100644
--- a/src/block/manager.rs
+++ b/src/block/manager.rs
@@ -1,10 +1,8 @@
-use std::convert::TryInto;
use std::path::PathBuf;
use std::pin::Pin;
use std::sync::Arc;
use std::time::Duration;
-use arc_swap::ArcSwapOption;
use async_trait::async_trait;
use bytes::Bytes;
use serde::{Deserialize, Serialize};
@@ -13,25 +11,20 @@ use futures::Stream;
use futures_util::stream::StreamExt;
use tokio::fs;
use tokio::io::{AsyncReadExt, AsyncWriteExt, BufReader};
-use tokio::select;
-use tokio::sync::{mpsc, watch, Mutex, Notify};
+use tokio::sync::{mpsc, Mutex};
use opentelemetry::{
trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
- Context, KeyValue,
+ Context,
};
use garage_rpc::rpc_helper::netapp::stream::{stream_asyncread, ByteStream};
use garage_db as db;
-use garage_db::counted_tree_hack::CountedTree;
-use garage_util::background::*;
use garage_util::data::*;
use garage_util::error::*;
use garage_util::metrics::RecordDuration;
-use garage_util::time::*;
-use garage_util::tranquilizer::Tranquilizer;
use garage_rpc::rpc_helper::OrderTag;
use garage_rpc::system::System;
@@ -43,27 +36,13 @@ use crate::block::*;
use crate::metrics::*;
use crate::rc::*;
use crate::repair::*;
+use crate::resync::*;
/// Size under which data will be stored inlined in database instead of as files
pub const INLINE_THRESHOLD: usize = 3072;
// Timeout for RPCs that read and write blocks to remote nodes
-const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(60);
-// Timeout for RPCs that ask other nodes whether they need a copy
-// of a given block before we delete it locally
-// The timeout here is relatively low because we don't want to block
-// the entire resync loop when some nodes are not responding.
-// Nothing will be deleted if the nodes don't answer the queries,
-// we will just retry later.
-const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(15);
-
-// The delay between the time where a resync operation fails
-// and the time when it is retried, with exponential backoff
-// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure).
-const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60);
-// The minimum retry delay is 60 seconds = 1 minute
-// The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour)
-const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6;
+pub(crate) const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(60);
// The delay between the moment when the reference counter
// drops to zero, and the moment where we allow ourselves
@@ -100,22 +79,18 @@ pub struct BlockManager {
pub data_dir: PathBuf,
compression_level: Option<i32>,
- background_tranquility: u32,
mutation_lock: [Mutex<BlockManagerLocked>; 256],
pub(crate) rc: BlockRc,
-
- resync_queue: CountedTree,
- resync_notify: Notify,
- resync_errors: CountedTree,
+ pub resync: BlockResyncManager,
pub(crate) system: Arc<System>,
- endpoint: Arc<Endpoint<BlockRpc, Self>>,
+ pub(crate) endpoint: Arc<Endpoint<BlockRpc, Self>>,
- metrics: BlockManagerMetrics,
+ pub(crate) metrics: BlockManagerMetrics,
- tx_scrub_command: ArcSwapOption<mpsc::Sender<ScrubWorkerCommand>>,
+ tx_scrub_command: mpsc::Sender<ScrubWorkerCommand>,
}
// This custom struct contains functions that must only be ran
@@ -123,18 +98,11 @@ pub struct BlockManager {
// it INSIDE a Mutex.
struct BlockManagerLocked();
-enum ResyncIterResult {
- BusyDidSomething,
- BusyDidNothing,
- IdleFor(Duration),
-}
-
impl BlockManager {
pub fn new(
db: &db::Db,
data_dir: PathBuf,
compression_level: Option<i32>,
- background_tranquility: u32,
replication: TableShardedReplication,
system: Arc<System>,
) -> Arc<Self> {
@@ -143,42 +111,39 @@ impl BlockManager {
.expect("Unable to open block_local_rc tree");
let rc = BlockRc::new(rc);
- let resync_queue = db
- .open_tree("block_local_resync_queue")
- .expect("Unable to open block_local_resync_queue tree");
- let resync_queue =
- CountedTree::new(resync_queue).expect("Could not count block_local_resync_queue");
-
- let resync_errors = db
- .open_tree("block_local_resync_errors")
- .expect("Unable to open block_local_resync_errors tree");
- let resync_errors =
- CountedTree::new(resync_errors).expect("Could not count block_local_resync_errors");
+ let resync = BlockResyncManager::new(db, &system);
let endpoint = system
.netapp
.endpoint("garage_block/manager.rs/Rpc".to_string());
- let metrics = BlockManagerMetrics::new(resync_queue.clone(), resync_errors.clone());
+ let metrics = BlockManagerMetrics::new(resync.queue.clone(), resync.errors.clone());
+
+ let (scrub_tx, scrub_rx) = mpsc::channel(1);
let block_manager = Arc::new(Self {
replication,
data_dir,
compression_level,
- background_tranquility,
mutation_lock: [(); 256].map(|_| Mutex::new(BlockManagerLocked())),
rc,
- resync_queue,
- resync_notify: Notify::new(),
- resync_errors,
+ resync,
system,
endpoint,
metrics,
- tx_scrub_command: ArcSwapOption::new(None),
+ tx_scrub_command: scrub_tx,
});
block_manager.endpoint.set_handler(block_manager.clone());
- block_manager.clone().spawn_background_workers();
+ // Spawn a bunch of resync workers
+ for index in 0..MAX_RESYNC_WORKERS {
+ let worker = ResyncWorker::new(index, block_manager.clone());
+ block_manager.system.background.spawn_worker(worker);
+ }
+
+ // Spawn scrub worker
+ let scrub_worker = ScrubWorker::new(block_manager.clone(), scrub_rx);
+ block_manager.system.background.spawn_worker(scrub_worker);
block_manager
}
@@ -232,7 +197,7 @@ impl BlockManager {
/// Ask nodes that might have a (possibly compressed) block for it
/// Return its entire body
- async fn rpc_get_raw_block(
+ pub(crate) async fn rpc_get_raw_block(
&self,
hash: &Hash,
order_tag: Option<OrderTag>,
@@ -343,20 +308,6 @@ impl BlockManager {
Ok(())
}
- /// Get lenght of resync queue
- pub fn resync_queue_len(&self) -> Result<usize, Error> {
- // This currently can't return an error because the CountedTree hack
- // doesn't error on .len(), but this will change when we remove the hack
- // (hopefully someday!)
- Ok(self.resync_queue.len())
- }
-
- /// Get number of blocks that have an error
- pub fn resync_errors_len(&self) -> Result<usize, Error> {
- // (see resync_queue_len comment)
- Ok(self.resync_errors.len())
- }
-
/// Get number of items in the refcount table
pub fn rc_len(&self) -> Result<usize, Error> {
Ok(self.rc.rc.len()?)
@@ -364,13 +315,7 @@ impl BlockManager {
/// Send command to start/stop/manager scrub worker
pub async fn send_scrub_command(&self, cmd: ScrubWorkerCommand) {
- let _ = self
- .tx_scrub_command
- .load()
- .as_ref()
- .unwrap()
- .send(cmd)
- .await;
+ let _ = self.tx_scrub_command.send(cmd).await;
}
//// ----- Managing the reference counter ----
@@ -391,7 +336,7 @@ impl BlockManager {
// we will fecth it from someone.
let this = self.clone();
tokio::spawn(async move {
- if let Err(e) = this.put_to_resync(&hash, 2 * BLOCK_RW_TIMEOUT) {
+ if let Err(e) = this.resync.put_to_resync(&hash, 2 * BLOCK_RW_TIMEOUT) {
error!("Block {:?} could not be put in resync queue: {}.", hash, e);
}
});
@@ -413,7 +358,9 @@ impl BlockManager {
// after that delay has passed.
let this = self.clone();
tokio::spawn(async move {
- if let Err(e) = this.put_to_resync(&hash, BLOCK_GC_DELAY + Duration::from_secs(10))
+ if let Err(e) = this
+ .resync
+ .put_to_resync(&hash, BLOCK_GC_DELAY + Duration::from_secs(10))
{
error!("Block {:?} could not be put in resync queue: {}.", hash, e);
}
@@ -437,7 +384,7 @@ impl BlockManager {
}
/// Write a block to disk
- async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<(), Error> {
+ pub(crate) async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<(), Error> {
let tracer = opentelemetry::global::tracer("garage");
let write_size = data.inner_buffer().len() as u64;
@@ -501,7 +448,7 @@ impl BlockManager {
Ok(c) => c,
Err(e) => {
// Not found but maybe we should have had it ??
- self.put_to_resync(hash, 2 * BLOCK_RW_TIMEOUT)?;
+ self.resync.put_to_resync(hash, 2 * BLOCK_RW_TIMEOUT)?;
return Err(Into::into(e));
}
};
@@ -528,23 +475,37 @@ impl BlockManager {
.await
.move_block_to_corrupted(hash, self)
.await?;
- self.put_to_resync(hash, Duration::from_millis(0))?;
+ self.resync.put_to_resync(hash, Duration::from_millis(0))?;
return Err(Error::CorruptData(*hash));
}
Ok(data)
}
- /// Check if this node should have a block, but don't actually have it
- async fn need_block(&self, hash: &Hash) -> Result<bool, Error> {
- let BlockStatus { exists, needed } = self.mutation_lock[hash.as_slice()[0] as usize]
+ /// Check if this node has a block and whether it needs it
+ pub(crate) async fn check_block_status(&self, hash: &Hash) -> Result<BlockStatus, Error> {
+ self.mutation_lock[hash.as_slice()[0] as usize]
.lock()
.await
.check_block_status(hash, self)
- .await?;
+ .await
+ }
+
+ /// Check if this node should have a block, but don't actually have it
+ async fn need_block(&self, hash: &Hash) -> Result<bool, Error> {
+ let BlockStatus { exists, needed } = self.check_block_status(hash).await?;
Ok(needed.is_nonzero() && !exists)
}
+ /// Delete block if it is not needed anymore
+ pub(crate) async fn delete_if_unneeded(&self, hash: &Hash) -> Result<(), Error> {
+ self.mutation_lock[hash.as_slice()[0] as usize]
+ .lock()
+ .await
+ .delete_if_unneeded(hash, self)
+ .await
+ }
+
/// Utility: gives the path of the directory in which a block should be found
fn block_dir(&self, hash: &Hash) -> PathBuf {
let mut path = self.data_dir.clone();
@@ -571,310 +532,6 @@ impl BlockManager {
path.set_extension("");
fs::metadata(&path).await.map(|_| false).map_err(Into::into)
}
-
- // ---- Resync loop ----
-
- // This part manages a queue of blocks that need to be
- // "resynchronized", i.e. that need to have a check that
- // they are at present if we need them, or that they are
- // deleted once the garbage collection delay has passed.
- //
- // Here are some explanations on how the resync queue works.
- // There are two Sled trees that are used to have information
- // about the status of blocks that need to be resynchronized:
- //
- // - resync_queue: a tree that is ordered first by a timestamp
- // (in milliseconds since Unix epoch) that is the time at which
- // the resync must be done, and second by block hash.
- // The key in this tree is just:
- // concat(timestamp (8 bytes), hash (32 bytes))
- // The value is the same 32-byte hash.
- //
- // - resync_errors: a tree that indicates for each block
- // if the last resync resulted in an error, and if so,
- // the following two informations (see the ErrorCounter struct):
- // - how many consecutive resync errors for this block?
- // - when was the last try?
- // These two informations are used to implement an
- // exponential backoff retry strategy.
- // The key in this tree is the 32-byte hash of the block,
- // and the value is the encoded ErrorCounter value.
- //
- // We need to have these two trees, because the resync queue
- // is not just a queue of items to process, but a set of items
- // that are waiting a specific delay until we can process them
- // (the delay being necessary both internally for the exponential
- // backoff strategy, and exposed as a parameter when adding items
- // to the queue, e.g. to wait until the GC delay has passed).
- // This is why we need one tree ordered by time, and one
- // ordered by identifier of item to be processed (block hash).
- //
- // When the worker wants to process an item it takes from
- // resync_queue, it checks in resync_errors that if there is an
- // exponential back-off delay to await, it has passed before we
- // process the item. If not, the item in the queue is skipped
- // (but added back for later processing after the time of the
- // delay).
- //
- // An alternative that would have seemed natural is to
- // only add items to resync_queue with a processing time that is
- // after the delay, but there are several issues with this:
- // - This requires to synchronize updates to resync_queue and
- // resync_errors (with the current model, there is only one thread,
- // the worker thread, that accesses resync_errors,
- // so no need to synchronize) by putting them both in a lock.
- // This would mean that block_incref might need to take a lock
- // before doing its thing, meaning it has much more chances of
- // not completing successfully if something bad happens to Garage.
- // Currently Garage is not able to recover from block_incref that
- // doesn't complete successfully, because it is necessary to ensure
- // the consistency between the state of the block manager and
- // information in the BlockRef table.
- // - If a resync fails, we put that block in the resync_errors table,
- // and also add it back to resync_queue to be processed after
- // the exponential back-off delay,
- // but maybe the block is already scheduled to be resynced again
- // at another time that is before the exponential back-off delay,
- // and we have no way to check that easily. This means that
- // in all cases, we need to check the resync_errors table
- // in the resync loop at the time when a block is popped from
- // the resync_queue.
- // Overall, the current design is therefore simpler and more robust
- // because it tolerates inconsistencies between the resync_queue
- // and resync_errors table (items being scheduled in resync_queue
- // for times that are earlier than the exponential back-off delay
- // is a natural condition that is handled properly).
-
- fn spawn_background_workers(self: Arc<Self>) {
- // Launch a background workers for background resync loop processing
- let background = self.system.background.clone();
- let worker = ResyncWorker::new(self.clone());
- tokio::spawn(async move {
- tokio::time::sleep(Duration::from_secs(10)).await;
- background.spawn_worker(worker);
- });
-
- // Launch a background worker for data store scrubs
- let (scrub_tx, scrub_rx) = mpsc::channel(1);
- self.tx_scrub_command.store(Some(Arc::new(scrub_tx)));
- let scrub_worker = ScrubWorker::new(self.clone(), scrub_rx);
- self.system.background.spawn_worker(scrub_worker);
- }
-
- pub(crate) fn put_to_resync(&self, hash: &Hash, delay: Duration) -> db::Result<()> {
- let when = now_msec() + delay.as_millis() as u64;
- self.put_to_resync_at(hash, when)
- }
-
- fn put_to_resync_at(&self, hash: &Hash, when: u64) -> db::Result<()> {
- trace!("Put resync_queue: {} {:?}", when, hash);
- let mut key = u64::to_be_bytes(when).to_vec();
- key.extend(hash.as_ref());
- self.resync_queue.insert(key, hash.as_ref())?;
- self.resync_notify.notify_waiters();
- Ok(())
- }
-
- async fn resync_iter(&self) -> Result<ResyncIterResult, db::Error> {
- if let Some((time_bytes, hash_bytes)) = self.resync_queue.first()? {
- let time_msec = u64::from_be_bytes(time_bytes[0..8].try_into().unwrap());
- let now = now_msec();
-
- if now >= time_msec {
- let hash = Hash::try_from(&hash_bytes[..]).unwrap();
-
- if let Some(ec) = self.resync_errors.get(hash.as_slice())? {
- let ec = ErrorCounter::decode(&ec);
- if now < ec.next_try() {
- // if next retry after an error is not yet,
- // don't do resync and return early, but still
- // make sure the item is still in queue at expected time
- self.put_to_resync_at(&hash, ec.next_try())?;
- // ec.next_try() > now >= time_msec, so this remove
- // is not removing the one we added just above
- // (we want to do the remove after the insert to ensure
- // that the item is not lost if we crash in-between)
- self.resync_queue.remove(time_bytes)?;
- return Ok(ResyncIterResult::BusyDidNothing);
- }
- }
-
- let tracer = opentelemetry::global::tracer("garage");
- let trace_id = gen_uuid();
- let span = tracer
- .span_builder("Resync block")
- .with_trace_id(
- opentelemetry::trace::TraceId::from_hex(&hex::encode(
- &trace_id.as_slice()[..16],
- ))
- .unwrap(),
- )
- .with_attributes(vec![KeyValue::new("block", format!("{:?}", hash))])
- .start(&tracer);
-
- let res = self
- .resync_block(&hash)
- .with_context(Context::current_with_span(span))
- .bound_record_duration(&self.metrics.resync_duration)
- .await;
-
- self.metrics.resync_counter.add(1);
-
- if let Err(e) = &res {
- self.metrics.resync_error_counter.add(1);
- warn!("Error when resyncing {:?}: {}", hash, e);
-
- let err_counter = match self.resync_errors.get(hash.as_slice())? {
- Some(ec) => ErrorCounter::decode(&ec).add1(now + 1),
- None => ErrorCounter::new(now + 1),
- };
-
- self.resync_errors
- .insert(hash.as_slice(), err_counter.encode())?;
-
- self.put_to_resync_at(&hash, err_counter.next_try())?;
- // err_counter.next_try() >= now + 1 > now,
- // the entry we remove from the queue is not
- // the entry we inserted with put_to_resync_at
- self.resync_queue.remove(time_bytes)?;
- } else {
- self.resync_errors.remove(hash.as_slice())?;
- self.resync_queue.remove(time_bytes)?;
- }
-
- Ok(ResyncIterResult::BusyDidSomething)
- } else {
- Ok(ResyncIterResult::IdleFor(Duration::from_millis(
- time_msec - now,
- )))
- }
- } else {
- // Here we wait either for a notification that an item has been
- // added to the queue, or for a constant delay of 10 secs to expire.
- // The delay avoids a race condition where the notification happens
- // between the time we checked the queue and the first poll
- // to resync_notify.notified(): if that happens, we'll just loop
- // back 10 seconds later, which is fine.
- Ok(ResyncIterResult::IdleFor(Duration::from_secs(10)))
- }
- }
-
- async fn resync_block(&self, hash: &Hash) -> Result<(), Error> {
- let BlockStatus { exists, needed } = self.mutation_lock[hash.as_slice()[0] as usize]
- .lock()
- .await
- .check_block_status(hash, self)
- .await?;
-
- if exists != needed.is_needed() || exists != needed.is_nonzero() {
- debug!(
- "Resync block {:?}: exists {}, nonzero rc {}, deletable {}",
- hash,
- exists,
- needed.is_nonzero(),
- needed.is_deletable(),
- );
- }
-
- if exists && needed.is_deletable() {
- info!("Resync block {:?}: offloading and deleting", hash);
-
- let mut who = self.replication.write_nodes(hash);
- if who.len() < self.replication.write_quorum() {
- return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string()));
- }
- who.retain(|id| *id != self.system.id);
-
- let who_needs_resps = self
- .system
- .rpc
- .call_many(
- &self.endpoint,
- &who,
- BlockRpc::NeedBlockQuery(*hash),
- RequestStrategy::with_priority(PRIO_BACKGROUND)
- .with_timeout(NEED_BLOCK_QUERY_TIMEOUT),
- )
- .await?;
-
- let mut need_nodes = vec![];
- for (node, needed) in who_needs_resps.into_iter() {
- match needed.err_context("NeedBlockQuery RPC")? {
- BlockRpc::NeedBlockReply(needed) => {
- if needed {
- need_nodes.push(node);
- }
- }
- m => {
- return Err(Error::unexpected_rpc_message(m));
- }
- }
- }
-
- if !need_nodes.is_empty() {
- trace!(
- "Block {:?} needed by {} nodes, sending",
- hash,
- need_nodes.len()
- );
-
- for node in need_nodes.iter() {
- self.metrics
- .resync_send_counter
- .add(1, &[KeyValue::new("to", format!("{:?}", node))]);
- }
-
- let block = self.read_block(hash).await?;
- let (header, bytes) = block.into_parts();
- let put_block_message = Req::new(BlockRpc::PutBlock {
- hash: *hash,
- header,
- })?
- .with_stream_from_buffer(bytes);
- self.system
- .rpc
- .try_call_many(
- &self.endpoint,
- &need_nodes[..],
- put_block_message,
- RequestStrategy::with_priority(PRIO_BACKGROUND)
- .with_quorum(need_nodes.len())
- .with_timeout(BLOCK_RW_TIMEOUT),
- )
- .await
- .err_context("PutBlock RPC")?;
- }
- info!(
- "Deleting unneeded block {:?}, offload finished ({} / {})",
- hash,
- need_nodes.len(),
- who.len()
- );
-
- self.mutation_lock[hash.as_slice()[0] as usize]
- .lock()
- .await
- .delete_if_unneeded(hash, self)
- .await?;
-
- self.rc.clear_deleted_block_rc(hash)?;
- }
-
- if needed.is_nonzero() && !exists {
- info!(
- "Resync block {:?}: fetching absent but needed block (refcount > 0)",
- hash
- );
-
- let block_data = self.rpc_get_raw_block(hash, None).await?;
-
- self.metrics.resync_recv_counter.add(1);
-
- self.write_block(hash, &block_data).await?;
- }
-
- Ok(())
- }
}
#[async_trait]
@@ -895,80 +552,9 @@ impl StreamingEndpointHandler<BlockRpc> for BlockManager {
}
}
-struct ResyncWorker {
- manager: Arc<BlockManager>,
- tranquilizer: Tranquilizer,
- next_delay: Duration,
-}
-
-impl ResyncWorker {
- fn new(manager: Arc<BlockManager>) -> Self {
- Self {
- manager,
- tranquilizer: Tranquilizer::new(30),
- next_delay: Duration::from_secs(10),
- }
- }
-}
-
-#[async_trait]
-impl Worker for ResyncWorker {
- fn name(&self) -> String {
- "Block resync worker".into()
- }
-
- fn info(&self) -> Option<String> {
- 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.is_empty() {
- Some(ret.join(", "))
- } else {
- None
- }
- }
-
- async fn work(&mut self, _must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error> {
- 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(WorkerState::Busy),
- Ok(ResyncIterResult::IdleFor(delay)) => {
- self.next_delay = delay;
- Ok(WorkerState::Idle)
- }
- Err(e) => {
- // The errors that we have here are only Sled errors
- // We don't really know how to handle them so just ¯\_(ツ)_/¯
- // (there is kind of an assumption that Sled won't error on us,
- // if it does there is not much we can do -- TODO should we just panic?)
- // Here we just give the error to the worker manager,
- // it will print it to the logs and increment a counter
- Err(e.into())
- }
- }
- }
-
- async fn wait_for_work(&mut self, _must_exit: &watch::Receiver<bool>) -> WorkerState {
- select! {
- _ = tokio::time::sleep(self.next_delay) => (),
- _ = self.manager.resync_notify.notified() => (),
- };
- WorkerState::Busy
- }
-}
-
-struct BlockStatus {
- exists: bool,
- needed: RcEntry,
+pub(crate) struct BlockStatus {
+ pub(crate) exists: bool,
+ pub(crate) needed: RcEntry,
}
impl BlockManagerLocked {
@@ -1074,53 +660,6 @@ impl BlockManagerLocked {
}
}
-/// Counts the number of errors when resyncing a block,
-/// and the time of the last try.
-/// Used to implement exponential backoff.
-#[derive(Clone, Copy, Debug)]
-struct ErrorCounter {
- errors: u64,
- last_try: u64,
-}
-
-impl ErrorCounter {
- fn new(now: u64) -> Self {
- Self {
- errors: 1,
- last_try: now,
- }
- }
-
- fn decode(data: &[u8]) -> Self {
- Self {
- errors: u64::from_be_bytes(data[0..8].try_into().unwrap()),
- last_try: u64::from_be_bytes(data[8..16].try_into().unwrap()),
- }
- }
- fn encode(&self) -> Vec<u8> {
- [
- u64::to_be_bytes(self.errors),
- u64::to_be_bytes(self.last_try),
- ]
- .concat()
- }
-
- fn add1(self, now: u64) -> Self {
- Self {
- errors: self.errors + 1,
- last_try: now,
- }
- }
-
- fn delay_msec(&self) -> u64 {
- (RESYNC_RETRY_DELAY.as_millis() as u64)
- << std::cmp::min(self.errors - 1, RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER)
- }
- fn next_try(&self) -> u64 {
- self.last_try + self.delay_msec()
- }
-}
-
async fn read_stream_to_end(mut stream: ByteStream) -> Result<Bytes, Error> {
let mut parts: Vec<Bytes> = vec![];
while let Some(part) = stream.next().await {
diff --git a/src/block/repair.rs b/src/block/repair.rs
index 07ff6772..e2884b69 100644
--- a/src/block/repair.rs
+++ b/src/block/repair.rs
@@ -19,7 +19,17 @@ use garage_util::tranquilizer::Tranquilizer;
use crate::manager::*;
-const SCRUB_INTERVAL: Duration = Duration::from_secs(3600 * 24 * 30); // full scrub every 30 days
+// Full scrub every 30 days
+const SCRUB_INTERVAL: Duration = Duration::from_secs(3600 * 24 * 30);
+// Scrub tranquility is initially set to 4, but can be changed in the CLI
+// and the updated version is persisted over Garage restarts
+const INITIAL_SCRUB_TRANQUILITY: u32 = 4;
+
+// ---- ---- ----
+// FIRST KIND OF REPAIR: FINDING MISSING BLOCKS/USELESS BLOCKS
+// This is a one-shot repair operation that can be launched,
+// checks everything, and then exits.
+// ---- ---- ----
pub struct RepairWorker {
manager: Arc<BlockManager>,
@@ -102,7 +112,9 @@ impl Worker for RepairWorker {
}
for hash in batch_of_hashes.into_iter() {
- self.manager.put_to_resync(&hash, Duration::from_secs(0))?;
+ self.manager
+ .resync
+ .put_to_resync(&hash, Duration::from_secs(0))?;
self.next_start = Some(hash)
}
@@ -114,7 +126,9 @@ impl Worker for RepairWorker {
// This allows us to find blocks we are storing but don't actually need,
// so that we can offload them if necessary and then delete them locally.
if let Some(hash) = bi.next().await? {
- self.manager.put_to_resync(&hash, Duration::from_secs(0))?;
+ self.manager
+ .resync
+ .put_to_resync(&hash, Duration::from_secs(0))?;
Ok(WorkerState::Busy)
} else {
Ok(WorkerState::Done)
@@ -128,7 +142,13 @@ impl Worker for RepairWorker {
}
}
-// ----
+// ---- ---- ----
+// SECOND KIND OF REPAIR: SCRUBBING THE DATASTORE
+// This is significantly more complex than the process above,
+// as it is a continuously-running task that triggers automatically
+// every SCRUB_INTERVAL, but can also be triggered manually
+// and whose parameter (esp. speed) can be controlled at runtime.
+// ---- ---- ----
pub struct ScrubWorker {
manager: Arc<BlockManager>,
@@ -176,7 +196,7 @@ impl ScrubWorker {
Ok(v) => v,
Err(_) => ScrubWorkerPersisted {
time_last_complete_scrub: 0,
- tranquility: 4,
+ tranquility: INITIAL_SCRUB_TRANQUILITY,
corruptions_detected: 0,
},
};
@@ -343,7 +363,9 @@ impl Worker for ScrubWorker {
}
}
-// ----
+// ---- ---- ----
+// UTILITY FOR ENUMERATING THE BLOCK STORE
+// ---- ---- ----
struct BlockStoreIterator {
path: Vec<ReadingDir>,
diff --git a/src/block/resync.rs b/src/block/resync.rs
new file mode 100644
index 00000000..bde3e98c
--- /dev/null
+++ b/src/block/resync.rs
@@ -0,0 +1,599 @@
+use std::collections::HashSet;
+use std::convert::TryInto;
+use std::sync::{Arc, Mutex};
+use std::time::Duration;
+
+use arc_swap::ArcSwap;
+use async_trait::async_trait;
+use serde::{Deserialize, Serialize};
+
+use tokio::select;
+use tokio::sync::{watch, Notify};
+
+use opentelemetry::{
+ trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
+ Context, KeyValue,
+};
+
+use garage_db as db;
+use garage_db::counted_tree_hack::CountedTree;
+
+use garage_util::background::*;
+use garage_util::data::*;
+use garage_util::error::*;
+use garage_util::metrics::RecordDuration;
+use garage_util::persister::Persister;
+use garage_util::time::*;
+use garage_util::tranquilizer::Tranquilizer;
+
+use garage_rpc::system::System;
+use garage_rpc::*;
+
+use garage_table::replication::TableReplication;
+
+use crate::manager::*;
+
+// Timeout for RPCs that ask other nodes whether they need a copy
+// of a given block before we delete it locally
+// The timeout here is relatively low because we don't want to block
+// the entire resync loop when some nodes are not responding.
+// Nothing will be deleted if the nodes don't answer the queries,
+// we will just retry later.
+const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(15);
+
+// The delay between the time where a resync operation fails
+// and the time when it is retried, with exponential backoff
+// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure).
+pub(crate) const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60);
+// The minimum retry delay is 60 seconds = 1 minute
+// The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour)
+pub(crate) const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6;
+
+// No more than 4 resync workers can be running in the system
+pub(crate) const MAX_RESYNC_WORKERS: usize = 4;
+// Resync tranquility is initially set to 2, but can be changed in the CLI
+// and the updated version is persisted over Garage restarts
+const INITIAL_RESYNC_TRANQUILITY: u32 = 2;
+
+pub struct BlockResyncManager {
+ pub(crate) queue: CountedTree,
+ pub(crate) notify: Notify,
+ pub(crate) errors: CountedTree,
+
+ busy_set: BusySet,
+
+ persister: Persister<ResyncPersistedConfig>,
+ persisted: ArcSwap<ResyncPersistedConfig>,
+}
+
+#[derive(Serialize, Deserialize, Clone, Copy)]
+struct ResyncPersistedConfig {
+ n_workers: usize,
+ tranquility: u32,
+}
+
+enum ResyncIterResult {
+ BusyDidSomething,
+ BusyDidNothing,
+ IdleFor(Duration),
+}
+
+type BusySet = Arc<Mutex<HashSet<Vec<u8>>>>;
+
+struct BusyBlock {
+ time_bytes: Vec<u8>,
+ hash_bytes: Vec<u8>,
+ busy_set: BusySet,
+}
+
+impl BlockResyncManager {
+ pub(crate) fn new(db: &db::Db, system: &System) -> Self {
+ let queue = db
+ .open_tree("block_local_resync_queue")
+ .expect("Unable to open block_local_resync_queue tree");
+ let queue = CountedTree::new(queue).expect("Could not count block_local_resync_queue");
+
+ let errors = db
+ .open_tree("block_local_resync_errors")
+ .expect("Unable to open block_local_resync_errors tree");
+ let errors = CountedTree::new(errors).expect("Could not count block_local_resync_errors");
+
+ let persister = Persister::new(&system.metadata_dir, "resync_cfg");
+ let persisted = match persister.load() {
+ Ok(v) => v,
+ Err(_) => ResyncPersistedConfig {
+ n_workers: 1,
+ tranquility: INITIAL_RESYNC_TRANQUILITY,
+ },
+ };
+
+ Self {
+ queue,
+ notify: Notify::new(),
+ errors,
+ busy_set: Arc::new(Mutex::new(HashSet::new())),
+ persister,
+ persisted: ArcSwap::new(Arc::new(persisted)),
+ }
+ }
+
+ /// Get lenght of resync queue
+ pub fn queue_len(&self) -> Result<usize, Error> {
+ // This currently can't return an error because the CountedTree hack
+ // doesn't error on .len(), but this will change when we remove the hack
+ // (hopefully someday!)
+ Ok(self.queue.len())
+ }
+
+ /// Get number of blocks that have an error
+ pub fn errors_len(&self) -> Result<usize, Error> {
+ // (see queue_len comment)
+ Ok(self.errors.len())
+ }
+
+ // ---- Resync loop ----
+
+ // This part manages a queue of blocks that need to be
+ // "resynchronized", i.e. that need to have a check that
+ // they are at present if we need them, or that they are
+ // deleted once the garbage collection delay has passed.
+ //
+ // Here are some explanations on how the resync queue works.
+ // There are two Sled trees that are used to have information
+ // about the status of blocks that need to be resynchronized:
+ //
+ // - resync.queue: a tree that is ordered first by a timestamp
+ // (in milliseconds since Unix epoch) that is the time at which
+ // the resync must be done, and second by block hash.
+ // The key in this tree is just:
+ // concat(timestamp (8 bytes), hash (32 bytes))
+ // The value is the same 32-byte hash.
+ //
+ // - resync.errors: a tree that indicates for each block
+ // if the last resync resulted in an error, and if so,
+ // the following two informations (see the ErrorCounter struct):
+ // - how many consecutive resync errors for this block?
+ // - when was the last try?
+ // These two informations are used to implement an
+ // exponential backoff retry strategy.
+ // The key in this tree is the 32-byte hash of the block,
+ // and the value is the encoded ErrorCounter value.
+ //
+ // We need to have these two trees, because the resync queue
+ // is not just a queue of items to process, but a set of items
+ // that are waiting a specific delay until we can process them
+ // (the delay being necessary both internally for the exponential
+ // backoff strategy, and exposed as a parameter when adding items
+ // to the queue, e.g. to wait until the GC delay has passed).
+ // This is why we need one tree ordered by time, and one
+ // ordered by identifier of item to be processed (block hash).
+ //
+ // When the worker wants to process an item it takes from
+ // resync.queue, it checks in resync.errors that if there is an
+ // exponential back-off delay to await, it has passed before we
+ // process the item. If not, the item in the queue is skipped
+ // (but added back for later processing after the time of the
+ // delay).
+ //
+ // An alternative that would have seemed natural is to
+ // only add items to resync.queue with a processing time that is
+ // after the delay, but there are several issues with this:
+ // - This requires to synchronize updates to resync.queue and
+ // resync.errors (with the current model, there is only one thread,
+ // the worker thread, that accesses resync.errors,
+ // so no need to synchronize) by putting them both in a lock.
+ // This would mean that block_incref might need to take a lock
+ // before doing its thing, meaning it has much more chances of
+ // not completing successfully if something bad happens to Garage.
+ // Currently Garage is not able to recover from block_incref that
+ // doesn't complete successfully, because it is necessary to ensure
+ // the consistency between the state of the block manager and
+ // information in the BlockRef table.
+ // - If a resync fails, we put that block in the resync.errors table,
+ // and also add it back to resync.queue to be processed after
+ // the exponential back-off delay,
+ // but maybe the block is already scheduled to be resynced again
+ // at another time that is before the exponential back-off delay,
+ // and we have no way to check that easily. This means that
+ // in all cases, we need to check the resync.errors table
+ // in the resync loop at the time when a block is popped from
+ // the resync.queue.
+ // Overall, the current design is therefore simpler and more robust
+ // because it tolerates inconsistencies between the resync.queue
+ // and resync.errors table (items being scheduled in resync.queue
+ // for times that are earlier than the exponential back-off delay
+ // is a natural condition that is handled properly).
+
+ pub(crate) fn put_to_resync(&self, hash: &Hash, delay: Duration) -> db::Result<()> {
+ let when = now_msec() + delay.as_millis() as u64;
+ self.put_to_resync_at(hash, when)
+ }
+
+ pub(crate) fn put_to_resync_at(&self, hash: &Hash, when: u64) -> db::Result<()> {
+ trace!("Put resync_queue: {} {:?}", when, hash);
+ let mut key = u64::to_be_bytes(when).to_vec();
+ key.extend(hash.as_ref());
+ self.queue.insert(key, hash.as_ref())?;
+ self.notify.notify_waiters();
+ Ok(())
+ }
+
+ async fn resync_iter(&self, manager: &BlockManager) -> Result<ResyncIterResult, db::Error> {
+ if let Some(block) = self.get_block_to_resync()? {
+ let time_msec = u64::from_be_bytes(block.time_bytes[0..8].try_into().unwrap());
+ let now = now_msec();
+
+ if now >= time_msec {
+ let hash = Hash::try_from(&block.hash_bytes[..]).unwrap();
+
+ if let Some(ec) = self.errors.get(hash.as_slice())? {
+ let ec = ErrorCounter::decode(&ec);
+ if now < ec.next_try() {
+ // if next retry after an error is not yet,
+ // don't do resync and return early, but still
+ // make sure the item is still in queue at expected time
+ self.put_to_resync_at(&hash, ec.next_try())?;
+ // ec.next_try() > now >= time_msec, so this remove
+ // is not removing the one we added just above
+ // (we want to do the remove after the insert to ensure
+ // that the item is not lost if we crash in-between)
+ self.queue.remove(&block.time_bytes)?;
+ return Ok(ResyncIterResult::BusyDidNothing);
+ }
+ }
+
+ let tracer = opentelemetry::global::tracer("garage");
+ let trace_id = gen_uuid();
+ let span = tracer
+ .span_builder("Resync block")
+ .with_trace_id(
+ opentelemetry::trace::TraceId::from_hex(&hex::encode(
+ &trace_id.as_slice()[..16],
+ ))
+ .unwrap(),
+ )
+ .with_attributes(vec![KeyValue::new("block", format!("{:?}", hash))])
+ .start(&tracer);
+
+ let res = self
+ .resync_block(manager, &hash)
+ .with_context(Context::current_with_span(span))
+ .bound_record_duration(&manager.metrics.resync_duration)
+ .await;
+
+ manager.metrics.resync_counter.add(1);
+
+ if let Err(e) = &res {
+ manager.metrics.resync_error_counter.add(1);
+ warn!("Error when resyncing {:?}: {}", hash, e);
+
+ let err_counter = match self.errors.get(hash.as_slice())? {
+ Some(ec) => ErrorCounter::decode(&ec).add1(now + 1),
+ None => ErrorCounter::new(now + 1),
+ };
+
+ self.errors.insert(hash.as_slice(), err_counter.encode())?;
+
+ self.put_to_resync_at(&hash, err_counter.next_try())?;
+ // err_counter.next_try() >= now + 1 > now,
+ // the entry we remove from the queue is not
+ // the entry we inserted with put_to_resync_at
+ self.queue.remove(&block.time_bytes)?;
+ } else {
+ self.errors.remove(hash.as_slice())?;
+ self.queue.remove(&block.time_bytes)?;
+ }
+
+ Ok(ResyncIterResult::BusyDidSomething)
+ } else {
+ Ok(ResyncIterResult::IdleFor(Duration::from_millis(
+ time_msec - now,
+ )))
+ }
+ } else {
+ // Here we wait either for a notification that an item has been
+ // added to the queue, or for a constant delay of 10 secs to expire.
+ // The delay avoids a race condition where the notification happens
+ // between the time we checked the queue and the first poll
+ // to resync_notify.notified(): if that happens, we'll just loop
+ // back 10 seconds later, which is fine.
+ Ok(ResyncIterResult::IdleFor(Duration::from_secs(10)))
+ }
+ }
+
+ fn get_block_to_resync(&self) -> Result<Option<BusyBlock>, db::Error> {
+ let mut busy = self.busy_set.lock().unwrap();
+ for it in self.queue.iter()? {
+ let (time_bytes, hash_bytes) = it?;
+ if !busy.contains(&time_bytes) {
+ busy.insert(time_bytes.clone());
+ return Ok(Some(BusyBlock {
+ time_bytes,
+ hash_bytes,
+ busy_set: self.busy_set.clone(),
+ }));
+ }
+ }
+ Ok(None)
+ }
+
+ async fn resync_block(&self, manager: &BlockManager, hash: &Hash) -> Result<(), Error> {
+ let BlockStatus { exists, needed } = manager.check_block_status(hash).await?;
+
+ if exists != needed.is_needed() || exists != needed.is_nonzero() {
+ debug!(
+ "Resync block {:?}: exists {}, nonzero rc {}, deletable {}",
+ hash,
+ exists,
+ needed.is_nonzero(),
+ needed.is_deletable(),
+ );
+ }
+
+ if exists && needed.is_deletable() {
+ info!("Resync block {:?}: offloading and deleting", hash);
+
+ let mut who = manager.replication.write_nodes(hash);
+ if who.len() < manager.replication.write_quorum() {
+ return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string()));
+ }
+ who.retain(|id| *id != manager.system.id);
+
+ let who_needs_resps = manager
+ .system
+ .rpc
+ .call_many(
+ &manager.endpoint,
+ &who,
+ BlockRpc::NeedBlockQuery(*hash),
+ RequestStrategy::with_priority(PRIO_BACKGROUND)
+ .with_timeout(NEED_BLOCK_QUERY_TIMEOUT),
+ )
+ .await?;
+
+ let mut need_nodes = vec![];
+ for (node, needed) in who_needs_resps {
+ match needed.err_context("NeedBlockQuery RPC")? {
+ BlockRpc::NeedBlockReply(needed) => {
+ if needed {
+ need_nodes.push(node);
+ }
+ }
+ m => {
+ return Err(Error::unexpected_rpc_message(m));
+ }
+ }
+ }
+
+ if !need_nodes.is_empty() {
+ trace!(
+ "Block {:?} needed by {} nodes, sending",
+ hash,
+ need_nodes.len()
+ );
+
+ for node in need_nodes.iter() {
+ manager
+ .metrics
+ .resync_send_counter
+ .add(1, &[KeyValue::new("to", format!("{:?}", node))]);
+ }
+
+ let block = manager.read_block(hash).await?;
+ let (header, bytes) = block.into_parts();
+ let put_block_message = Req::new(BlockRpc::PutBlock {
+ hash: *hash,
+ header,
+ })?
+ .with_stream_from_buffer(bytes);
+ manager
+ .system
+ .rpc
+ .try_call_many(
+ &manager.endpoint,
+ &need_nodes[..],
+ put_block_message,
+ RequestStrategy::with_priority(PRIO_BACKGROUND)
+ .with_quorum(need_nodes.len())
+ .with_timeout(BLOCK_RW_TIMEOUT),
+ )
+ .await
+ .err_context("PutBlock RPC")?;
+ }
+ info!(
+ "Deleting unneeded block {:?}, offload finished ({} / {})",
+ hash,
+ need_nodes.len(),
+ who.len()
+ );
+
+ manager.delete_if_unneeded(hash).await?;
+
+ manager.rc.clear_deleted_block_rc(hash)?;
+ }
+
+ if needed.is_nonzero() && !exists {
+ info!(
+ "Resync block {:?}: fetching absent but needed block (refcount > 0)",
+ hash
+ );
+
+ let block_data = manager.rpc_get_raw_block(hash, None).await?;
+
+ manager.metrics.resync_recv_counter.add(1);
+
+ manager.write_block(hash, &block_data).await?;
+ }
+
+ Ok(())
+ }
+
+ async fn update_persisted(
+ &self,
+ update: impl Fn(&mut ResyncPersistedConfig),
+ ) -> Result<(), Error> {
+ let mut cfg: ResyncPersistedConfig = *self.persisted.load().as_ref();
+ update(&mut cfg);
+ self.persister.save_async(&cfg).await?;
+ self.persisted.store(Arc::new(cfg));
+ self.notify.notify_waiters();
+ Ok(())
+ }
+
+ pub async fn set_n_workers(&self, n_workers: usize) -> Result<(), Error> {
+ if !(1..=MAX_RESYNC_WORKERS).contains(&n_workers) {
+ return Err(Error::Message(format!(
+ "Invalid number of resync workers, must be between 1 and {}",
+ MAX_RESYNC_WORKERS
+ )));
+ }
+ self.update_persisted(|cfg| cfg.n_workers = n_workers).await
+ }
+
+ pub async fn set_tranquility(&self, tranquility: u32) -> Result<(), Error> {
+ self.update_persisted(|cfg| cfg.tranquility = tranquility)
+ .await
+ }
+}
+
+impl Drop for BusyBlock {
+ fn drop(&mut self) {
+ let mut busy = self.busy_set.lock().unwrap();
+ busy.remove(&self.time_bytes);
+ }
+}
+
+pub(crate) struct ResyncWorker {
+ index: usize,
+ manager: Arc<BlockManager>,
+ tranquilizer: Tranquilizer,
+ next_delay: Duration,
+}
+
+impl ResyncWorker {
+ pub(crate) fn new(index: usize, manager: Arc<BlockManager>) -> Self {
+ Self {
+ index,
+ manager,
+ tranquilizer: Tranquilizer::new(30),
+ next_delay: Duration::from_secs(10),
+ }
+ }
+}
+
+#[async_trait]
+impl Worker for ResyncWorker {
+ fn name(&self) -> String {
+ format!("Block resync worker #{}", self.index + 1)
+ }
+
+ fn info(&self) -> Option<String> {
+ let persisted = self.manager.resync.persisted.load();
+
+ if self.index >= persisted.n_workers {
+ return Some("(unused)".into());
+ }
+
+ let mut ret = vec![];
+ ret.push(format!("tranquility = {}", persisted.tranquility));
+
+ let qlen = self.manager.resync.queue_len().unwrap_or(0);
+ if qlen > 0 {
+ ret.push(format!("{} blocks in queue", qlen));
+ }
+
+ let elen = self.manager.resync.errors_len().unwrap_or(0);
+ if elen > 0 {
+ ret.push(format!("{} blocks in error state", elen));
+ }
+
+ Some(ret.join(", "))
+ }
+
+ async fn work(&mut self, _must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error> {
+ if self.index >= self.manager.resync.persisted.load().n_workers {
+ return Ok(WorkerState::Idle);
+ }
+
+ self.tranquilizer.reset();
+ match self.manager.resync.resync_iter(&self.manager).await {
+ Ok(ResyncIterResult::BusyDidSomething) => Ok(self
+ .tranquilizer
+ .tranquilize_worker(self.manager.resync.persisted.load().tranquility)),
+ Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerState::Busy),
+ Ok(ResyncIterResult::IdleFor(delay)) => {
+ self.next_delay = delay;
+ Ok(WorkerState::Idle)
+ }
+ Err(e) => {
+ // The errors that we have here are only Sled errors
+ // We don't really know how to handle them so just ¯\_(ツ)_/¯
+ // (there is kind of an assumption that Sled won't error on us,
+ // if it does there is not much we can do -- TODO should we just panic?)
+ // Here we just give the error to the worker manager,
+ // it will print it to the logs and increment a counter
+ Err(e.into())
+ }
+ }
+ }
+
+ async fn wait_for_work(&mut self, _must_exit: &watch::Receiver<bool>) -> WorkerState {
+ while self.index >= self.manager.resync.persisted.load().n_workers {
+ self.manager.resync.notify.notified().await
+ }
+
+ select! {
+ _ = tokio::time::sleep(self.next_delay) => (),
+ _ = self.manager.resync.notify.notified() => (),
+ };
+
+ WorkerState::Busy
+ }
+}
+
+/// Counts the number of errors when resyncing a block,
+/// and the time of the last try.
+/// Used to implement exponential backoff.
+#[derive(Clone, Copy, Debug)]
+struct ErrorCounter {
+ errors: u64,
+ last_try: u64,
+}
+
+impl ErrorCounter {
+ fn new(now: u64) -> Self {
+ Self {
+ errors: 1,
+ last_try: now,
+ }
+ }
+
+ fn decode(data: &[u8]) -> Self {
+ Self {
+ errors: u64::from_be_bytes(data[0..8].try_into().unwrap()),
+ last_try: u64::from_be_bytes(data[8..16].try_into().unwrap()),
+ }
+ }
+ fn encode(&self) -> Vec<u8> {
+ [
+ u64::to_be_bytes(self.errors),
+ u64::to_be_bytes(self.last_try),
+ ]
+ .concat()
+ }
+
+ fn add1(self, now: u64) -> Self {
+ Self {
+ errors: self.errors + 1,
+ last_try: now,
+ }
+ }
+
+ fn delay_msec(&self) -> u64 {
+ (RESYNC_RETRY_DELAY.as_millis() as u64)
+ << std::cmp::min(self.errors - 1, RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER)
+ }
+ fn next_try(&self) -> u64 {
+ self.last_try + self.delay_msec()
+ }
+}
diff --git a/src/garage/admin.rs b/src/garage/admin.rs
index 64a448fc..7ba9330c 100644
--- a/src/garage/admin.rs
+++ b/src/garage/admin.rs
@@ -15,6 +15,8 @@ use garage_table::*;
use garage_rpc::*;
+use garage_block::repair::ScrubWorkerCommand;
+
use garage_model::bucket_alias_table::*;
use garage_model::bucket_table::*;
use garage_model::garage::Garage;
@@ -779,13 +781,13 @@ impl AdminRpcHandler {
writeln!(
&mut ret,
" resync queue length: {}",
- self.garage.block_manager.resync_queue_len()?
+ self.garage.block_manager.resync.queue_len()?
)
.unwrap();
writeln!(
&mut ret,
" blocks with resync errors: {}",
- self.garage.block_manager.resync_errors_len()?
+ self.garage.block_manager.resync.errors_len()?
)
.unwrap();
@@ -836,6 +838,32 @@ impl AdminRpcHandler {
let workers = self.garage.background.get_worker_info();
Ok(AdminRpc::WorkerList(workers, opt))
}
+ WorkerCmd::Set { opt } => match opt {
+ WorkerSetCmd::ScrubTranquility { tranquility } => {
+ let scrub_command = ScrubWorkerCommand::SetTranquility(tranquility);
+ self.garage
+ .block_manager
+ .send_scrub_command(scrub_command)
+ .await;
+ Ok(AdminRpc::Ok("Scrub tranquility updated".into()))
+ }
+ WorkerSetCmd::ResyncNWorkers { n_workers } => {
+ self.garage
+ .block_manager
+ .resync
+ .set_n_workers(n_workers)
+ .await?;
+ Ok(AdminRpc::Ok("Number of resync workers updated".into()))
+ }
+ WorkerSetCmd::ResyncTranquility { tranquility } => {
+ self.garage
+ .block_manager
+ .resync
+ .set_tranquility(tranquility)
+ .await?;
+ Ok(AdminRpc::Ok("Resync tranquility updated".into()))
+ }
+ },
}
}
}
diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs
index 9274f80f..0388cef5 100644
--- a/src/garage/cli/structs.rs
+++ b/src/garage/cli/structs.rs
@@ -501,6 +501,12 @@ pub enum WorkerCmd {
#[structopt(flatten)]
opt: WorkerListOpt,
},
+ /// Set worker parameter
+ #[structopt(name = "set", version = version::garage())]
+ Set {
+ #[structopt(subcommand)]
+ opt: WorkerSetCmd,
+ },
}
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone, Copy)]
@@ -512,3 +518,16 @@ pub struct WorkerListOpt {
#[structopt(short = "e", long = "errors")]
pub errors: bool,
}
+
+#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)]
+pub enum WorkerSetCmd {
+ /// Set tranquility of scrub operations
+ #[structopt(name = "scrub-tranquility", version = version::garage())]
+ ScrubTranquility { tranquility: u32 },
+ /// Set number of concurrent block resync workers
+ #[structopt(name = "resync-n-workers", version = version::garage())]
+ ResyncNWorkers { n_workers: usize },
+ /// Set tranquility of block resync operations
+ #[structopt(name = "resync-tranquility", version = version::garage())]
+ ResyncTranquility { tranquility: u32 },
+}
diff --git a/src/model/garage.rs b/src/model/garage.rs
index 0d239df6..c67f1fe0 100644
--- a/src/model/garage.rs
+++ b/src/model/garage.rs
@@ -164,7 +164,6 @@ impl Garage {
&db,
config.data_dir.clone(),
config.compression_level,
- config.block_manager_background_tranquility,
data_rep_param,
system.clone(),
);
diff --git a/src/rpc/Cargo.toml b/src/rpc/Cargo.toml
index 6f3c81ed..1a2ce954 100644
--- a/src/rpc/Cargo.toml
+++ b/src/rpc/Cargo.toml
@@ -45,7 +45,7 @@ tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi
tokio-stream = { version = "0.1", features = ["net"] }
opentelemetry = "0.17"
-#netapp = { version = "0.4.4", features = ["telemetry"] }
+#netapp = { version = "0.4.5", features = ["telemetry"] }
netapp = { version = "0.5.0", git = "https://git.deuxfleurs.fr/lx/netapp", branch = "stream-body", features = ["telemetry"] }
hyper = { version = "0.14", features = ["client", "http1", "runtime", "tcp"] }
diff --git a/src/util/config.rs b/src/util/config.rs
index e8ef4fdd..a2bb8fb3 100644
--- a/src/util/config.rs
+++ b/src/util/config.rs
@@ -23,10 +23,6 @@ pub struct Config {
#[serde(default = "default_block_size")]
pub block_size: usize,
- /// Size of data blocks to save to disk
- #[serde(default = "default_block_manager_background_tranquility")]
- pub block_manager_background_tranquility: u32,
-
/// Replication mode. Supported values:
/// - none, 1 -> no replication
/// - 2 -> 2-way replication
@@ -147,9 +143,6 @@ fn default_sled_flush_every_ms() -> u64 {
fn default_block_size() -> usize {
1048576
}
-fn default_block_manager_background_tranquility() -> u32 {
- 2
-}
/// Read and parse configuration
pub fn read_config(config_file: PathBuf) -> Result<Config, Error> {