2022-06-06 14:29:02 +00:00
|
|
|
use core::ops::Bound;
|
|
|
|
|
2021-11-03 21:07:43 +00:00
|
|
|
use std::convert::TryInto;
|
2021-04-23 19:57:32 +00:00
|
|
|
use std::path::{Path, PathBuf};
|
2020-04-17 13:36:16 +00:00
|
|
|
use std::sync::Arc;
|
2022-02-22 13:52:41 +00:00
|
|
|
use std::time::Duration;
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
use async_trait::async_trait;
|
2022-02-17 22:28:23 +00:00
|
|
|
use serde::{Deserialize, Serialize};
|
|
|
|
|
2020-04-17 17:16:08 +00:00
|
|
|
use futures::future::*;
|
2020-04-22 20:32:58 +00:00
|
|
|
use futures::select;
|
2020-04-09 21:45:07 +00:00
|
|
|
use tokio::fs;
|
2021-03-15 21:36:41 +00:00
|
|
|
use tokio::io::{AsyncReadExt, AsyncWriteExt};
|
2020-04-22 20:32:58 +00:00
|
|
|
use tokio::sync::{watch, Mutex, Notify};
|
2022-02-17 22:28:23 +00:00
|
|
|
|
2022-02-18 19:39:55 +00:00
|
|
|
use opentelemetry::{
|
|
|
|
trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
|
|
|
|
Context, KeyValue,
|
|
|
|
};
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2022-06-02 14:58:00 +00:00
|
|
|
use garage_db as db;
|
|
|
|
|
2020-04-24 10:10:01 +00:00
|
|
|
use garage_util::data::*;
|
2021-10-26 17:13:41 +00:00
|
|
|
use garage_util::error::*;
|
2022-02-22 13:52:41 +00:00
|
|
|
use garage_util::metrics::RecordDuration;
|
2021-03-15 15:21:41 +00:00
|
|
|
use garage_util::time::*;
|
2021-11-03 17:28:43 +00:00
|
|
|
use garage_util::tranquilizer::Tranquilizer;
|
2020-04-18 17:39:08 +00:00
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
use garage_rpc::system::System;
|
|
|
|
use garage_rpc::*;
|
2020-04-23 17:05:46 +00:00
|
|
|
|
2021-03-26 18:41:46 +00:00
|
|
|
use garage_table::replication::{TableReplication, TableShardedReplication};
|
2020-04-23 17:05:46 +00:00
|
|
|
|
2022-03-15 11:12:12 +00:00
|
|
|
use crate::block::*;
|
2022-03-15 11:31:23 +00:00
|
|
|
use crate::metrics::*;
|
2022-03-15 11:23:33 +00:00
|
|
|
use crate::rc::*;
|
2020-04-11 21:00:26 +00:00
|
|
|
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Size under which data will be stored inlined in database instead of as files
|
2020-04-18 17:30:05 +00:00
|
|
|
pub const INLINE_THRESHOLD: usize = 3072;
|
|
|
|
|
2021-10-28 12:32:55 +00:00
|
|
|
// Timeout for RPCs that read and write blocks to remote nodes
|
|
|
|
const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(30);
|
|
|
|
// Timeout for RPCs that ask other nodes whether they need a copy
|
|
|
|
// of a given block before we delete it locally
|
2020-04-17 17:16:08 +00:00
|
|
|
const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5);
|
2021-10-28 12:32:55 +00:00
|
|
|
|
|
|
|
// The delay between the time where a resync operation fails
|
2022-02-25 19:42:56 +00:00
|
|
|
// and the time when it is retried, with exponential backoff
|
|
|
|
// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure).
|
2021-10-28 12:32:55 +00:00
|
|
|
const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60);
|
2022-04-07 08:24:20 +00:00
|
|
|
// 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;
|
2021-10-28 12:32:55 +00:00
|
|
|
|
|
|
|
// The delay between the moment when the reference counter
|
|
|
|
// drops to zero, and the moment where we allow ourselves
|
|
|
|
// to delete the block locally.
|
2022-03-15 11:23:33 +00:00
|
|
|
pub(crate) const BLOCK_GC_DELAY: Duration = Duration::from_secs(600);
|
2020-04-17 17:16:08 +00:00
|
|
|
|
2021-03-26 20:53:28 +00:00
|
|
|
/// RPC messages used to share blocks of data between nodes
|
2020-04-18 17:21:34 +00:00
|
|
|
#[derive(Debug, Serialize, Deserialize)]
|
2021-10-14 09:50:12 +00:00
|
|
|
pub enum BlockRpc {
|
2020-04-18 17:21:34 +00:00
|
|
|
Ok,
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Message to ask for a block of data, by hash
|
2020-04-18 17:21:34 +00:00
|
|
|
GetBlock(Hash),
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Message to send a block of data, either because requested, of for first delivery of new
|
|
|
|
/// block
|
2021-12-15 10:26:43 +00:00
|
|
|
PutBlock {
|
|
|
|
hash: Hash,
|
|
|
|
data: DataBlock,
|
|
|
|
},
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Ask other node if they should have this block, but don't actually have it
|
2020-04-18 17:21:34 +00:00
|
|
|
NeedBlockQuery(Hash),
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Response : whether the node do require that block
|
2020-04-18 17:21:34 +00:00
|
|
|
NeedBlockReply(bool),
|
|
|
|
}
|
|
|
|
|
2021-10-15 09:05:09 +00:00
|
|
|
impl Rpc for BlockRpc {
|
|
|
|
type Response = Result<BlockRpc, Error>;
|
2021-10-14 09:50:12 +00:00
|
|
|
}
|
2020-04-18 17:21:34 +00:00
|
|
|
|
2021-03-26 20:53:28 +00:00
|
|
|
/// The block manager, handling block exchange between nodes, and block storage on local node
|
2020-04-11 21:00:26 +00:00
|
|
|
pub struct BlockManager {
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Replication strategy, allowing to find on which node blocks should be located
|
2020-04-23 17:05:46 +00:00
|
|
|
pub replication: TableShardedReplication,
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Directory in which block are stored
|
2020-04-11 21:00:26 +00:00
|
|
|
pub data_dir: PathBuf,
|
2022-03-15 11:31:23 +00:00
|
|
|
|
2022-03-15 11:04:12 +00:00
|
|
|
compression_level: Option<i32>,
|
2022-03-15 11:31:23 +00:00
|
|
|
background_tranquility: u32,
|
2021-10-22 10:09:03 +00:00
|
|
|
|
|
|
|
mutation_lock: Mutex<BlockManagerLocked>,
|
2020-04-22 20:32:58 +00:00
|
|
|
|
2022-03-15 15:47:13 +00:00
|
|
|
rc: BlockRc,
|
2020-04-22 20:32:58 +00:00
|
|
|
|
2022-06-02 14:58:00 +00:00
|
|
|
resync_queue: db::Tree,
|
2021-03-15 18:51:16 +00:00
|
|
|
resync_notify: Notify,
|
2022-06-02 14:58:00 +00:00
|
|
|
resync_errors: db::Tree,
|
2020-04-22 20:32:58 +00:00
|
|
|
|
2021-03-15 18:51:16 +00:00
|
|
|
system: Arc<System>,
|
2021-10-14 09:50:12 +00:00
|
|
|
endpoint: Arc<Endpoint<BlockRpc, Self>>,
|
2022-02-16 13:23:04 +00:00
|
|
|
|
|
|
|
metrics: BlockManagerMetrics,
|
2020-04-09 21:45:07 +00:00
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
// This custom struct contains functions that must only be ran
|
|
|
|
// when the lock is held. We ensure that it is the case by storing
|
|
|
|
// it INSIDE a Mutex.
|
|
|
|
struct BlockManagerLocked();
|
|
|
|
|
2020-04-11 21:00:26 +00:00
|
|
|
impl BlockManager {
|
2020-04-18 17:21:34 +00:00
|
|
|
pub fn new(
|
2022-06-02 14:58:00 +00:00
|
|
|
db: &db::Db,
|
2020-04-18 17:21:34 +00:00
|
|
|
data_dir: PathBuf,
|
2022-03-15 11:04:12 +00:00
|
|
|
compression_level: Option<i32>,
|
2022-03-15 11:31:23 +00:00
|
|
|
background_tranquility: u32,
|
2020-04-23 17:05:46 +00:00
|
|
|
replication: TableShardedReplication,
|
2020-04-18 17:21:34 +00:00
|
|
|
system: Arc<System>,
|
|
|
|
) -> Arc<Self> {
|
2020-04-11 21:53:32 +00:00
|
|
|
let rc = db
|
|
|
|
.open_tree("block_local_rc")
|
2020-04-11 21:00:26 +00:00
|
|
|
.expect("Unable to open block_local_rc tree");
|
2022-03-15 11:23:33 +00:00
|
|
|
let rc = BlockRc::new(rc);
|
2020-04-17 13:36:16 +00:00
|
|
|
|
|
|
|
let resync_queue = db
|
|
|
|
.open_tree("block_local_resync_queue")
|
|
|
|
.expect("Unable to open block_local_resync_queue tree");
|
|
|
|
|
2022-02-25 19:42:56 +00:00
|
|
|
let resync_errors = db
|
|
|
|
.open_tree("block_local_resync_errors")
|
|
|
|
.expect("Unable to open block_local_resync_errors tree");
|
|
|
|
|
2021-10-19 14:16:10 +00:00
|
|
|
let endpoint = system
|
|
|
|
.netapp
|
2022-05-10 11:16:57 +00:00
|
|
|
.endpoint("garage_block/manager.rs/Rpc".to_string());
|
2020-04-18 17:21:34 +00:00
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
let manager_locked = BlockManagerLocked();
|
|
|
|
|
2022-02-25 19:42:56 +00:00
|
|
|
let metrics = BlockManagerMetrics::new(resync_queue.clone(), resync_errors.clone());
|
2022-02-16 13:23:04 +00:00
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
let block_manager = Arc::new(Self {
|
2020-04-23 17:05:46 +00:00
|
|
|
replication,
|
2020-04-22 20:32:58 +00:00
|
|
|
data_dir,
|
2022-03-15 11:04:12 +00:00
|
|
|
compression_level,
|
2022-03-15 11:31:23 +00:00
|
|
|
background_tranquility,
|
2021-10-22 10:09:03 +00:00
|
|
|
mutation_lock: Mutex::new(manager_locked),
|
2020-04-11 21:00:26 +00:00
|
|
|
rc,
|
2020-04-17 13:36:16 +00:00
|
|
|
resync_queue,
|
2020-04-22 20:32:58 +00:00
|
|
|
resync_notify: Notify::new(),
|
2022-02-25 19:42:56 +00:00
|
|
|
resync_errors,
|
2020-04-17 13:36:16 +00:00
|
|
|
system,
|
2021-10-14 09:50:12 +00:00
|
|
|
endpoint,
|
2022-02-16 13:23:04 +00:00
|
|
|
metrics,
|
2020-04-18 17:21:34 +00:00
|
|
|
});
|
2021-10-14 09:50:12 +00:00
|
|
|
block_manager.endpoint.set_handler(block_manager.clone());
|
2020-04-23 14:40:59 +00:00
|
|
|
|
2022-03-15 11:12:12 +00:00
|
|
|
block_manager.clone().spawn_background_worker();
|
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
block_manager
|
2020-04-23 14:40:59 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
/// Ask nodes that might have a (possibly compressed) block for it
|
|
|
|
async fn rpc_get_raw_block(&self, hash: &Hash) -> Result<DataBlock, Error> {
|
2021-10-26 08:20:05 +00:00
|
|
|
let who = self.replication.read_nodes(hash);
|
2021-10-22 10:09:03 +00:00
|
|
|
let resps = self
|
|
|
|
.system
|
|
|
|
.rpc
|
|
|
|
.try_call_many(
|
|
|
|
&self.endpoint,
|
|
|
|
&who[..],
|
|
|
|
BlockRpc::GetBlock(*hash),
|
|
|
|
RequestStrategy::with_priority(PRIO_NORMAL)
|
|
|
|
.with_quorum(1)
|
|
|
|
.with_timeout(BLOCK_RW_TIMEOUT)
|
|
|
|
.interrupt_after_quorum(true),
|
|
|
|
)
|
|
|
|
.await?;
|
|
|
|
|
|
|
|
for resp in resps {
|
2021-12-15 10:26:43 +00:00
|
|
|
if let BlockRpc::PutBlock { data, .. } = resp {
|
|
|
|
return Ok(data);
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
2020-04-17 18:58:10 +00:00
|
|
|
}
|
2021-10-22 10:09:03 +00:00
|
|
|
Err(Error::Message(format!(
|
|
|
|
"Unable to read block {:?}: no valid blocks returned",
|
|
|
|
hash
|
|
|
|
)))
|
2020-04-11 21:00:26 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
// ---- Public interface ----
|
|
|
|
|
|
|
|
/// Ask nodes that might have a block for it
|
|
|
|
pub async fn rpc_get_block(&self, hash: &Hash) -> Result<Vec<u8>, Error> {
|
|
|
|
self.rpc_get_raw_block(hash).await?.verify_get(*hash)
|
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Send block to nodes that should have it
|
|
|
|
pub async fn rpc_put_block(&self, hash: Hash, data: Vec<u8>) -> Result<(), Error> {
|
|
|
|
let who = self.replication.write_nodes(&hash);
|
2022-03-15 11:04:12 +00:00
|
|
|
let data = DataBlock::from_buffer(data, self.compression_level);
|
2021-10-22 10:09:03 +00:00
|
|
|
self.system
|
|
|
|
.rpc
|
|
|
|
.try_call_many(
|
|
|
|
&self.endpoint,
|
|
|
|
&who[..],
|
2021-12-15 10:26:43 +00:00
|
|
|
BlockRpc::PutBlock { hash, data },
|
2021-10-22 10:09:03 +00:00
|
|
|
RequestStrategy::with_priority(PRIO_NORMAL)
|
|
|
|
.with_quorum(self.replication.write_quorum())
|
|
|
|
.with_timeout(BLOCK_RW_TIMEOUT),
|
|
|
|
)
|
|
|
|
.await?;
|
|
|
|
Ok(())
|
|
|
|
}
|
2020-04-11 21:00:26 +00:00
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Launch the repair procedure on the data store
|
|
|
|
///
|
|
|
|
/// This will list all blocks locally present, as well as those
|
|
|
|
/// that are required because of refcount > 0, and will try
|
|
|
|
/// to fix any mismatch between the two.
|
|
|
|
pub async fn repair_data_store(&self, must_exit: &watch::Receiver<bool>) -> Result<(), Error> {
|
2021-10-28 12:32:55 +00:00
|
|
|
// 1. Repair blocks from RC table.
|
2022-06-06 14:29:02 +00:00
|
|
|
let mut next_start: Option<Hash> = None;
|
|
|
|
loop {
|
2022-06-06 14:31:43 +00:00
|
|
|
// We have to do this complicated two-step process where we first read a bunch
|
|
|
|
// of hashes from the RC table, and then insert them in the to-resync queue,
|
|
|
|
// because of SQLite. Basically, as long as we have an iterator on a DB table,
|
|
|
|
// we can't do anything else on the DB. The naive approach (which we had previously)
|
|
|
|
// of just iterating on the RC table and inserting items one to one in the resync
|
|
|
|
// queue can't work here, it would just provoke a deadlock in the SQLite adapter code.
|
|
|
|
// This is mostly because the Rust bindings for SQLite assume a worst-case scenario
|
|
|
|
// where SQLite is not compiled in thread-safe mode, so we have to wrap everything
|
2022-06-06 14:32:06 +00:00
|
|
|
// in a mutex (see db/sqlite_adapter.rs and discussion in PR #322).
|
2022-06-06 14:29:02 +00:00
|
|
|
let mut batch_of_hashes = vec![];
|
|
|
|
let start_bound = match next_start.as_ref() {
|
|
|
|
None => Bound::Unbounded,
|
|
|
|
Some(x) => Bound::Excluded(x.as_slice()),
|
|
|
|
};
|
|
|
|
for entry in self
|
|
|
|
.rc
|
|
|
|
.rc
|
|
|
|
.range::<&[u8], _>((start_bound, Bound::Unbounded))?
|
|
|
|
{
|
|
|
|
let (hash, _) = entry?;
|
|
|
|
let hash = Hash::try_from(&hash[..]).unwrap();
|
|
|
|
batch_of_hashes.push(hash);
|
|
|
|
if batch_of_hashes.len() >= 1000 {
|
|
|
|
break;
|
|
|
|
}
|
2022-06-03 10:56:35 +00:00
|
|
|
}
|
2022-06-06 14:29:02 +00:00
|
|
|
if batch_of_hashes.is_empty() {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
for hash in batch_of_hashes.into_iter() {
|
|
|
|
self.put_to_resync(&hash, Duration::from_secs(0))?;
|
|
|
|
next_start = Some(hash)
|
|
|
|
}
|
|
|
|
|
|
|
|
if *must_exit.borrow() {
|
2021-10-22 10:09:03 +00:00
|
|
|
return Ok(());
|
|
|
|
}
|
|
|
|
}
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
// 2. Repair blocks actually on disk
|
2021-06-23 23:34:28 +00:00
|
|
|
// Lists all blocks on disk and adds them to the resync queue.
|
|
|
|
// This allows us to find blocks we are storing but don't actually need,
|
|
|
|
// so that we can offload them if necessary and then delete them locally.
|
|
|
|
self.for_each_file(
|
|
|
|
(),
|
2022-03-15 15:47:13 +00:00
|
|
|
move |_, hash| async move {
|
|
|
|
self.put_to_resync(&hash, Duration::from_secs(0))
|
|
|
|
.map_err(Into::into)
|
|
|
|
},
|
2021-06-23 23:34:28 +00:00
|
|
|
must_exit,
|
|
|
|
)
|
|
|
|
.await
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
|
2021-10-27 08:36:04 +00:00
|
|
|
/// Verify integrity of each block on disk. Use `speed_limit` to limit the load generated by
|
|
|
|
/// this function.
|
|
|
|
pub async fn scrub_data_store(
|
|
|
|
&self,
|
|
|
|
must_exit: &watch::Receiver<bool>,
|
2021-11-03 17:28:43 +00:00
|
|
|
tranquility: u32,
|
2021-10-27 08:36:04 +00:00
|
|
|
) -> Result<(), Error> {
|
2021-11-03 17:28:43 +00:00
|
|
|
let tranquilizer = Tranquilizer::new(30);
|
2021-10-27 08:36:04 +00:00
|
|
|
self.for_each_file(
|
2021-11-03 17:28:43 +00:00
|
|
|
tranquilizer,
|
|
|
|
move |mut tranquilizer, hash| async move {
|
|
|
|
let _ = self.read_block(&hash).await;
|
|
|
|
tranquilizer.tranquilize(tranquility).await;
|
|
|
|
Ok(tranquilizer)
|
2021-10-27 08:36:04 +00:00
|
|
|
},
|
|
|
|
must_exit,
|
|
|
|
)
|
|
|
|
.await
|
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Get lenght of resync queue
|
2022-06-06 14:29:02 +00:00
|
|
|
pub fn resync_queue_len(&self) -> Result<usize, Error> {
|
|
|
|
Ok(self.resync_queue.len()?)
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
|
2022-03-28 13:47:23 +00:00
|
|
|
/// Get number of blocks that have an error
|
2022-06-06 14:29:02 +00:00
|
|
|
pub fn resync_errors_len(&self) -> Result<usize, Error> {
|
|
|
|
Ok(self.resync_errors.len()?)
|
2022-03-28 13:47:23 +00:00
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Get number of items in the refcount table
|
2022-06-06 14:29:02 +00:00
|
|
|
pub fn rc_len(&self) -> Result<usize, Error> {
|
|
|
|
Ok(self.rc.rc.len()?)
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
//// ----- Managing the reference counter ----
|
|
|
|
|
|
|
|
/// Increment the number of time a block is used, putting it to resynchronization if it is
|
|
|
|
/// required, but not known
|
2022-06-06 13:46:00 +00:00
|
|
|
pub fn block_incref(self: &Arc<Self>, tx: &mut db::Transaction, hash: Hash) -> db::Result<()> {
|
|
|
|
if self.rc.block_incref(tx, &hash)? {
|
2021-10-28 12:32:55 +00:00
|
|
|
// When the reference counter is incremented, there is
|
|
|
|
// normally a node that is responsible for sending us the
|
|
|
|
// data of the block. However that operation may fail,
|
|
|
|
// so in all cases we add the block here to the todo list
|
|
|
|
// to check later that it arrived correctly, and if not
|
|
|
|
// we will fecth it from someone.
|
2022-06-06 13:46:00 +00:00
|
|
|
let this = self.clone();
|
|
|
|
tokio::spawn(async move {
|
|
|
|
tokio::time::sleep(Duration::from_secs(1)).await;
|
|
|
|
if let Err(e) = this.put_to_resync(&hash, 2 * BLOCK_RW_TIMEOUT) {
|
|
|
|
error!("Block {:?} could not be put in resync queue: {}.", hash, e);
|
|
|
|
}
|
|
|
|
});
|
2020-04-11 21:00:26 +00:00
|
|
|
}
|
2021-10-22 10:09:03 +00:00
|
|
|
Ok(())
|
|
|
|
}
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Decrement the number of time a block is used
|
2022-06-06 13:46:00 +00:00
|
|
|
pub fn block_decref(self: &Arc<Self>, tx: &mut db::Transaction, hash: Hash) -> db::Result<()> {
|
|
|
|
if self.rc.block_decref(tx, &hash)? {
|
2022-03-15 11:23:33 +00:00
|
|
|
// When the RC is decremented, it might drop to zero,
|
|
|
|
// indicating that we don't need the block.
|
|
|
|
// There is a delay before we garbage collect it;
|
|
|
|
// make sure that it is handled in the resync loop
|
|
|
|
// after that delay has passed.
|
2022-06-06 13:46:00 +00:00
|
|
|
let this = self.clone();
|
|
|
|
tokio::spawn(async move {
|
|
|
|
tokio::time::sleep(Duration::from_secs(1)).await;
|
|
|
|
if let Err(e) = this.put_to_resync(&hash, BLOCK_GC_DELAY + Duration::from_secs(10))
|
|
|
|
{
|
|
|
|
error!("Block {:?} could not be put in resync queue: {}.", hash, e);
|
|
|
|
}
|
|
|
|
});
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
Ok(())
|
|
|
|
}
|
2020-04-11 21:00:26 +00:00
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
// ---- Reading and writing blocks locally ----
|
|
|
|
|
|
|
|
/// Write a block to disk
|
2021-12-15 10:26:43 +00:00
|
|
|
async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<BlockRpc, Error> {
|
2022-02-16 13:23:04 +00:00
|
|
|
let write_size = data.inner_buffer().len() as u64;
|
|
|
|
|
|
|
|
let res = self
|
|
|
|
.mutation_lock
|
2021-10-22 10:09:03 +00:00
|
|
|
.lock()
|
|
|
|
.await
|
|
|
|
.write_block(hash, data, self)
|
2022-02-22 12:53:59 +00:00
|
|
|
.bound_record_duration(&self.metrics.block_write_duration)
|
2022-02-16 13:23:04 +00:00
|
|
|
.await?;
|
|
|
|
|
|
|
|
self.metrics.bytes_written.add(write_size);
|
|
|
|
|
|
|
|
Ok(res)
|
2020-04-09 21:45:07 +00:00
|
|
|
}
|
|
|
|
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Read block from disk, verifying it's integrity
|
2021-10-14 09:50:12 +00:00
|
|
|
async fn read_block(&self, hash: &Hash) -> Result<BlockRpc, Error> {
|
2022-02-22 13:52:41 +00:00
|
|
|
let data = self
|
|
|
|
.read_block_internal(hash)
|
2022-02-22 12:53:59 +00:00
|
|
|
.bound_record_duration(&self.metrics.block_read_duration)
|
|
|
|
.await?;
|
|
|
|
|
2022-02-22 13:52:41 +00:00
|
|
|
self.metrics
|
|
|
|
.bytes_read
|
|
|
|
.add(data.inner_buffer().len() as u64);
|
2022-02-22 12:53:59 +00:00
|
|
|
|
|
|
|
Ok(BlockRpc::PutBlock { hash: *hash, data })
|
|
|
|
}
|
2022-02-16 13:23:04 +00:00
|
|
|
|
2022-02-22 12:53:59 +00:00
|
|
|
async fn read_block_internal(&self, hash: &Hash) -> Result<DataBlock, Error> {
|
2021-12-15 10:26:43 +00:00
|
|
|
let mut path = self.block_path(hash);
|
|
|
|
let compressed = match self.is_block_compressed(hash).await {
|
|
|
|
Ok(c) => c,
|
2020-04-17 16:38:11 +00:00
|
|
|
Err(e) => {
|
|
|
|
// Not found but maybe we should have had it ??
|
2021-10-28 12:32:55 +00:00
|
|
|
self.put_to_resync(hash, 2 * BLOCK_RW_TIMEOUT)?;
|
2020-04-17 16:38:11 +00:00
|
|
|
return Err(Into::into(e));
|
|
|
|
}
|
|
|
|
};
|
2021-12-15 10:26:43 +00:00
|
|
|
if compressed {
|
|
|
|
path.set_extension("zst");
|
|
|
|
}
|
|
|
|
let mut f = fs::File::open(&path).await?;
|
|
|
|
|
2020-04-11 21:00:26 +00:00
|
|
|
let mut data = vec![];
|
|
|
|
f.read_to_end(&mut data).await?;
|
2020-04-17 13:36:16 +00:00
|
|
|
drop(f);
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
let data = if compressed {
|
|
|
|
DataBlock::Compressed(data)
|
|
|
|
} else {
|
|
|
|
DataBlock::Plain(data)
|
|
|
|
};
|
|
|
|
|
|
|
|
if data.verify(*hash).is_err() {
|
2022-02-16 13:23:04 +00:00
|
|
|
self.metrics.corruption_counter.add(1);
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
self.mutation_lock
|
|
|
|
.lock()
|
|
|
|
.await
|
|
|
|
.move_block_to_corrupted(hash, self)
|
|
|
|
.await?;
|
2021-10-28 12:32:55 +00:00
|
|
|
self.put_to_resync(hash, Duration::from_millis(0))?;
|
2020-04-21 17:08:42 +00:00
|
|
|
return Err(Error::CorruptData(*hash));
|
2020-04-17 13:36:16 +00:00
|
|
|
}
|
2020-04-11 21:00:26 +00:00
|
|
|
|
2022-02-22 12:53:59 +00:00
|
|
|
Ok(data)
|
2020-04-11 21:00:26 +00:00
|
|
|
}
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2021-03-26 20:53:28 +00:00
|
|
|
/// Check if this node should have a block, but don't actually have it
|
2021-04-06 03:25:28 +00:00
|
|
|
async fn need_block(&self, hash: &Hash) -> Result<bool, Error> {
|
2021-10-22 10:09:03 +00:00
|
|
|
let BlockStatus { exists, needed } = self
|
|
|
|
.mutation_lock
|
|
|
|
.lock()
|
|
|
|
.await
|
|
|
|
.check_block_status(hash, self)
|
|
|
|
.await?;
|
2021-10-28 12:32:55 +00:00
|
|
|
Ok(needed.is_nonzero() && !exists)
|
2020-04-17 17:16:08 +00:00
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Utility: gives the path of the directory in which a block should be found
|
2020-04-11 21:00:26 +00:00
|
|
|
fn block_dir(&self, hash: &Hash) -> PathBuf {
|
|
|
|
let mut path = self.data_dir.clone();
|
|
|
|
path.push(hex::encode(&hash.as_slice()[0..1]));
|
|
|
|
path.push(hex::encode(&hash.as_slice()[1..2]));
|
|
|
|
path
|
|
|
|
}
|
2021-10-22 10:09:03 +00:00
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
/// Utility: give the full path where a block should be found, minus extension if block is
|
|
|
|
/// compressed
|
2020-04-17 17:20:17 +00:00
|
|
|
fn block_path(&self, hash: &Hash) -> PathBuf {
|
|
|
|
let mut path = self.block_dir(hash);
|
|
|
|
path.push(hex::encode(hash.as_ref()));
|
|
|
|
path
|
|
|
|
}
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
/// Utility: check if block is stored compressed. Error if block is not stored
|
|
|
|
async fn is_block_compressed(&self, hash: &Hash) -> Result<bool, Error> {
|
|
|
|
let mut path = self.block_path(hash);
|
|
|
|
path.set_extension("zst");
|
|
|
|
if fs::metadata(&path).await.is_ok() {
|
|
|
|
return Ok(true);
|
|
|
|
}
|
|
|
|
path.set_extension("");
|
|
|
|
fs::metadata(&path).await.map(|_| false).map_err(Into::into)
|
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
// ---- Resync loop ----
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2022-03-15 15:47:13 +00:00
|
|
|
// 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).
|
|
|
|
|
2022-03-15 11:12:12 +00:00
|
|
|
fn spawn_background_worker(self: Arc<Self>) {
|
2022-03-01 10:57:18 +00:00
|
|
|
// Launch a background workers for background resync loop processing
|
|
|
|
let background = self.system.background.clone();
|
|
|
|
tokio::spawn(async move {
|
|
|
|
tokio::time::sleep(Duration::from_secs(10)).await;
|
|
|
|
background.spawn_worker("block resync worker".into(), move |must_exit| {
|
|
|
|
self.resync_loop(must_exit)
|
2021-10-22 10:09:03 +00:00
|
|
|
});
|
2022-03-01 10:57:18 +00:00
|
|
|
});
|
2020-04-17 13:36:16 +00:00
|
|
|
}
|
|
|
|
|
2022-06-06 13:46:00 +00:00
|
|
|
fn put_to_resync(&self, hash: &Hash, delay: Duration) -> db::Result<()> {
|
2021-03-15 13:46:37 +00:00
|
|
|
let when = now_msec() + delay.as_millis() as u64;
|
2022-02-25 19:42:56 +00:00
|
|
|
self.put_to_resync_at(hash, when)
|
|
|
|
}
|
|
|
|
|
2022-06-06 13:46:00 +00:00
|
|
|
fn put_to_resync_at(&self, hash: &Hash, when: u64) -> db::Result<()> {
|
2020-04-21 12:54:55 +00:00
|
|
|
trace!("Put resync_queue: {} {:?}", when, hash);
|
2020-04-17 15:09:57 +00:00
|
|
|
let mut key = u64::to_be_bytes(when).to_vec();
|
|
|
|
key.extend(hash.as_ref());
|
|
|
|
self.resync_queue.insert(key, hash.as_ref())?;
|
2021-03-15 21:36:41 +00:00
|
|
|
self.resync_notify.notify_waiters();
|
2020-04-17 15:09:57 +00:00
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
2021-03-15 22:14:12 +00:00
|
|
|
async fn resync_loop(self: Arc<Self>, mut must_exit: watch::Receiver<bool>) {
|
2021-11-03 17:28:43 +00:00
|
|
|
let mut tranquilizer = Tranquilizer::new(30);
|
|
|
|
|
2020-04-17 13:36:16 +00:00
|
|
|
while !*must_exit.borrow() {
|
2021-11-03 17:28:43 +00:00
|
|
|
match self.resync_iter(&mut must_exit).await {
|
|
|
|
Ok(true) => {
|
2022-03-15 11:31:23 +00:00
|
|
|
tranquilizer.tranquilize(self.background_tranquility).await;
|
2021-11-03 17:28:43 +00:00
|
|
|
}
|
|
|
|
Ok(false) => {
|
|
|
|
tranquilizer.reset();
|
|
|
|
}
|
|
|
|
Err(e) => {
|
|
|
|
// The errors that we have here are only Sled errors
|
|
|
|
// We don't really know how to handle them so just ¯\_(ツ)_/¯
|
|
|
|
// (there is kind of an assumption that Sled won't error on us,
|
|
|
|
// if it does there is not much we can do -- TODO should we just panic?)
|
|
|
|
error!(
|
|
|
|
"Could not do a resync iteration: {} (this is a very bad error)",
|
|
|
|
e
|
|
|
|
);
|
|
|
|
tranquilizer.reset();
|
2021-03-15 22:14:12 +00:00
|
|
|
}
|
2021-03-15 19:09:44 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-03-15 15:47:13 +00:00
|
|
|
// The result of resync_iter is:
|
|
|
|
// - Ok(true) -> a block was processed (successfully or not)
|
|
|
|
// - Ok(false) -> no block was processed, but we are ready for the next iteration
|
|
|
|
// - Err(_) -> a Sled error occurred when reading/writing from resync_queue/resync_errors
|
2022-06-02 14:58:00 +00:00
|
|
|
async fn resync_iter(&self, must_exit: &mut watch::Receiver<bool>) -> Result<bool, db::Error> {
|
2022-06-06 12:01:44 +00:00
|
|
|
if let Some((time_bytes, hash_bytes)) = self.resync_queue.first()? {
|
2021-11-03 21:07:43 +00:00
|
|
|
let time_msec = u64::from_be_bytes(time_bytes[0..8].try_into().unwrap());
|
2021-03-15 19:09:44 +00:00
|
|
|
let now = now_msec();
|
2022-03-01 13:55:37 +00:00
|
|
|
|
2021-03-15 19:09:44 +00:00
|
|
|
if now >= time_msec {
|
|
|
|
let hash = Hash::try_from(&hash_bytes[..]).unwrap();
|
2022-02-18 19:39:55 +00:00
|
|
|
|
2022-02-25 19:42:56 +00:00
|
|
|
if let Some(ec) = self.resync_errors.get(hash.as_slice())? {
|
2022-06-06 12:01:44 +00:00
|
|
|
let ec = ErrorCounter::decode(&ec);
|
2022-02-25 19:42:56 +00:00
|
|
|
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())?;
|
2022-03-01 13:55:37 +00:00
|
|
|
// ec.next_try() > now >= time_msec, so this remove
|
|
|
|
// is not removing the one we added just above
|
2022-03-15 15:47:13 +00:00
|
|
|
// (we want to do the remove after the insert to ensure
|
|
|
|
// that the item is not lost if we crash in-between)
|
2022-03-01 13:55:37 +00:00
|
|
|
self.resync_queue.remove(time_bytes)?;
|
2022-02-25 19:42:56 +00:00
|
|
|
return Ok(false);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-02-18 19:39:55 +00:00
|
|
|
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))
|
2022-02-22 12:53:59 +00:00
|
|
|
.bound_record_duration(&self.metrics.resync_duration)
|
2022-02-18 19:39:55 +00:00
|
|
|
.await;
|
2022-02-16 13:23:04 +00:00
|
|
|
|
|
|
|
self.metrics.resync_counter.add(1);
|
|
|
|
|
2021-03-15 19:09:44 +00:00
|
|
|
if let Err(e) = &res {
|
2022-02-16 13:23:04 +00:00
|
|
|
self.metrics.resync_error_counter.add(1);
|
2021-03-15 19:09:44 +00:00
|
|
|
warn!("Error when resyncing {:?}: {}", hash, e);
|
2022-02-25 19:42:56 +00:00
|
|
|
|
|
|
|
let err_counter = match self.resync_errors.get(hash.as_slice())? {
|
2022-06-06 12:01:44 +00:00
|
|
|
Some(ec) => ErrorCounter::decode(&ec).add1(now + 1),
|
2022-03-01 13:55:37 +00:00
|
|
|
None => ErrorCounter::new(now + 1),
|
2022-02-25 19:42:56 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
self.resync_errors
|
|
|
|
.insert(hash.as_slice(), err_counter.encode())?;
|
2022-03-01 13:55:37 +00:00
|
|
|
|
|
|
|
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)?;
|
2022-02-25 19:42:56 +00:00
|
|
|
} else {
|
|
|
|
self.resync_errors.remove(hash.as_slice())?;
|
2022-03-01 13:55:37 +00:00
|
|
|
self.resync_queue.remove(time_bytes)?;
|
2020-04-22 20:32:58 +00:00
|
|
|
}
|
2022-02-25 19:42:56 +00:00
|
|
|
|
2021-11-03 17:28:43 +00:00
|
|
|
Ok(true)
|
2020-04-22 20:32:58 +00:00
|
|
|
} else {
|
2021-03-15 21:36:41 +00:00
|
|
|
let delay = tokio::time::sleep(Duration::from_millis(time_msec - now));
|
2020-04-22 20:32:58 +00:00
|
|
|
select! {
|
2021-04-23 19:57:32 +00:00
|
|
|
_ = delay.fuse() => {},
|
|
|
|
_ = self.resync_notify.notified().fuse() => {},
|
|
|
|
_ = must_exit.changed().fuse() => {},
|
2020-04-17 13:36:16 +00:00
|
|
|
}
|
2021-11-03 17:28:43 +00:00
|
|
|
Ok(false)
|
2020-04-11 21:00:26 +00:00
|
|
|
}
|
2021-03-15 19:09:44 +00:00
|
|
|
} else {
|
2022-03-15 15:47:13 +00:00
|
|
|
// 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.
|
|
|
|
let delay = tokio::time::sleep(Duration::from_secs(10));
|
2021-03-15 19:09:44 +00:00
|
|
|
select! {
|
2022-03-15 15:47:13 +00:00
|
|
|
_ = delay.fuse() => {},
|
2021-04-23 19:57:32 +00:00
|
|
|
_ = self.resync_notify.notified().fuse() => {},
|
|
|
|
_ = must_exit.changed().fuse() => {},
|
2021-03-15 19:09:44 +00:00
|
|
|
}
|
2021-11-03 17:28:43 +00:00
|
|
|
Ok(false)
|
2020-04-11 21:00:26 +00:00
|
|
|
}
|
|
|
|
}
|
2020-04-17 13:36:16 +00:00
|
|
|
|
2021-03-15 19:09:44 +00:00
|
|
|
async fn resync_block(&self, hash: &Hash) -> Result<(), Error> {
|
2021-10-22 10:09:03 +00:00
|
|
|
let BlockStatus { exists, needed } = self
|
|
|
|
.mutation_lock
|
|
|
|
.lock()
|
|
|
|
.await
|
|
|
|
.check_block_status(hash, self)
|
|
|
|
.await?;
|
2020-04-17 13:36:16 +00:00
|
|
|
|
2021-10-28 12:32:55 +00:00
|
|
|
if exists != needed.is_needed() || exists != needed.is_nonzero() {
|
|
|
|
debug!(
|
|
|
|
"Resync block {:?}: exists {}, nonzero rc {}, deletable {}",
|
|
|
|
hash,
|
|
|
|
exists,
|
|
|
|
needed.is_nonzero(),
|
|
|
|
needed.is_deletable(),
|
2020-04-21 12:54:55 +00:00
|
|
|
);
|
|
|
|
}
|
2020-04-17 15:09:57 +00:00
|
|
|
|
2021-10-28 12:32:55 +00:00
|
|
|
if exists && needed.is_deletable() {
|
|
|
|
info!("Resync block {:?}: offloading and deleting", hash);
|
2021-02-24 10:58:03 +00:00
|
|
|
|
2021-10-26 08:20:05 +00:00
|
|
|
let mut who = self.replication.write_nodes(hash);
|
2021-03-16 10:14:27 +00:00
|
|
|
if who.len() < self.replication.write_quorum() {
|
2021-04-23 19:57:32 +00:00
|
|
|
return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string()));
|
2021-03-11 18:06:27 +00:00
|
|
|
}
|
2021-02-24 10:58:03 +00:00
|
|
|
who.retain(|id| *id != self.system.id);
|
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
let msg = Arc::new(BlockRpc::NeedBlockQuery(*hash));
|
2021-02-24 10:58:03 +00:00
|
|
|
let who_needs_fut = who.iter().map(|to| {
|
2021-10-14 09:50:12 +00:00
|
|
|
self.system.rpc.call_arc(
|
|
|
|
&self.endpoint,
|
|
|
|
*to,
|
|
|
|
msg.clone(),
|
2021-10-26 17:13:41 +00:00
|
|
|
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
2021-10-14 09:50:12 +00:00
|
|
|
.with_timeout(NEED_BLOCK_QUERY_TIMEOUT),
|
|
|
|
)
|
2021-02-24 10:58:03 +00:00
|
|
|
});
|
|
|
|
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()) {
|
2021-10-26 17:13:41 +00:00
|
|
|
match needed.err_context("NeedBlockQuery RPC")? {
|
2021-10-14 09:50:12 +00:00
|
|
|
BlockRpc::NeedBlockReply(needed) => {
|
2021-02-24 10:58:03 +00:00
|
|
|
if needed {
|
|
|
|
need_nodes.push(*node);
|
2020-04-17 17:16:08 +00:00
|
|
|
}
|
|
|
|
}
|
2022-01-03 12:58:05 +00:00
|
|
|
m => {
|
|
|
|
return Err(Error::unexpected_rpc_message(m));
|
2021-02-24 10:58:03 +00:00
|
|
|
}
|
2020-04-17 17:16:08 +00:00
|
|
|
}
|
2021-02-24 10:58:03 +00:00
|
|
|
}
|
|
|
|
|
2021-04-23 19:57:32 +00:00
|
|
|
if !need_nodes.is_empty() {
|
2021-03-05 14:09:18 +00:00
|
|
|
trace!(
|
|
|
|
"Block {:?} needed by {} nodes, sending",
|
|
|
|
hash,
|
|
|
|
need_nodes.len()
|
|
|
|
);
|
2020-04-17 17:16:08 +00:00
|
|
|
|
2022-02-16 13:23:04 +00:00
|
|
|
for node in need_nodes.iter() {
|
|
|
|
self.metrics
|
|
|
|
.resync_send_counter
|
|
|
|
.add(1, &[KeyValue::new("to", format!("{:?}", node))]);
|
|
|
|
}
|
|
|
|
|
2021-10-28 12:32:55 +00:00
|
|
|
let put_block_message = self.read_block(hash).await?;
|
2021-10-14 09:50:12 +00:00
|
|
|
self.system
|
|
|
|
.rpc
|
2021-03-12 20:52:19 +00:00
|
|
|
.try_call_many(
|
2021-10-14 09:50:12 +00:00
|
|
|
&self.endpoint,
|
2021-03-12 18:57:37 +00:00
|
|
|
&need_nodes[..],
|
|
|
|
put_block_message,
|
2021-10-26 17:13:41 +00:00
|
|
|
RequestStrategy::with_priority(PRIO_BACKGROUND)
|
2021-10-14 09:50:12 +00:00
|
|
|
.with_quorum(need_nodes.len())
|
2021-03-12 20:52:19 +00:00
|
|
|
.with_timeout(BLOCK_RW_TIMEOUT),
|
|
|
|
)
|
2021-10-28 12:32:55 +00:00
|
|
|
.await
|
|
|
|
.err_context("PutBlock RPC")?;
|
2020-04-17 15:09:57 +00:00
|
|
|
}
|
2021-03-16 10:14:27 +00:00
|
|
|
info!(
|
2021-10-28 12:32:55 +00:00
|
|
|
"Deleting unneeded block {:?}, offload finished ({} / {})",
|
2021-03-05 14:09:18 +00:00
|
|
|
hash,
|
|
|
|
need_nodes.len(),
|
|
|
|
who.len()
|
|
|
|
);
|
2021-02-24 10:58:03 +00:00
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
self.mutation_lock
|
|
|
|
.lock()
|
|
|
|
.await
|
|
|
|
.delete_if_unneeded(hash, self)
|
|
|
|
.await?;
|
2021-10-28 12:32:55 +00:00
|
|
|
|
2022-03-15 11:23:33 +00:00
|
|
|
self.rc.clear_deleted_block_rc(hash)?;
|
2020-04-17 13:36:16 +00:00
|
|
|
}
|
|
|
|
|
2021-10-28 12:32:55 +00:00
|
|
|
if needed.is_nonzero() && !exists {
|
|
|
|
info!(
|
|
|
|
"Resync block {:?}: fetching absent but needed block (refcount > 0)",
|
|
|
|
hash
|
|
|
|
);
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
let block_data = self.rpc_get_raw_block(hash).await?;
|
2022-02-16 13:23:04 +00:00
|
|
|
|
|
|
|
self.metrics.resync_recv_counter.add(1);
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
self.write_block(hash, &block_data).await?;
|
2020-04-17 13:36:16 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Ok(())
|
|
|
|
}
|
2020-04-18 17:21:34 +00:00
|
|
|
|
2021-10-28 12:32:55 +00:00
|
|
|
// ---- Utility: iteration on files in the data directory ----
|
|
|
|
|
2021-06-23 23:34:28 +00:00
|
|
|
async fn for_each_file<F, Fut, State>(
|
|
|
|
&self,
|
|
|
|
state: State,
|
|
|
|
mut f: F,
|
|
|
|
must_exit: &watch::Receiver<bool>,
|
|
|
|
) -> Result<(), Error>
|
|
|
|
where
|
|
|
|
F: FnMut(State, Hash) -> Fut + Send,
|
|
|
|
Fut: Future<Output = Result<State, Error>> + Send,
|
|
|
|
State: Send,
|
|
|
|
{
|
|
|
|
self.for_each_file_rec(&self.data_dir, state, &mut f, must_exit)
|
|
|
|
.await
|
|
|
|
.map(|_| ())
|
|
|
|
}
|
|
|
|
|
|
|
|
fn for_each_file_rec<'a, F, Fut, State>(
|
2021-03-05 14:09:18 +00:00
|
|
|
&'a self,
|
2021-04-23 19:57:32 +00:00
|
|
|
path: &'a Path,
|
2021-06-23 23:34:28 +00:00
|
|
|
mut state: State,
|
|
|
|
f: &'a mut F,
|
2021-03-05 14:09:18 +00:00
|
|
|
must_exit: &'a watch::Receiver<bool>,
|
2021-06-23 23:34:28 +00:00
|
|
|
) -> BoxFuture<'a, Result<State, Error>>
|
|
|
|
where
|
|
|
|
F: FnMut(State, Hash) -> Fut + Send,
|
|
|
|
Fut: Future<Output = Result<State, Error>> + Send,
|
|
|
|
State: Send + 'a,
|
|
|
|
{
|
2021-02-24 10:58:03 +00:00
|
|
|
async move {
|
|
|
|
let mut ls_data_dir = fs::read_dir(path).await?;
|
2021-06-23 23:34:28 +00:00
|
|
|
while let Some(data_dir_ent) = ls_data_dir.next_entry().await? {
|
|
|
|
if *must_exit.borrow() {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
2021-02-24 10:58:03 +00:00
|
|
|
let name = data_dir_ent.file_name();
|
2021-06-23 23:34:28 +00:00
|
|
|
let name = if let Ok(n) = name.into_string() {
|
|
|
|
n
|
|
|
|
} else {
|
|
|
|
continue;
|
2020-04-19 20:36:36 +00:00
|
|
|
};
|
2021-02-24 10:58:03 +00:00
|
|
|
let ent_type = data_dir_ent.file_type().await?;
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
let name = name.strip_suffix(".zst").unwrap_or(&name);
|
2021-02-24 10:58:03 +00:00
|
|
|
if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() {
|
2021-06-23 23:34:28 +00:00
|
|
|
state = self
|
|
|
|
.for_each_file_rec(&data_dir_ent.path(), state, f, must_exit)
|
2021-03-05 14:09:18 +00:00
|
|
|
.await?;
|
2021-02-24 10:58:03 +00:00
|
|
|
} else if name.len() == 64 {
|
2021-06-23 23:34:28 +00:00
|
|
|
let hash_bytes = if let Ok(h) = hex::decode(&name) {
|
|
|
|
h
|
|
|
|
} else {
|
|
|
|
continue;
|
2021-02-24 10:58:03 +00:00
|
|
|
};
|
|
|
|
let mut hash = [0u8; 32];
|
|
|
|
hash.copy_from_slice(&hash_bytes[..]);
|
2021-06-23 23:34:28 +00:00
|
|
|
state = f(state, hash.into()).await?;
|
2020-04-19 20:36:36 +00:00
|
|
|
}
|
|
|
|
}
|
2021-06-23 23:34:28 +00:00
|
|
|
Ok(state)
|
2021-03-05 14:09:18 +00:00
|
|
|
}
|
|
|
|
.boxed()
|
2020-04-19 20:36:36 +00:00
|
|
|
}
|
2020-04-17 13:36:16 +00:00
|
|
|
}
|
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
#[async_trait]
|
|
|
|
impl EndpointHandler<BlockRpc> for BlockManager {
|
2021-10-15 09:05:09 +00:00
|
|
|
async fn handle(
|
|
|
|
self: &Arc<Self>,
|
|
|
|
message: &BlockRpc,
|
|
|
|
_from: NodeID,
|
|
|
|
) -> Result<BlockRpc, Error> {
|
|
|
|
match message {
|
2021-12-15 10:26:43 +00:00
|
|
|
BlockRpc::PutBlock { hash, data } => self.write_block(hash, data).await,
|
2021-10-15 09:05:09 +00:00
|
|
|
BlockRpc::GetBlock(h) => self.read_block(h).await,
|
|
|
|
BlockRpc::NeedBlockQuery(h) => self.need_block(h).await.map(BlockRpc::NeedBlockReply),
|
2022-01-03 12:58:05 +00:00
|
|
|
m => Err(Error::unexpected_rpc_message(m)),
|
2021-10-15 09:05:09 +00:00
|
|
|
}
|
2021-10-14 09:50:12 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
struct BlockStatus {
|
|
|
|
exists: bool,
|
2021-10-28 12:32:55 +00:00
|
|
|
needed: RcEntry,
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
impl BlockManagerLocked {
|
|
|
|
async fn check_block_status(
|
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
|
|
|
mgr: &BlockManager,
|
|
|
|
) -> Result<BlockStatus, Error> {
|
2021-12-15 10:26:43 +00:00
|
|
|
let exists = mgr.is_block_compressed(hash).await.is_ok();
|
2022-03-15 11:23:33 +00:00
|
|
|
let needed = mgr.rc.get_block_rc(hash)?;
|
2021-10-22 10:09:03 +00:00
|
|
|
|
|
|
|
Ok(BlockStatus { exists, needed })
|
|
|
|
}
|
|
|
|
|
|
|
|
async fn write_block(
|
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
2021-12-15 10:26:43 +00:00
|
|
|
data: &DataBlock,
|
2021-10-22 10:09:03 +00:00
|
|
|
mgr: &BlockManager,
|
|
|
|
) -> Result<BlockRpc, Error> {
|
2021-12-15 10:26:43 +00:00
|
|
|
let compressed = data.is_compressed();
|
|
|
|
let data = data.inner_buffer();
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
let mut path = mgr.block_dir(hash);
|
2022-03-01 10:52:12 +00:00
|
|
|
let directory = path.clone();
|
2021-10-22 10:09:03 +00:00
|
|
|
path.push(hex::encode(hash));
|
2022-03-01 10:52:12 +00:00
|
|
|
|
|
|
|
fs::create_dir_all(&directory).await?;
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
let to_delete = match (mgr.is_block_compressed(hash).await, compressed) {
|
|
|
|
(Ok(true), _) => return Ok(BlockRpc::Ok),
|
|
|
|
(Ok(false), false) => return Ok(BlockRpc::Ok),
|
|
|
|
(Ok(false), true) => {
|
|
|
|
let path_to_delete = path.clone();
|
|
|
|
path.set_extension("zst");
|
|
|
|
Some(path_to_delete)
|
|
|
|
}
|
|
|
|
(Err(_), compressed) => {
|
|
|
|
if compressed {
|
|
|
|
path.set_extension("zst");
|
|
|
|
}
|
|
|
|
None
|
|
|
|
}
|
|
|
|
};
|
2021-10-22 10:09:03 +00:00
|
|
|
|
|
|
|
let mut path2 = path.clone();
|
|
|
|
path2.set_extension("tmp");
|
|
|
|
let mut f = fs::File::create(&path2).await?;
|
|
|
|
f.write_all(data).await?;
|
2022-03-01 10:52:12 +00:00
|
|
|
f.sync_all().await?;
|
2021-10-22 10:09:03 +00:00
|
|
|
drop(f);
|
|
|
|
|
|
|
|
fs::rename(path2, path).await?;
|
2021-12-15 10:26:43 +00:00
|
|
|
if let Some(to_delete) = to_delete {
|
|
|
|
fs::remove_file(to_delete).await?;
|
|
|
|
}
|
2021-10-22 10:09:03 +00:00
|
|
|
|
2022-03-14 10:54:00 +00:00
|
|
|
// We want to ensure that when this function returns, data is properly persisted
|
|
|
|
// to disk. The first step is the sync_all above that does an fsync on the data file.
|
|
|
|
// Now, we do an fsync on the containing directory, to ensure that the rename
|
|
|
|
// is persisted properly. See:
|
|
|
|
// http://thedjbway.b0llix.net/qmail/syncdir.html
|
2022-03-01 10:52:12 +00:00
|
|
|
let dir = fs::OpenOptions::new()
|
|
|
|
.read(true)
|
|
|
|
.mode(0)
|
|
|
|
.open(directory)
|
|
|
|
.await?;
|
|
|
|
dir.sync_all().await?;
|
|
|
|
drop(dir);
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
Ok(BlockRpc::Ok)
|
|
|
|
}
|
|
|
|
|
|
|
|
async fn move_block_to_corrupted(&self, hash: &Hash, mgr: &BlockManager) -> Result<(), Error> {
|
|
|
|
warn!(
|
|
|
|
"Block {:?} is corrupted. Renaming to .corrupted and resyncing.",
|
|
|
|
hash
|
|
|
|
);
|
2021-12-15 10:26:43 +00:00
|
|
|
let mut path = mgr.block_path(hash);
|
2021-10-22 10:09:03 +00:00
|
|
|
let mut path2 = path.clone();
|
2021-12-15 10:26:43 +00:00
|
|
|
if mgr.is_block_compressed(hash).await? {
|
|
|
|
path.set_extension("zst");
|
|
|
|
path2.set_extension("zst.corrupted");
|
|
|
|
} else {
|
|
|
|
path2.set_extension("corrupted");
|
|
|
|
}
|
2021-10-22 10:09:03 +00:00
|
|
|
fs::rename(path, path2).await?;
|
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
|
|
|
|
async fn delete_if_unneeded(&self, hash: &Hash, mgr: &BlockManager) -> Result<(), Error> {
|
|
|
|
let BlockStatus { exists, needed } = self.check_block_status(hash, mgr).await?;
|
|
|
|
|
2021-10-28 12:32:55 +00:00
|
|
|
if exists && needed.is_deletable() {
|
2021-12-15 10:26:43 +00:00
|
|
|
let mut path = mgr.block_path(hash);
|
|
|
|
if mgr.is_block_compressed(hash).await? {
|
|
|
|
path.set_extension("zst");
|
|
|
|
}
|
2021-10-22 10:09:03 +00:00
|
|
|
fs::remove_file(path).await?;
|
2022-02-16 13:23:04 +00:00
|
|
|
mgr.metrics.delete_counter.add(1);
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-02-25 19:42:56 +00:00
|
|
|
/// 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,
|
|
|
|
}
|
|
|
|
|
2022-03-01 13:55:37 +00:00
|
|
|
impl ErrorCounter {
|
|
|
|
fn new(now: u64) -> Self {
|
2022-02-25 19:42:56 +00:00
|
|
|
Self {
|
|
|
|
errors: 1,
|
2022-03-01 13:55:37 +00:00
|
|
|
last_try: now,
|
2022-02-25 19:42:56 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-06 15:01:49 +00:00
|
|
|
fn decode(data: &[u8]) -> Self {
|
2022-02-25 19:42:56 +00:00
|
|
|
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()
|
|
|
|
}
|
|
|
|
|
2022-03-01 13:55:37 +00:00
|
|
|
fn add1(self, now: u64) -> Self {
|
2022-02-25 19:42:56 +00:00
|
|
|
Self {
|
|
|
|
errors: self.errors + 1,
|
2022-03-01 13:55:37 +00:00
|
|
|
last_try: now,
|
2022-02-25 19:42:56 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
fn delay_msec(&self) -> u64 {
|
2022-04-07 08:24:20 +00:00
|
|
|
(RESYNC_RETRY_DELAY.as_millis() as u64)
|
|
|
|
<< std::cmp::min(self.errors - 1, RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER)
|
2022-02-25 19:42:56 +00:00
|
|
|
}
|
|
|
|
fn next_try(&self) -> u64 {
|
|
|
|
self.last_try + self.delay_msec()
|
|
|
|
}
|
|
|
|
}
|