forked from Deuxfleurs/garage
block manager: refactor: split resync into separate file
This commit is contained in:
parent
943d76c583
commit
47be652a1f
5 changed files with 595 additions and 555 deletions
|
@ -3,6 +3,7 @@ extern crate tracing;
|
||||||
|
|
||||||
pub mod manager;
|
pub mod manager;
|
||||||
pub mod repair;
|
pub mod repair;
|
||||||
|
pub mod resync;
|
||||||
|
|
||||||
mod block;
|
mod block;
|
||||||
mod metrics;
|
mod metrics;
|
||||||
|
|
|
@ -1,33 +1,19 @@
|
||||||
use std::convert::TryInto;
|
|
||||||
use std::path::PathBuf;
|
use std::path::PathBuf;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use std::time::Duration;
|
use std::time::Duration;
|
||||||
|
|
||||||
use arc_swap::{ArcSwap, ArcSwapOption};
|
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use serde::{Deserialize, Serialize};
|
use serde::{Deserialize, Serialize};
|
||||||
|
|
||||||
use futures::future::*;
|
|
||||||
use tokio::fs;
|
use tokio::fs;
|
||||||
use tokio::io::{AsyncReadExt, AsyncWriteExt};
|
use tokio::io::{AsyncReadExt, AsyncWriteExt};
|
||||||
use tokio::select;
|
use tokio::sync::{mpsc, Mutex};
|
||||||
use tokio::sync::{mpsc, watch, Mutex, Notify};
|
|
||||||
|
|
||||||
use opentelemetry::{
|
|
||||||
trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
|
|
||||||
Context, KeyValue,
|
|
||||||
};
|
|
||||||
|
|
||||||
use garage_db as db;
|
use garage_db as db;
|
||||||
use garage_db::counted_tree_hack::CountedTree;
|
|
||||||
|
|
||||||
use garage_util::background::*;
|
|
||||||
use garage_util::data::*;
|
use garage_util::data::*;
|
||||||
use garage_util::error::*;
|
use garage_util::error::*;
|
||||||
use garage_util::metrics::RecordDuration;
|
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::system::System;
|
||||||
use garage_rpc::*;
|
use garage_rpc::*;
|
||||||
|
@ -38,27 +24,13 @@ use crate::block::*;
|
||||||
use crate::metrics::*;
|
use crate::metrics::*;
|
||||||
use crate::rc::*;
|
use crate::rc::*;
|
||||||
use crate::repair::*;
|
use crate::repair::*;
|
||||||
|
use crate::resync::*;
|
||||||
|
|
||||||
/// Size under which data will be stored inlined in database instead of as files
|
/// Size under which data will be stored inlined in database instead of as files
|
||||||
pub const INLINE_THRESHOLD: usize = 3072;
|
pub const INLINE_THRESHOLD: usize = 3072;
|
||||||
|
|
||||||
// Timeout for RPCs that read and write blocks to remote nodes
|
// Timeout for RPCs that read and write blocks to remote nodes
|
||||||
const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(30);
|
pub(crate) const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(30);
|
||||||
// Timeout for RPCs that ask other nodes whether they need a copy
|
|
||||||
// of a given block before we delete it locally
|
|
||||||
const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5);
|
|
||||||
|
|
||||||
// The delay between the time where a resync operation fails
|
|
||||||
// and the time when it is retried, with exponential backoff
|
|
||||||
// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure).
|
|
||||||
const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60);
|
|
||||||
// The minimum retry delay is 60 seconds = 1 minute
|
|
||||||
// The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour)
|
|
||||||
const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6;
|
|
||||||
|
|
||||||
// Resync tranquility is initially set to 2, but can be changed in the CLI
|
|
||||||
// and the updated version is persisted over Garage restarts
|
|
||||||
const INITIAL_RESYNC_TRANQUILITY: u32 = 2;
|
|
||||||
|
|
||||||
// The delay between the moment when the reference counter
|
// The delay between the moment when the reference counter
|
||||||
// drops to zero, and the moment where we allow ourselves
|
// drops to zero, and the moment where we allow ourselves
|
||||||
|
@ -96,35 +68,23 @@ pub struct BlockManager {
|
||||||
|
|
||||||
compression_level: Option<i32>,
|
compression_level: Option<i32>,
|
||||||
|
|
||||||
mutation_lock: Mutex<BlockManagerLocked>,
|
pub(crate) mutation_lock: Mutex<BlockManagerLocked>,
|
||||||
|
|
||||||
pub(crate) rc: BlockRc,
|
pub(crate) rc: BlockRc,
|
||||||
|
pub resync: BlockResyncManager,
|
||||||
resync_queue: CountedTree,
|
|
||||||
resync_notify: Notify,
|
|
||||||
resync_errors: CountedTree,
|
|
||||||
|
|
||||||
resync_persister: Persister<ResyncPersistedConfig>,
|
|
||||||
resync_persisted: ArcSwap<ResyncPersistedConfig>,
|
|
||||||
|
|
||||||
pub(crate) system: Arc<System>,
|
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
|
// This custom struct contains functions that must only be ran
|
||||||
// when the lock is held. We ensure that it is the case by storing
|
// when the lock is held. We ensure that it is the case by storing
|
||||||
// it INSIDE a Mutex.
|
// it INSIDE a Mutex.
|
||||||
struct BlockManagerLocked();
|
pub(crate) struct BlockManagerLocked();
|
||||||
|
|
||||||
enum ResyncIterResult {
|
|
||||||
BusyDidSomething,
|
|
||||||
BusyDidNothing,
|
|
||||||
IdleFor(Duration),
|
|
||||||
}
|
|
||||||
|
|
||||||
impl BlockManager {
|
impl BlockManager {
|
||||||
pub fn new(
|
pub fn new(
|
||||||
|
@ -139,25 +99,7 @@ impl BlockManager {
|
||||||
.expect("Unable to open block_local_rc tree");
|
.expect("Unable to open block_local_rc tree");
|
||||||
let rc = BlockRc::new(rc);
|
let rc = BlockRc::new(rc);
|
||||||
|
|
||||||
let resync_queue = db
|
let resync = BlockResyncManager::new(db, &system);
|
||||||
.open_tree("block_local_resync_queue")
|
|
||||||
.expect("Unable to open block_local_resync_queue tree");
|
|
||||||
let resync_queue =
|
|
||||||
CountedTree::new(resync_queue).expect("Could not count block_local_resync_queue");
|
|
||||||
|
|
||||||
let resync_errors = db
|
|
||||||
.open_tree("block_local_resync_errors")
|
|
||||||
.expect("Unable to open block_local_resync_errors tree");
|
|
||||||
let resync_errors =
|
|
||||||
CountedTree::new(resync_errors).expect("Could not count block_local_resync_errors");
|
|
||||||
|
|
||||||
let resync_persister = Persister::new(&system.metadata_dir, "resync_cfg");
|
|
||||||
let resync_persisted = match resync_persister.load() {
|
|
||||||
Ok(v) => v,
|
|
||||||
Err(_) => ResyncPersistedConfig {
|
|
||||||
tranquility: INITIAL_RESYNC_TRANQUILITY,
|
|
||||||
},
|
|
||||||
};
|
|
||||||
|
|
||||||
let endpoint = system
|
let endpoint = system
|
||||||
.netapp
|
.netapp
|
||||||
|
@ -165,7 +107,9 @@ impl BlockManager {
|
||||||
|
|
||||||
let manager_locked = BlockManagerLocked();
|
let manager_locked = BlockManagerLocked();
|
||||||
|
|
||||||
let metrics = BlockManagerMetrics::new(resync_queue.clone(), resync_errors.clone());
|
let metrics = BlockManagerMetrics::new(resync.queue.clone(), resync.errors.clone());
|
||||||
|
|
||||||
|
let (scrub_tx, scrub_rx) = mpsc::channel(1);
|
||||||
|
|
||||||
let block_manager = Arc::new(Self {
|
let block_manager = Arc::new(Self {
|
||||||
replication,
|
replication,
|
||||||
|
@ -173,25 +117,31 @@ impl BlockManager {
|
||||||
compression_level,
|
compression_level,
|
||||||
mutation_lock: Mutex::new(manager_locked),
|
mutation_lock: Mutex::new(manager_locked),
|
||||||
rc,
|
rc,
|
||||||
resync_queue,
|
resync,
|
||||||
resync_notify: Notify::new(),
|
|
||||||
resync_errors,
|
|
||||||
resync_persister,
|
|
||||||
resync_persisted: ArcSwap::new(Arc::new(resync_persisted)),
|
|
||||||
system,
|
system,
|
||||||
endpoint,
|
endpoint,
|
||||||
metrics,
|
metrics,
|
||||||
tx_scrub_command: ArcSwapOption::new(None),
|
tx_scrub_command: scrub_tx,
|
||||||
});
|
});
|
||||||
block_manager.endpoint.set_handler(block_manager.clone());
|
block_manager.endpoint.set_handler(block_manager.clone());
|
||||||
|
|
||||||
block_manager.clone().spawn_background_workers();
|
// Spawn one resync worker
|
||||||
|
let background = block_manager.system.background.clone();
|
||||||
|
let worker = ResyncWorker::new(block_manager.clone());
|
||||||
|
tokio::spawn(async move {
|
||||||
|
tokio::time::sleep(Duration::from_secs(10)).await;
|
||||||
|
background.spawn_worker(worker);
|
||||||
|
});
|
||||||
|
|
||||||
|
// Spawn scrub worker
|
||||||
|
let scrub_worker = ScrubWorker::new(block_manager.clone(), scrub_rx);
|
||||||
|
block_manager.system.background.spawn_worker(scrub_worker);
|
||||||
|
|
||||||
block_manager
|
block_manager
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Ask nodes that might have a (possibly compressed) block for it
|
/// Ask nodes that might have a (possibly compressed) block for it
|
||||||
async fn rpc_get_raw_block(&self, hash: &Hash) -> Result<DataBlock, Error> {
|
pub(crate) async fn rpc_get_raw_block(&self, hash: &Hash) -> Result<DataBlock, Error> {
|
||||||
let who = self.replication.read_nodes(hash);
|
let who = self.replication.read_nodes(hash);
|
||||||
let resps = self
|
let resps = self
|
||||||
.system
|
.system
|
||||||
|
@ -243,20 +193,6 @@ impl BlockManager {
|
||||||
Ok(())
|
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
|
/// Get number of items in the refcount table
|
||||||
pub fn rc_len(&self) -> Result<usize, Error> {
|
pub fn rc_len(&self) -> Result<usize, Error> {
|
||||||
Ok(self.rc.rc.len()?)
|
Ok(self.rc.rc.len()?)
|
||||||
|
@ -264,13 +200,7 @@ impl BlockManager {
|
||||||
|
|
||||||
/// Send command to start/stop/manager scrub worker
|
/// Send command to start/stop/manager scrub worker
|
||||||
pub async fn send_scrub_command(&self, cmd: ScrubWorkerCommand) {
|
pub async fn send_scrub_command(&self, cmd: ScrubWorkerCommand) {
|
||||||
let _ = self
|
let _ = self.tx_scrub_command.send(cmd).await;
|
||||||
.tx_scrub_command
|
|
||||||
.load()
|
|
||||||
.as_ref()
|
|
||||||
.unwrap()
|
|
||||||
.send(cmd)
|
|
||||||
.await;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
//// ----- Managing the reference counter ----
|
//// ----- Managing the reference counter ----
|
||||||
|
@ -291,7 +221,7 @@ impl BlockManager {
|
||||||
// we will fecth it from someone.
|
// we will fecth it from someone.
|
||||||
let this = self.clone();
|
let this = self.clone();
|
||||||
tokio::spawn(async move {
|
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);
|
error!("Block {:?} could not be put in resync queue: {}.", hash, e);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -313,7 +243,9 @@ impl BlockManager {
|
||||||
// after that delay has passed.
|
// after that delay has passed.
|
||||||
let this = self.clone();
|
let this = self.clone();
|
||||||
tokio::spawn(async move {
|
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);
|
error!("Block {:?} could not be put in resync queue: {}.", hash, e);
|
||||||
}
|
}
|
||||||
|
@ -325,7 +257,11 @@ impl BlockManager {
|
||||||
// ---- Reading and writing blocks locally ----
|
// ---- Reading and writing blocks locally ----
|
||||||
|
|
||||||
/// Write a block to disk
|
/// Write a block to disk
|
||||||
async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<BlockRpc, Error> {
|
pub(crate) async fn write_block(
|
||||||
|
&self,
|
||||||
|
hash: &Hash,
|
||||||
|
data: &DataBlock,
|
||||||
|
) -> Result<BlockRpc, Error> {
|
||||||
let write_size = data.inner_buffer().len() as u64;
|
let write_size = data.inner_buffer().len() as u64;
|
||||||
|
|
||||||
let res = self
|
let res = self
|
||||||
|
@ -361,7 +297,7 @@ impl BlockManager {
|
||||||
Ok(c) => c,
|
Ok(c) => c,
|
||||||
Err(e) => {
|
Err(e) => {
|
||||||
// Not found but maybe we should have had it ??
|
// 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));
|
return Err(Into::into(e));
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -388,7 +324,7 @@ impl BlockManager {
|
||||||
.await
|
.await
|
||||||
.move_block_to_corrupted(hash, self)
|
.move_block_to_corrupted(hash, self)
|
||||||
.await?;
|
.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));
|
return Err(Error::CorruptData(*hash));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -432,322 +368,6 @@ impl BlockManager {
|
||||||
path.set_extension("");
|
path.set_extension("");
|
||||||
fs::metadata(&path).await.map(|_| false).map_err(Into::into)
|
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
|
|
||||||
.lock()
|
|
||||||
.await
|
|
||||||
.check_block_status(hash, self)
|
|
||||||
.await?;
|
|
||||||
|
|
||||||
if exists != needed.is_needed() || exists != needed.is_nonzero() {
|
|
||||||
debug!(
|
|
||||||
"Resync block {:?}: exists {}, nonzero rc {}, deletable {}",
|
|
||||||
hash,
|
|
||||||
exists,
|
|
||||||
needed.is_nonzero(),
|
|
||||||
needed.is_deletable(),
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
if exists && needed.is_deletable() {
|
|
||||||
info!("Resync block {:?}: offloading and deleting", hash);
|
|
||||||
|
|
||||||
let mut who = self.replication.write_nodes(hash);
|
|
||||||
if who.len() < self.replication.write_quorum() {
|
|
||||||
return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string()));
|
|
||||||
}
|
|
||||||
who.retain(|id| *id != self.system.id);
|
|
||||||
|
|
||||||
let msg = Arc::new(BlockRpc::NeedBlockQuery(*hash));
|
|
||||||
let who_needs_fut = who.iter().map(|to| {
|
|
||||||
self.system.rpc.call_arc(
|
|
||||||
&self.endpoint,
|
|
||||||
*to,
|
|
||||||
msg.clone(),
|
|
||||||
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
|
||||||
.with_timeout(NEED_BLOCK_QUERY_TIMEOUT),
|
|
||||||
)
|
|
||||||
});
|
|
||||||
let who_needs_resps = join_all(who_needs_fut).await;
|
|
||||||
|
|
||||||
let mut need_nodes = vec![];
|
|
||||||
for (node, needed) in who.iter().zip(who_needs_resps.into_iter()) {
|
|
||||||
match needed.err_context("NeedBlockQuery RPC")? {
|
|
||||||
BlockRpc::NeedBlockReply(needed) => {
|
|
||||||
if needed {
|
|
||||||
need_nodes.push(*node);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
m => {
|
|
||||||
return Err(Error::unexpected_rpc_message(m));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if !need_nodes.is_empty() {
|
|
||||||
trace!(
|
|
||||||
"Block {:?} needed by {} nodes, sending",
|
|
||||||
hash,
|
|
||||||
need_nodes.len()
|
|
||||||
);
|
|
||||||
|
|
||||||
for node in need_nodes.iter() {
|
|
||||||
self.metrics
|
|
||||||
.resync_send_counter
|
|
||||||
.add(1, &[KeyValue::new("to", format!("{:?}", node))]);
|
|
||||||
}
|
|
||||||
|
|
||||||
let put_block_message = self.read_block(hash).await?;
|
|
||||||
self.system
|
|
||||||
.rpc
|
|
||||||
.try_call_many(
|
|
||||||
&self.endpoint,
|
|
||||||
&need_nodes[..],
|
|
||||||
put_block_message,
|
|
||||||
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
|
||||||
.with_quorum(need_nodes.len())
|
|
||||||
.with_timeout(BLOCK_RW_TIMEOUT),
|
|
||||||
)
|
|
||||||
.await
|
|
||||||
.err_context("PutBlock RPC")?;
|
|
||||||
}
|
|
||||||
info!(
|
|
||||||
"Deleting unneeded block {:?}, offload finished ({} / {})",
|
|
||||||
hash,
|
|
||||||
need_nodes.len(),
|
|
||||||
who.len()
|
|
||||||
);
|
|
||||||
|
|
||||||
self.mutation_lock
|
|
||||||
.lock()
|
|
||||||
.await
|
|
||||||
.delete_if_unneeded(hash, self)
|
|
||||||
.await?;
|
|
||||||
|
|
||||||
self.rc.clear_deleted_block_rc(hash)?;
|
|
||||||
}
|
|
||||||
|
|
||||||
if needed.is_nonzero() && !exists {
|
|
||||||
info!(
|
|
||||||
"Resync block {:?}: fetching absent but needed block (refcount > 0)",
|
|
||||||
hash
|
|
||||||
);
|
|
||||||
|
|
||||||
let block_data = self.rpc_get_raw_block(hash).await?;
|
|
||||||
|
|
||||||
self.metrics.resync_recv_counter.add(1);
|
|
||||||
|
|
||||||
self.write_block(hash, &block_data).await?;
|
|
||||||
}
|
|
||||||
|
|
||||||
Ok(())
|
|
||||||
}
|
|
||||||
|
|
||||||
async fn update_resync_persisted(
|
|
||||||
&self,
|
|
||||||
update: impl Fn(&mut ResyncPersistedConfig),
|
|
||||||
) -> Result<(), Error> {
|
|
||||||
let mut cfg: ResyncPersistedConfig = *self.resync_persisted.load().as_ref();
|
|
||||||
update(&mut cfg);
|
|
||||||
self.resync_persister.save_async(&cfg).await?;
|
|
||||||
self.resync_persisted.store(Arc::new(cfg));
|
|
||||||
self.resync_notify.notify_one();
|
|
||||||
Ok(())
|
|
||||||
}
|
|
||||||
|
|
||||||
pub async fn set_resync_tranquility(&self, tranquility: u32) -> Result<(), Error> {
|
|
||||||
self.update_resync_persisted(|cfg| cfg.tranquility = tranquility)
|
|
||||||
.await
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#[async_trait]
|
#[async_trait]
|
||||||
|
@ -766,92 +386,13 @@ impl EndpointHandler<BlockRpc> for BlockManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Serialize, Deserialize, Clone, Copy)]
|
pub(crate) struct BlockStatus {
|
||||||
struct ResyncPersistedConfig {
|
pub(crate) exists: bool,
|
||||||
tranquility: u32,
|
pub(crate) needed: RcEntry,
|
||||||
}
|
|
||||||
|
|
||||||
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![];
|
|
||||||
ret.push(format!(
|
|
||||||
"tranquility = {}",
|
|
||||||
self.manager.resync_persisted.load().tranquility
|
|
||||||
));
|
|
||||||
|
|
||||||
let qlen = self.manager.resync_queue_len().unwrap_or(0);
|
|
||||||
if qlen > 0 {
|
|
||||||
ret.push(format!("{} blocks in queue", qlen));
|
|
||||||
}
|
|
||||||
|
|
||||||
let elen = self.manager.resync_errors_len().unwrap_or(0);
|
|
||||||
if elen > 0 {
|
|
||||||
ret.push(format!("{} blocks in error state", elen));
|
|
||||||
}
|
|
||||||
|
|
||||||
Some(ret.join(", "))
|
|
||||||
}
|
|
||||||
|
|
||||||
async fn work(&mut self, _must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error> {
|
|
||||||
self.tranquilizer.reset();
|
|
||||||
match self.manager.resync_iter().await {
|
|
||||||
Ok(ResyncIterResult::BusyDidSomething) => Ok(self
|
|
||||||
.tranquilizer
|
|
||||||
.tranquilize_worker(self.manager.resync_persisted.load().tranquility)),
|
|
||||||
Ok(ResyncIterResult::BusyDidNothing) => Ok(WorkerState::Busy),
|
|
||||||
Ok(ResyncIterResult::IdleFor(delay)) => {
|
|
||||||
self.next_delay = delay;
|
|
||||||
Ok(WorkerState::Idle)
|
|
||||||
}
|
|
||||||
Err(e) => {
|
|
||||||
// The errors that we have here are only Sled errors
|
|
||||||
// We don't really know how to handle them so just ¯\_(ツ)_/¯
|
|
||||||
// (there is kind of an assumption that Sled won't error on us,
|
|
||||||
// if it does there is not much we can do -- TODO should we just panic?)
|
|
||||||
// Here we just give the error to the worker manager,
|
|
||||||
// it will print it to the logs and increment a counter
|
|
||||||
Err(e.into())
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
async fn wait_for_work(&mut self, _must_exit: &watch::Receiver<bool>) -> WorkerState {
|
|
||||||
select! {
|
|
||||||
_ = tokio::time::sleep(self.next_delay) => (),
|
|
||||||
_ = self.manager.resync_notify.notified() => (),
|
|
||||||
};
|
|
||||||
WorkerState::Busy
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
struct BlockStatus {
|
|
||||||
exists: bool,
|
|
||||||
needed: RcEntry,
|
|
||||||
}
|
}
|
||||||
|
|
||||||
impl BlockManagerLocked {
|
impl BlockManagerLocked {
|
||||||
async fn check_block_status(
|
pub(crate) async fn check_block_status(
|
||||||
&self,
|
&self,
|
||||||
hash: &Hash,
|
hash: &Hash,
|
||||||
mgr: &BlockManager,
|
mgr: &BlockManager,
|
||||||
|
@ -938,7 +479,11 @@ impl BlockManagerLocked {
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
async fn delete_if_unneeded(&self, hash: &Hash, mgr: &BlockManager) -> Result<(), Error> {
|
pub(crate) async fn delete_if_unneeded(
|
||||||
|
&self,
|
||||||
|
hash: &Hash,
|
||||||
|
mgr: &BlockManager,
|
||||||
|
) -> Result<(), Error> {
|
||||||
let BlockStatus { exists, needed } = self.check_block_status(hash, mgr).await?;
|
let BlockStatus { exists, needed } = self.check_block_status(hash, mgr).await?;
|
||||||
|
|
||||||
if exists && needed.is_deletable() {
|
if exists && needed.is_deletable() {
|
||||||
|
@ -952,50 +497,3 @@ impl BlockManagerLocked {
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Counts the number of errors when resyncing a block,
|
|
||||||
/// and the time of the last try.
|
|
||||||
/// Used to implement exponential backoff.
|
|
||||||
#[derive(Clone, Copy, Debug)]
|
|
||||||
struct ErrorCounter {
|
|
||||||
errors: u64,
|
|
||||||
last_try: u64,
|
|
||||||
}
|
|
||||||
|
|
||||||
impl ErrorCounter {
|
|
||||||
fn new(now: u64) -> Self {
|
|
||||||
Self {
|
|
||||||
errors: 1,
|
|
||||||
last_try: now,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
fn decode(data: &[u8]) -> Self {
|
|
||||||
Self {
|
|
||||||
errors: u64::from_be_bytes(data[0..8].try_into().unwrap()),
|
|
||||||
last_try: u64::from_be_bytes(data[8..16].try_into().unwrap()),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
fn encode(&self) -> Vec<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()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
|
@ -112,7 +112,9 @@ impl Worker for RepairWorker {
|
||||||
}
|
}
|
||||||
|
|
||||||
for hash in batch_of_hashes.into_iter() {
|
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)
|
self.next_start = Some(hash)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -124,7 +126,9 @@ impl Worker for RepairWorker {
|
||||||
// This allows us to find blocks we are storing but don't actually need,
|
// 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.
|
// so that we can offload them if necessary and then delete them locally.
|
||||||
if let Some(hash) = bi.next().await? {
|
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)
|
Ok(WorkerState::Busy)
|
||||||
} else {
|
} else {
|
||||||
Ok(WorkerState::Done)
|
Ok(WorkerState::Done)
|
||||||
|
|
536
src/block/resync.rs
Normal file
536
src/block/resync.rs
Normal file
|
@ -0,0 +1,536 @@
|
||||||
|
use std::convert::TryInto;
|
||||||
|
use std::sync::Arc;
|
||||||
|
use std::time::Duration;
|
||||||
|
|
||||||
|
use arc_swap::ArcSwap;
|
||||||
|
use async_trait::async_trait;
|
||||||
|
use serde::{Deserialize, Serialize};
|
||||||
|
|
||||||
|
use futures::future::*;
|
||||||
|
use tokio::select;
|
||||||
|
use tokio::sync::{watch, Notify};
|
||||||
|
|
||||||
|
use opentelemetry::{
|
||||||
|
trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
|
||||||
|
Context, KeyValue,
|
||||||
|
};
|
||||||
|
|
||||||
|
use garage_db as db;
|
||||||
|
use garage_db::counted_tree_hack::CountedTree;
|
||||||
|
|
||||||
|
use garage_util::background::*;
|
||||||
|
use garage_util::data::*;
|
||||||
|
use garage_util::error::*;
|
||||||
|
use garage_util::metrics::RecordDuration;
|
||||||
|
use garage_util::persister::Persister;
|
||||||
|
use garage_util::time::*;
|
||||||
|
use garage_util::tranquilizer::Tranquilizer;
|
||||||
|
|
||||||
|
use garage_rpc::system::System;
|
||||||
|
use garage_rpc::*;
|
||||||
|
|
||||||
|
use garage_table::replication::TableReplication;
|
||||||
|
|
||||||
|
use crate::manager::*;
|
||||||
|
|
||||||
|
// Timeout for RPCs that ask other nodes whether they need a copy
|
||||||
|
// of a given block before we delete it locally
|
||||||
|
pub(crate) const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5);
|
||||||
|
|
||||||
|
// The delay between the time where a resync operation fails
|
||||||
|
// and the time when it is retried, with exponential backoff
|
||||||
|
// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure).
|
||||||
|
pub(crate) const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60);
|
||||||
|
// The minimum retry delay is 60 seconds = 1 minute
|
||||||
|
// The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour)
|
||||||
|
pub(crate) const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6;
|
||||||
|
// Resync tranquility is initially set to 2, but can be changed in the CLI
|
||||||
|
// and the updated version is persisted over Garage restarts
|
||||||
|
const INITIAL_RESYNC_TRANQUILITY: u32 = 2;
|
||||||
|
|
||||||
|
pub struct BlockResyncManager {
|
||||||
|
pub(crate) queue: CountedTree,
|
||||||
|
pub(crate) notify: Notify,
|
||||||
|
pub(crate) errors: CountedTree,
|
||||||
|
|
||||||
|
persister: Persister<ResyncPersistedConfig>,
|
||||||
|
persisted: ArcSwap<ResyncPersistedConfig>,
|
||||||
|
}
|
||||||
|
|
||||||
|
#[derive(Serialize, Deserialize, Clone, Copy)]
|
||||||
|
struct ResyncPersistedConfig {
|
||||||
|
tranquility: u32,
|
||||||
|
}
|
||||||
|
|
||||||
|
enum ResyncIterResult {
|
||||||
|
BusyDidSomething,
|
||||||
|
BusyDidNothing,
|
||||||
|
IdleFor(Duration),
|
||||||
|
}
|
||||||
|
|
||||||
|
impl BlockResyncManager {
|
||||||
|
pub(crate) fn new(db: &db::Db, system: &System) -> Self {
|
||||||
|
let queue = db
|
||||||
|
.open_tree("block_local_resync_queue")
|
||||||
|
.expect("Unable to open block_local_resync_queue tree");
|
||||||
|
let queue = CountedTree::new(queue).expect("Could not count block_local_resync_queue");
|
||||||
|
|
||||||
|
let errors = db
|
||||||
|
.open_tree("block_local_resync_errors")
|
||||||
|
.expect("Unable to open block_local_resync_errors tree");
|
||||||
|
let errors = CountedTree::new(errors).expect("Could not count block_local_resync_errors");
|
||||||
|
|
||||||
|
let persister = Persister::new(&system.metadata_dir, "resync_cfg");
|
||||||
|
let persisted = match persister.load() {
|
||||||
|
Ok(v) => v,
|
||||||
|
Err(_) => ResyncPersistedConfig {
|
||||||
|
tranquility: INITIAL_RESYNC_TRANQUILITY,
|
||||||
|
},
|
||||||
|
};
|
||||||
|
|
||||||
|
Self {
|
||||||
|
queue,
|
||||||
|
notify: Notify::new(),
|
||||||
|
errors,
|
||||||
|
persister,
|
||||||
|
persisted: ArcSwap::new(Arc::new(persisted)),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Get lenght of resync queue
|
||||||
|
pub fn queue_len(&self) -> Result<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((time_bytes, hash_bytes)) = self.queue.first()? {
|
||||||
|
let time_msec = u64::from_be_bytes(time_bytes[0..8].try_into().unwrap());
|
||||||
|
let now = now_msec();
|
||||||
|
|
||||||
|
if now >= time_msec {
|
||||||
|
let hash = Hash::try_from(&hash_bytes[..]).unwrap();
|
||||||
|
|
||||||
|
if let Some(ec) = self.errors.get(hash.as_slice())? {
|
||||||
|
let ec = ErrorCounter::decode(&ec);
|
||||||
|
if now < ec.next_try() {
|
||||||
|
// if next retry after an error is not yet,
|
||||||
|
// don't do resync and return early, but still
|
||||||
|
// make sure the item is still in queue at expected time
|
||||||
|
self.put_to_resync_at(&hash, ec.next_try())?;
|
||||||
|
// ec.next_try() > now >= time_msec, so this remove
|
||||||
|
// is not removing the one we added just above
|
||||||
|
// (we want to do the remove after the insert to ensure
|
||||||
|
// that the item is not lost if we crash in-between)
|
||||||
|
self.queue.remove(time_bytes)?;
|
||||||
|
return Ok(ResyncIterResult::BusyDidNothing);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
let tracer = opentelemetry::global::tracer("garage");
|
||||||
|
let trace_id = gen_uuid();
|
||||||
|
let span = tracer
|
||||||
|
.span_builder("Resync block")
|
||||||
|
.with_trace_id(
|
||||||
|
opentelemetry::trace::TraceId::from_hex(&hex::encode(
|
||||||
|
&trace_id.as_slice()[..16],
|
||||||
|
))
|
||||||
|
.unwrap(),
|
||||||
|
)
|
||||||
|
.with_attributes(vec![KeyValue::new("block", format!("{:?}", hash))])
|
||||||
|
.start(&tracer);
|
||||||
|
|
||||||
|
let res = self
|
||||||
|
.resync_block(manager, &hash)
|
||||||
|
.with_context(Context::current_with_span(span))
|
||||||
|
.bound_record_duration(&manager.metrics.resync_duration)
|
||||||
|
.await;
|
||||||
|
|
||||||
|
manager.metrics.resync_counter.add(1);
|
||||||
|
|
||||||
|
if let Err(e) = &res {
|
||||||
|
manager.metrics.resync_error_counter.add(1);
|
||||||
|
warn!("Error when resyncing {:?}: {}", hash, e);
|
||||||
|
|
||||||
|
let err_counter = match self.errors.get(hash.as_slice())? {
|
||||||
|
Some(ec) => ErrorCounter::decode(&ec).add1(now + 1),
|
||||||
|
None => ErrorCounter::new(now + 1),
|
||||||
|
};
|
||||||
|
|
||||||
|
self.errors.insert(hash.as_slice(), err_counter.encode())?;
|
||||||
|
|
||||||
|
self.put_to_resync_at(&hash, err_counter.next_try())?;
|
||||||
|
// err_counter.next_try() >= now + 1 > now,
|
||||||
|
// the entry we remove from the queue is not
|
||||||
|
// the entry we inserted with put_to_resync_at
|
||||||
|
self.queue.remove(time_bytes)?;
|
||||||
|
} else {
|
||||||
|
self.errors.remove(hash.as_slice())?;
|
||||||
|
self.queue.remove(time_bytes)?;
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(ResyncIterResult::BusyDidSomething)
|
||||||
|
} else {
|
||||||
|
Ok(ResyncIterResult::IdleFor(Duration::from_millis(
|
||||||
|
time_msec - now,
|
||||||
|
)))
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Here we wait either for a notification that an item has been
|
||||||
|
// added to the queue, or for a constant delay of 10 secs to expire.
|
||||||
|
// The delay avoids a race condition where the notification happens
|
||||||
|
// between the time we checked the queue and the first poll
|
||||||
|
// to resync_notify.notified(): if that happens, we'll just loop
|
||||||
|
// back 10 seconds later, which is fine.
|
||||||
|
Ok(ResyncIterResult::IdleFor(Duration::from_secs(10)))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
async fn resync_block(&self, manager: &BlockManager, hash: &Hash) -> Result<(), Error> {
|
||||||
|
let BlockStatus { exists, needed } = manager
|
||||||
|
.mutation_lock
|
||||||
|
.lock()
|
||||||
|
.await
|
||||||
|
.check_block_status(hash, manager)
|
||||||
|
.await?;
|
||||||
|
|
||||||
|
if exists != needed.is_needed() || exists != needed.is_nonzero() {
|
||||||
|
debug!(
|
||||||
|
"Resync block {:?}: exists {}, nonzero rc {}, deletable {}",
|
||||||
|
hash,
|
||||||
|
exists,
|
||||||
|
needed.is_nonzero(),
|
||||||
|
needed.is_deletable(),
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
if exists && needed.is_deletable() {
|
||||||
|
info!("Resync block {:?}: offloading and deleting", hash);
|
||||||
|
|
||||||
|
let mut who = manager.replication.write_nodes(hash);
|
||||||
|
if who.len() < manager.replication.write_quorum() {
|
||||||
|
return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string()));
|
||||||
|
}
|
||||||
|
who.retain(|id| *id != manager.system.id);
|
||||||
|
|
||||||
|
let msg = Arc::new(BlockRpc::NeedBlockQuery(*hash));
|
||||||
|
let who_needs_fut = who.iter().map(|to| {
|
||||||
|
manager.system.rpc.call_arc(
|
||||||
|
&manager.endpoint,
|
||||||
|
*to,
|
||||||
|
msg.clone(),
|
||||||
|
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
||||||
|
.with_timeout(NEED_BLOCK_QUERY_TIMEOUT),
|
||||||
|
)
|
||||||
|
});
|
||||||
|
let who_needs_resps = join_all(who_needs_fut).await;
|
||||||
|
|
||||||
|
let mut need_nodes = vec![];
|
||||||
|
for (node, needed) in who.iter().zip(who_needs_resps.into_iter()) {
|
||||||
|
match needed.err_context("NeedBlockQuery RPC")? {
|
||||||
|
BlockRpc::NeedBlockReply(needed) => {
|
||||||
|
if needed {
|
||||||
|
need_nodes.push(*node);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
m => {
|
||||||
|
return Err(Error::unexpected_rpc_message(m));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if !need_nodes.is_empty() {
|
||||||
|
trace!(
|
||||||
|
"Block {:?} needed by {} nodes, sending",
|
||||||
|
hash,
|
||||||
|
need_nodes.len()
|
||||||
|
);
|
||||||
|
|
||||||
|
for node in need_nodes.iter() {
|
||||||
|
manager
|
||||||
|
.metrics
|
||||||
|
.resync_send_counter
|
||||||
|
.add(1, &[KeyValue::new("to", format!("{:?}", node))]);
|
||||||
|
}
|
||||||
|
|
||||||
|
let put_block_message = manager.read_block(hash).await?;
|
||||||
|
manager
|
||||||
|
.system
|
||||||
|
.rpc
|
||||||
|
.try_call_many(
|
||||||
|
&manager.endpoint,
|
||||||
|
&need_nodes[..],
|
||||||
|
put_block_message,
|
||||||
|
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
||||||
|
.with_quorum(need_nodes.len())
|
||||||
|
.with_timeout(BLOCK_RW_TIMEOUT),
|
||||||
|
)
|
||||||
|
.await
|
||||||
|
.err_context("PutBlock RPC")?;
|
||||||
|
}
|
||||||
|
info!(
|
||||||
|
"Deleting unneeded block {:?}, offload finished ({} / {})",
|
||||||
|
hash,
|
||||||
|
need_nodes.len(),
|
||||||
|
who.len()
|
||||||
|
);
|
||||||
|
|
||||||
|
manager
|
||||||
|
.mutation_lock
|
||||||
|
.lock()
|
||||||
|
.await
|
||||||
|
.delete_if_unneeded(hash, manager)
|
||||||
|
.await?;
|
||||||
|
|
||||||
|
manager.rc.clear_deleted_block_rc(hash)?;
|
||||||
|
}
|
||||||
|
|
||||||
|
if needed.is_nonzero() && !exists {
|
||||||
|
info!(
|
||||||
|
"Resync block {:?}: fetching absent but needed block (refcount > 0)",
|
||||||
|
hash
|
||||||
|
);
|
||||||
|
|
||||||
|
let block_data = manager.rpc_get_raw_block(hash).await?;
|
||||||
|
|
||||||
|
manager.metrics.resync_recv_counter.add(1);
|
||||||
|
|
||||||
|
manager.write_block(hash, &block_data).await?;
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
async fn update_persisted(
|
||||||
|
&self,
|
||||||
|
update: impl Fn(&mut ResyncPersistedConfig),
|
||||||
|
) -> Result<(), Error> {
|
||||||
|
let mut cfg: ResyncPersistedConfig = *self.persisted.load().as_ref();
|
||||||
|
update(&mut cfg);
|
||||||
|
self.persister.save_async(&cfg).await?;
|
||||||
|
self.persisted.store(Arc::new(cfg));
|
||||||
|
self.notify.notify_one();
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
|
||||||
|
pub async fn set_tranquility(&self, tranquility: u32) -> Result<(), Error> {
|
||||||
|
self.update_persisted(|cfg| cfg.tranquility = tranquility)
|
||||||
|
.await
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pub(crate) struct ResyncWorker {
|
||||||
|
manager: Arc<BlockManager>,
|
||||||
|
tranquilizer: Tranquilizer,
|
||||||
|
next_delay: Duration,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl ResyncWorker {
|
||||||
|
pub(crate) 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![];
|
||||||
|
ret.push(format!(
|
||||||
|
"tranquility = {}",
|
||||||
|
self.manager.resync.persisted.load().tranquility
|
||||||
|
));
|
||||||
|
|
||||||
|
let qlen = self.manager.resync.queue_len().unwrap_or(0);
|
||||||
|
if qlen > 0 {
|
||||||
|
ret.push(format!("{} blocks in queue", qlen));
|
||||||
|
}
|
||||||
|
|
||||||
|
let elen = self.manager.resync.errors_len().unwrap_or(0);
|
||||||
|
if elen > 0 {
|
||||||
|
ret.push(format!("{} blocks in error state", elen));
|
||||||
|
}
|
||||||
|
|
||||||
|
Some(ret.join(", "))
|
||||||
|
}
|
||||||
|
|
||||||
|
async fn work(&mut self, _must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error> {
|
||||||
|
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 {
|
||||||
|
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()
|
||||||
|
}
|
||||||
|
}
|
|
@ -781,13 +781,13 @@ impl AdminRpcHandler {
|
||||||
writeln!(
|
writeln!(
|
||||||
&mut ret,
|
&mut ret,
|
||||||
" resync queue length: {}",
|
" resync queue length: {}",
|
||||||
self.garage.block_manager.resync_queue_len()?
|
self.garage.block_manager.resync.queue_len()?
|
||||||
)
|
)
|
||||||
.unwrap();
|
.unwrap();
|
||||||
writeln!(
|
writeln!(
|
||||||
&mut ret,
|
&mut ret,
|
||||||
" blocks with resync errors: {}",
|
" blocks with resync errors: {}",
|
||||||
self.garage.block_manager.resync_errors_len()?
|
self.garage.block_manager.resync.errors_len()?
|
||||||
)
|
)
|
||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
|
@ -850,7 +850,8 @@ impl AdminRpcHandler {
|
||||||
WorkerSetCmd::ResyncTranquility { tranquility } => {
|
WorkerSetCmd::ResyncTranquility { tranquility } => {
|
||||||
self.garage
|
self.garage
|
||||||
.block_manager
|
.block_manager
|
||||||
.set_resync_tranquility(tranquility)
|
.resync
|
||||||
|
.set_tranquility(tranquility)
|
||||||
.await?;
|
.await?;
|
||||||
Ok(AdminRpc::Ok("Resync tranquility updated".into()))
|
Ok(AdminRpc::Ok("Resync tranquility updated".into()))
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in a new issue