2022-07-08 11:30:26 +00:00
|
|
|
use std::path::PathBuf;
|
2022-07-22 16:20:27 +00:00
|
|
|
use std::pin::Pin;
|
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
|
|
|
|
2022-12-14 11:28:07 +00:00
|
|
|
use arc_swap::ArcSwapOption;
|
2021-10-14 09:50:12 +00:00
|
|
|
use async_trait::async_trait;
|
2022-07-18 15:18:47 +00:00
|
|
|
use bytes::Bytes;
|
2023-01-03 16:29:11 +00:00
|
|
|
use rand::prelude::*;
|
2022-02-17 22:28:23 +00:00
|
|
|
use serde::{Deserialize, Serialize};
|
|
|
|
|
2022-09-06 17:31:42 +00:00
|
|
|
use futures::Stream;
|
2022-07-22 16:20:27 +00:00
|
|
|
use futures_util::stream::StreamExt;
|
2020-04-09 21:45:07 +00:00
|
|
|
use tokio::fs;
|
2022-07-22 16:20:27 +00:00
|
|
|
use tokio::io::{AsyncReadExt, AsyncWriteExt, BufReader};
|
2022-09-12 14:57:38 +00:00
|
|
|
use tokio::sync::{mpsc, Mutex, MutexGuard};
|
2022-02-17 22:28:23 +00:00
|
|
|
|
2022-02-18 19:39:55 +00:00
|
|
|
use opentelemetry::{
|
|
|
|
trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
|
2022-09-05 10:40:17 +00:00
|
|
|
Context,
|
2022-02-18 19:39:55 +00:00
|
|
|
};
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2022-07-22 16:20:27 +00:00
|
|
|
use garage_rpc::rpc_helper::netapp::stream::{stream_asyncread, ByteStream};
|
|
|
|
|
2022-06-08 08:01:44 +00:00
|
|
|
use garage_db as db;
|
|
|
|
|
2023-01-04 12:07:13 +00:00
|
|
|
use garage_util::background::{vars, BackgroundRunner};
|
2023-09-04 12:49:49 +00:00
|
|
|
use garage_util::config::DataDirEnum;
|
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;
|
2023-09-04 12:49:49 +00:00
|
|
|
use garage_util::persister::{Persister, PersisterShared};
|
2023-01-04 12:07:13 +00:00
|
|
|
use garage_util::time::msec_to_rfc3339;
|
2020-04-18 17:39:08 +00:00
|
|
|
|
2022-09-01 10:58:20 +00:00
|
|
|
use garage_rpc::rpc_helper::OrderTag;
|
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::*;
|
2023-09-04 12:49:49 +00:00
|
|
|
use crate::layout::*;
|
2022-03-15 11:31:23 +00:00
|
|
|
use crate::metrics::*;
|
2022-03-15 11:23:33 +00:00
|
|
|
use crate::rc::*;
|
2022-07-08 11:30:26 +00:00
|
|
|
use crate::repair::*;
|
2022-09-02 14:47:15 +00:00
|
|
|
use crate::resync::*;
|
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
|
|
|
// 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
|
2022-09-01 10:58:20 +00:00
|
|
|
GetBlock(Hash, Option<OrderTag>),
|
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,
|
2022-07-22 16:20:27 +00:00
|
|
|
header: DataBlockHeader,
|
2021-12-15 10:26:43 +00:00
|
|
|
},
|
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,
|
2023-09-04 12:49:49 +00:00
|
|
|
|
|
|
|
/// Directory/ies in which block are stored
|
|
|
|
pub data_dir: DataDirEnum,
|
|
|
|
/// Data layout
|
|
|
|
pub(crate) data_layout: DataLayout,
|
2022-03-15 11:31:23 +00:00
|
|
|
|
2023-06-09 14:23:21 +00:00
|
|
|
data_fsync: bool,
|
2022-03-15 11:04:12 +00:00
|
|
|
compression_level: Option<i32>,
|
2021-10-22 10:09:03 +00:00
|
|
|
|
2023-09-05 13:39:21 +00:00
|
|
|
mutation_lock: Vec<Mutex<BlockManagerLocked>>,
|
2020-04-22 20:32:58 +00:00
|
|
|
|
2022-07-08 11:30:26 +00:00
|
|
|
pub(crate) rc: BlockRc,
|
2022-09-02 14:47:15 +00:00
|
|
|
pub resync: BlockResyncManager,
|
2020-04-22 20:32:58 +00:00
|
|
|
|
2022-07-08 11:30:26 +00:00
|
|
|
pub(crate) system: Arc<System>,
|
2022-09-02 14:47:15 +00:00
|
|
|
pub(crate) endpoint: Arc<Endpoint<BlockRpc, Self>>,
|
2022-02-16 13:23:04 +00:00
|
|
|
|
2022-09-02 14:47:15 +00:00
|
|
|
pub(crate) metrics: BlockManagerMetrics,
|
2022-07-08 11:30:26 +00:00
|
|
|
|
2023-01-04 12:07:13 +00:00
|
|
|
pub scrub_persister: PersisterShared<ScrubWorkerPersisted>,
|
2022-12-14 11:28:07 +00:00
|
|
|
tx_scrub_command: ArcSwapOption<mpsc::Sender<ScrubWorkerCommand>>,
|
2020-04-09 21:45:07 +00:00
|
|
|
}
|
|
|
|
|
2022-12-13 13:23:45 +00:00
|
|
|
#[derive(Serialize, Deserialize, Clone, Debug)]
|
|
|
|
pub struct BlockResyncErrorInfo {
|
|
|
|
pub hash: Hash,
|
|
|
|
pub refcount: u64,
|
|
|
|
pub error_count: u64,
|
|
|
|
pub last_try: u64,
|
|
|
|
pub next_try: u64,
|
|
|
|
}
|
|
|
|
|
2023-09-05 13:39:21 +00:00
|
|
|
// The number of different mutexes used to parallelize write access to data blocks
|
|
|
|
const MUTEX_COUNT: usize = 256;
|
|
|
|
|
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-08 08:01:44 +00:00
|
|
|
db: &db::Db,
|
2023-09-04 12:49:49 +00:00
|
|
|
data_dir: DataDirEnum,
|
2023-06-09 14:23:21 +00:00
|
|
|
data_fsync: bool,
|
2022-03-15 11:04:12 +00:00
|
|
|
compression_level: Option<i32>,
|
2020-04-23 17:05:46 +00:00
|
|
|
replication: TableShardedReplication,
|
2020-04-18 17:21:34 +00:00
|
|
|
system: Arc<System>,
|
2023-09-05 13:39:21 +00:00
|
|
|
) -> Result<Arc<Self>, Error> {
|
|
|
|
// Load or compute layout, i.e. assignment of data blocks to the different data directories
|
2023-09-04 12:49:49 +00:00
|
|
|
let layout_persister: Persister<DataLayout> =
|
|
|
|
Persister::new(&system.metadata_dir, "data_layout");
|
|
|
|
let data_layout = match layout_persister.load() {
|
|
|
|
Ok(mut layout) => {
|
2023-09-05 13:39:21 +00:00
|
|
|
layout
|
|
|
|
.update(&data_dir)
|
|
|
|
.ok_or_message("invalid data_dir config")?;
|
2023-09-04 12:49:49 +00:00
|
|
|
layout
|
|
|
|
}
|
2023-09-05 13:39:21 +00:00
|
|
|
Err(_) => DataLayout::initialize(&data_dir).ok_or_message("invalid data_dir config")?,
|
2023-09-04 12:49:49 +00:00
|
|
|
};
|
2023-09-05 11:43:38 +00:00
|
|
|
layout_persister
|
|
|
|
.save(&data_layout)
|
|
|
|
.expect("cannot save data_layout");
|
2023-09-04 12:49:49 +00:00
|
|
|
|
2023-09-05 13:39:21 +00:00
|
|
|
// Open metadata tables
|
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
|
|
|
|
2022-09-02 14:47:15 +00:00
|
|
|
let resync = BlockResyncManager::new(db, &system);
|
2022-02-25 19:42:56 +00:00
|
|
|
|
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
|
|
|
|
2023-01-09 18:49:33 +00:00
|
|
|
let metrics = BlockManagerMetrics::new(
|
|
|
|
compression_level,
|
|
|
|
rc.rc.clone(),
|
|
|
|
resync.queue.clone(),
|
|
|
|
resync.errors.clone(),
|
|
|
|
);
|
2022-09-02 14:47:15 +00:00
|
|
|
|
2023-01-04 12:07:13 +00:00
|
|
|
let scrub_persister = PersisterShared::new(&system.metadata_dir, "scrub_info");
|
|
|
|
|
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,
|
2023-09-04 12:49:49 +00:00
|
|
|
data_layout,
|
2023-06-09 14:23:21 +00:00
|
|
|
data_fsync,
|
2022-03-15 11:04:12 +00:00
|
|
|
compression_level,
|
2023-09-05 13:39:21 +00:00
|
|
|
mutation_lock: vec![(); MUTEX_COUNT]
|
|
|
|
.iter()
|
|
|
|
.map(|_| Mutex::new(BlockManagerLocked()))
|
|
|
|
.collect::<Vec<_>>(),
|
2020-04-11 21:00:26 +00:00
|
|
|
rc,
|
2022-09-02 14:47:15 +00:00
|
|
|
resync,
|
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,
|
2023-01-04 12:07:13 +00:00
|
|
|
scrub_persister,
|
2022-12-14 11:28:07 +00:00
|
|
|
tx_scrub_command: ArcSwapOption::new(None),
|
2020-04-18 17:21:34 +00:00
|
|
|
});
|
2021-10-14 09:50:12 +00:00
|
|
|
block_manager.endpoint.set_handler(block_manager.clone());
|
2023-03-09 17:08:47 +00:00
|
|
|
block_manager.scrub_persister.set_with(|_| ()).unwrap();
|
2020-04-23 14:40:59 +00:00
|
|
|
|
2023-09-05 13:39:21 +00:00
|
|
|
Ok(block_manager)
|
2022-12-14 11:28:07 +00:00
|
|
|
}
|
|
|
|
|
2022-12-14 11:51:16 +00:00
|
|
|
pub fn spawn_workers(self: &Arc<Self>, bg: &BackgroundRunner) {
|
2022-09-02 15:18:13 +00:00
|
|
|
// Spawn a bunch of resync workers
|
|
|
|
for index in 0..MAX_RESYNC_WORKERS {
|
2022-12-14 11:28:07 +00:00
|
|
|
let worker = ResyncWorker::new(index, self.clone());
|
2022-12-14 11:51:16 +00:00
|
|
|
bg.spawn_worker(worker);
|
2022-09-02 15:18:13 +00:00
|
|
|
}
|
2022-09-02 14:47:15 +00:00
|
|
|
|
|
|
|
// Spawn scrub worker
|
2022-12-14 11:28:07 +00:00
|
|
|
let (scrub_tx, scrub_rx) = mpsc::channel(1);
|
|
|
|
self.tx_scrub_command.store(Some(Arc::new(scrub_tx)));
|
2023-01-04 12:07:13 +00:00
|
|
|
bg.spawn_worker(ScrubWorker::new(
|
|
|
|
self.clone(),
|
|
|
|
scrub_rx,
|
|
|
|
self.scrub_persister.clone(),
|
|
|
|
));
|
|
|
|
}
|
|
|
|
|
|
|
|
pub fn register_bg_vars(&self, vars: &mut vars::BgVars) {
|
|
|
|
self.resync.register_bg_vars(vars);
|
|
|
|
|
|
|
|
vars.register_rw(
|
|
|
|
&self.scrub_persister,
|
|
|
|
"scrub-tranquility",
|
|
|
|
|p| p.get_with(|x| x.tranquility),
|
|
|
|
|p, tranquility| p.set_with(|x| x.tranquility = tranquility),
|
|
|
|
);
|
|
|
|
vars.register_ro(&self.scrub_persister, "scrub-last-completed", |p| {
|
|
|
|
p.get_with(|x| msec_to_rfc3339(x.time_last_complete_scrub))
|
|
|
|
});
|
2023-03-06 12:31:03 +00:00
|
|
|
vars.register_ro(&self.scrub_persister, "scrub-next-run", |p| {
|
|
|
|
p.get_with(|x| msec_to_rfc3339(x.time_next_run_scrub))
|
|
|
|
});
|
2023-01-04 12:07:13 +00:00
|
|
|
vars.register_ro(&self.scrub_persister, "scrub-corruptions_detected", |p| {
|
|
|
|
p.get_with(|x| x.corruptions_detected)
|
|
|
|
});
|
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
|
2022-07-22 16:20:27 +00:00
|
|
|
/// Return it as a stream with a header
|
|
|
|
async fn rpc_get_raw_block_streaming(
|
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
2022-09-01 10:58:20 +00:00
|
|
|
order_tag: Option<OrderTag>,
|
2022-07-22 16:20:27 +00:00
|
|
|
) -> Result<(DataBlockHeader, ByteStream), Error> {
|
2023-09-05 13:39:21 +00:00
|
|
|
self.rpc_get_raw_block_internal(hash, order_tag, |header, stream| async move {
|
|
|
|
Ok((header, stream))
|
|
|
|
})
|
|
|
|
.await
|
2022-07-22 16:20:27 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Ask nodes that might have a (possibly compressed) block for it
|
|
|
|
/// Return its entire body
|
2022-09-06 20:13:01 +00:00
|
|
|
pub(crate) async fn rpc_get_raw_block(
|
2022-09-01 10:58:20 +00:00
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
|
|
|
order_tag: Option<OrderTag>,
|
|
|
|
) -> Result<DataBlock, Error> {
|
2023-09-05 13:39:21 +00:00
|
|
|
self.rpc_get_raw_block_internal(hash, order_tag, |header, stream| async move {
|
|
|
|
read_stream_to_end(stream)
|
|
|
|
.await
|
|
|
|
.map(|data| DataBlock::from_parts(header, data))
|
|
|
|
})
|
|
|
|
.await
|
|
|
|
}
|
|
|
|
|
|
|
|
async fn rpc_get_raw_block_internal<F, Fut, T>(
|
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
|
|
|
order_tag: Option<OrderTag>,
|
|
|
|
f: F,
|
|
|
|
) -> Result<T, Error>
|
|
|
|
where
|
|
|
|
F: Fn(DataBlockHeader, ByteStream) -> Fut,
|
|
|
|
Fut: futures::Future<Output = Result<T, Error>>,
|
|
|
|
{
|
2021-10-26 08:20:05 +00:00
|
|
|
let who = self.replication.read_nodes(hash);
|
2022-09-06 17:31:42 +00:00
|
|
|
let who = self.system.rpc.request_order(&who);
|
2021-10-22 10:09:03 +00:00
|
|
|
|
2022-07-22 16:20:27 +00:00
|
|
|
for node in who.iter() {
|
|
|
|
let node_id = NodeID::from(*node);
|
2022-09-01 10:58:20 +00:00
|
|
|
let rpc = self.endpoint.call_streaming(
|
|
|
|
&node_id,
|
|
|
|
BlockRpc::GetBlock(*hash, order_tag),
|
2022-09-01 14:35:43 +00:00
|
|
|
PRIO_NORMAL | PRIO_SECONDARY,
|
2022-09-01 10:58:20 +00:00
|
|
|
);
|
2022-07-22 16:20:27 +00:00
|
|
|
tokio::select! {
|
|
|
|
res = rpc => {
|
|
|
|
let res = match res {
|
|
|
|
Ok(res) => res,
|
|
|
|
Err(e) => {
|
2023-09-06 15:49:30 +00:00
|
|
|
debug!("Get block {:?}: node {:?} could not be contacted: {}", hash, node, e);
|
2022-07-22 16:20:27 +00:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
let (header, stream) = match res.into_parts() {
|
|
|
|
(Ok(BlockRpc::PutBlock { hash: _, header }), Some(stream)) => (header, stream),
|
2023-09-06 15:49:30 +00:00
|
|
|
(Ok(_), _) => {
|
2023-09-05 13:57:25 +00:00
|
|
|
debug!("Get block {:?}: node {:?} returned a malformed response", hash, node);
|
2022-07-22 16:20:27 +00:00
|
|
|
continue;
|
|
|
|
}
|
2023-09-06 15:49:30 +00:00
|
|
|
(Err(e), _) => {
|
|
|
|
debug!("Get block {:?}: node {:?} returned error: {}", hash, node, e);
|
|
|
|
continue;
|
|
|
|
}
|
2022-07-22 16:20:27 +00:00
|
|
|
};
|
2023-09-05 13:39:21 +00:00
|
|
|
match f(header, stream).await {
|
|
|
|
Ok(ret) => return Ok(ret),
|
2022-07-22 16:20:27 +00:00
|
|
|
Err(e) => {
|
2023-09-05 13:57:25 +00:00
|
|
|
debug!("Get block {:?}: error reading stream from node {:?}: {}", hash, node, e);
|
2022-07-22 16:20:27 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-09-05 13:39:21 +00:00
|
|
|
// TODO: sleep less long (fail early), initiate a second request earlier
|
|
|
|
// if the first one doesn't succeed rapidly
|
|
|
|
// TODO: keep first request running when initiating a new one and take the
|
|
|
|
// one that finishes earlier
|
2022-09-19 18:12:19 +00:00
|
|
|
_ = tokio::time::sleep(self.system.rpc.rpc_timeout()) => {
|
2023-09-05 13:57:25 +00:00
|
|
|
debug!("Get block {:?}: node {:?} didn't return block in time, trying next.", hash, node);
|
2022-07-22 16:20:27 +00:00
|
|
|
}
|
|
|
|
};
|
2020-04-17 18:58:10 +00:00
|
|
|
}
|
2022-07-22 16:20:27 +00:00
|
|
|
|
2023-09-05 13:57:25 +00:00
|
|
|
let msg = format!("Get block {:?}: no node returned a valid block", hash);
|
|
|
|
debug!("{}", msg);
|
|
|
|
Err(Error::Message(msg))
|
2020-04-11 21:00:26 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
// ---- Public interface ----
|
|
|
|
|
2022-07-22 16:20:27 +00:00
|
|
|
/// Ask nodes that might have a block for it,
|
|
|
|
/// return it as a stream
|
|
|
|
pub async fn rpc_get_block_streaming(
|
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
2022-09-01 10:58:20 +00:00
|
|
|
order_tag: Option<OrderTag>,
|
2022-07-22 16:20:27 +00:00
|
|
|
) -> Result<
|
|
|
|
Pin<Box<dyn Stream<Item = Result<Bytes, std::io::Error>> + Send + Sync + 'static>>,
|
|
|
|
Error,
|
|
|
|
> {
|
2022-09-01 10:58:20 +00:00
|
|
|
let (header, stream) = self.rpc_get_raw_block_streaming(hash, order_tag).await?;
|
2022-07-22 16:20:27 +00:00
|
|
|
match header {
|
2022-09-12 14:57:38 +00:00
|
|
|
DataBlockHeader::Plain => Ok(stream),
|
2022-07-22 16:20:27 +00:00
|
|
|
DataBlockHeader::Compressed => {
|
|
|
|
// Too many things, I hate it.
|
|
|
|
let reader = stream_asyncread(stream);
|
|
|
|
let reader = BufReader::new(reader);
|
|
|
|
let reader = async_compression::tokio::bufread::ZstdDecoder::new(reader);
|
|
|
|
Ok(Box::pin(tokio_util::io::ReaderStream::new(reader)))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
/// Ask nodes that might have a block for it
|
2022-09-01 10:58:20 +00:00
|
|
|
pub async fn rpc_get_block(
|
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
|
|
|
order_tag: Option<OrderTag>,
|
|
|
|
) -> Result<Bytes, Error> {
|
|
|
|
self.rpc_get_raw_block(hash, order_tag)
|
|
|
|
.await?
|
|
|
|
.verify_get(*hash)
|
2021-12-15 10:26:43 +00:00
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Send block to nodes that should have it
|
2022-07-18 15:18:47 +00:00
|
|
|
pub async fn rpc_put_block(&self, hash: Hash, data: Bytes) -> Result<(), Error> {
|
2021-10-22 10:09:03 +00:00
|
|
|
let who = self.replication.write_nodes(&hash);
|
2022-07-22 16:20:27 +00:00
|
|
|
|
|
|
|
let (header, bytes) = DataBlock::from_buffer(data, self.compression_level)
|
|
|
|
.await
|
|
|
|
.into_parts();
|
|
|
|
let put_block_rpc =
|
|
|
|
Req::new(BlockRpc::PutBlock { hash, header })?.with_stream_from_buffer(bytes);
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
self.system
|
|
|
|
.rpc
|
|
|
|
.try_call_many(
|
|
|
|
&self.endpoint,
|
|
|
|
&who[..],
|
2022-07-22 16:20:27 +00:00
|
|
|
put_block_rpc,
|
2022-09-01 14:35:43 +00:00
|
|
|
RequestStrategy::with_priority(PRIO_NORMAL | PRIO_SECONDARY)
|
2022-09-19 18:12:19 +00:00
|
|
|
.with_quorum(self.replication.write_quorum()),
|
2021-10-22 10:09:03 +00:00
|
|
|
)
|
|
|
|
.await?;
|
2022-07-22 16:20:27 +00:00
|
|
|
|
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
|
|
|
/// Get number of items in the refcount table
|
2022-06-08 08:01:44 +00:00
|
|
|
pub fn rc_len(&self) -> Result<usize, Error> {
|
|
|
|
Ok(self.rc.rc.len()?)
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
|
2022-12-13 14:43:22 +00:00
|
|
|
/// Get number of items in the refcount table
|
|
|
|
pub fn rc_fast_len(&self) -> Result<Option<usize>, Error> {
|
|
|
|
Ok(self.rc.rc.fast_len()?)
|
|
|
|
}
|
|
|
|
|
2022-07-08 11:30:26 +00:00
|
|
|
/// Send command to start/stop/manager scrub worker
|
2022-12-14 11:28:07 +00:00
|
|
|
pub async fn send_scrub_command(&self, cmd: ScrubWorkerCommand) -> Result<(), Error> {
|
|
|
|
let tx = self.tx_scrub_command.load();
|
|
|
|
let tx = tx.as_ref().ok_or_message("scrub worker is not running")?;
|
|
|
|
tx.send(cmd).await.ok_or_message("send error")?;
|
|
|
|
Ok(())
|
2022-07-08 11:30:26 +00:00
|
|
|
}
|
|
|
|
|
2022-12-13 13:23:45 +00:00
|
|
|
/// Get the reference count of a block
|
|
|
|
pub fn get_block_rc(&self, hash: &Hash) -> Result<u64, Error> {
|
|
|
|
Ok(self.rc.get_block_rc(hash)?.as_u64())
|
|
|
|
}
|
|
|
|
|
|
|
|
/// List all resync errors
|
|
|
|
pub fn list_resync_errors(&self) -> Result<Vec<BlockResyncErrorInfo>, Error> {
|
|
|
|
let mut blocks = Vec::with_capacity(self.resync.errors.len());
|
|
|
|
for ent in self.resync.errors.iter()? {
|
|
|
|
let (hash, cnt) = ent?;
|
|
|
|
let cnt = ErrorCounter::decode(&cnt);
|
|
|
|
blocks.push(BlockResyncErrorInfo {
|
|
|
|
hash: Hash::try_from(&hash).unwrap(),
|
|
|
|
refcount: 0,
|
|
|
|
error_count: cnt.errors,
|
|
|
|
last_try: cnt.last_try,
|
|
|
|
next_try: cnt.next_try(),
|
|
|
|
});
|
|
|
|
}
|
|
|
|
for block in blocks.iter_mut() {
|
|
|
|
block.refcount = self.get_block_rc(&block.hash)?;
|
|
|
|
}
|
|
|
|
Ok(blocks)
|
|
|
|
}
|
|
|
|
|
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-08 08:01:44 +00:00
|
|
|
pub fn block_incref(
|
|
|
|
self: &Arc<Self>,
|
|
|
|
tx: &mut db::Transaction,
|
|
|
|
hash: Hash,
|
|
|
|
) -> db::TxOpResult<()> {
|
|
|
|
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-08 08:01:44 +00:00
|
|
|
let this = self.clone();
|
|
|
|
tokio::spawn(async move {
|
2022-09-19 18:12:19 +00:00
|
|
|
if let Err(e) = this
|
|
|
|
.resync
|
|
|
|
.put_to_resync(&hash, 2 * this.system.rpc.rpc_timeout())
|
|
|
|
{
|
2022-06-08 08:01:44 +00:00
|
|
|
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-08 08:01:44 +00:00
|
|
|
pub fn block_decref(
|
|
|
|
self: &Arc<Self>,
|
|
|
|
tx: &mut db::Transaction,
|
|
|
|
hash: Hash,
|
|
|
|
) -> db::TxOpResult<()> {
|
|
|
|
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-08 08:01:44 +00:00
|
|
|
let this = self.clone();
|
|
|
|
tokio::spawn(async move {
|
2022-09-02 14:47:15 +00:00
|
|
|
if let Err(e) = this
|
|
|
|
.resync
|
|
|
|
.put_to_resync(&hash, BLOCK_GC_DELAY + Duration::from_secs(10))
|
2022-06-08 08:01:44 +00:00
|
|
|
{
|
|
|
|
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 ----
|
|
|
|
|
2022-07-22 16:20:27 +00:00
|
|
|
async fn handle_put_block(
|
|
|
|
&self,
|
|
|
|
hash: Hash,
|
|
|
|
header: DataBlockHeader,
|
|
|
|
stream: Option<ByteStream>,
|
|
|
|
) -> Result<(), Error> {
|
|
|
|
let stream = stream.ok_or_message("missing stream")?;
|
|
|
|
let bytes = read_stream_to_end(stream).await?;
|
|
|
|
let data = DataBlock::from_parts(header, bytes);
|
|
|
|
self.write_block(&hash, &data).await
|
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
/// Write a block to disk
|
2022-09-06 20:13:01 +00:00
|
|
|
pub(crate) async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<(), Error> {
|
2022-07-18 16:40:57 +00:00
|
|
|
let tracer = opentelemetry::global::tracer("garage");
|
|
|
|
|
2022-09-12 14:57:38 +00:00
|
|
|
self.lock_mutate(hash)
|
2021-10-22 10:09:03 +00:00
|
|
|
.await
|
|
|
|
.write_block(hash, data, self)
|
2022-02-22 12:53:59 +00:00
|
|
|
.bound_record_duration(&self.metrics.block_write_duration)
|
2022-07-18 16:40:57 +00:00
|
|
|
.with_context(Context::current_with_span(
|
|
|
|
tracer.start("BlockManagerLocked::write_block"),
|
|
|
|
))
|
2022-02-16 13:23:04 +00:00
|
|
|
.await?;
|
|
|
|
|
2022-07-22 16:20:27 +00:00
|
|
|
Ok(())
|
2020-04-09 21:45:07 +00:00
|
|
|
}
|
|
|
|
|
2022-09-01 10:58:20 +00:00
|
|
|
async fn handle_get_block(&self, hash: &Hash, order_tag: Option<OrderTag>) -> Resp<BlockRpc> {
|
2022-07-22 16:20:27 +00:00
|
|
|
let block = match self.read_block(hash).await {
|
|
|
|
Ok(data) => data,
|
|
|
|
Err(e) => return Resp::new(Err(e)),
|
|
|
|
};
|
|
|
|
|
|
|
|
let (header, data) = block.into_parts();
|
2022-02-22 12:53:59 +00:00
|
|
|
|
2022-09-01 10:58:20 +00:00
|
|
|
let resp = Resp::new(Ok(BlockRpc::PutBlock {
|
2022-07-22 16:20:27 +00:00
|
|
|
hash: *hash,
|
|
|
|
header,
|
|
|
|
}))
|
2022-09-01 10:58:20 +00:00
|
|
|
.with_stream_from_buffer(data);
|
|
|
|
|
|
|
|
if let Some(order_tag) = order_tag {
|
|
|
|
resp.with_order_tag(order_tag)
|
|
|
|
} else {
|
|
|
|
resp
|
|
|
|
}
|
2022-07-22 16:20:27 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Read block from disk, verifying it's integrity
|
|
|
|
pub(crate) async fn read_block(&self, hash: &Hash) -> Result<DataBlock, Error> {
|
2023-09-07 10:41:36 +00:00
|
|
|
let tracer = opentelemetry::global::tracer("garage");
|
|
|
|
async {
|
|
|
|
match self.find_block(hash).await {
|
|
|
|
Some(p) => self.read_block_from(hash, &p).await,
|
|
|
|
None => {
|
|
|
|
// Not found but maybe we should have had it ??
|
|
|
|
self.resync
|
|
|
|
.put_to_resync(hash, 2 * self.system.rpc.rpc_timeout())?;
|
|
|
|
return Err(Error::Message(format!(
|
|
|
|
"block {:?} not found on node",
|
|
|
|
hash
|
|
|
|
)));
|
|
|
|
}
|
2020-04-17 16:38:11 +00:00
|
|
|
}
|
2023-09-05 13:04:59 +00:00
|
|
|
}
|
2023-09-07 10:41:36 +00:00
|
|
|
.bound_record_duration(&self.metrics.block_read_duration)
|
|
|
|
.with_context(Context::current_with_span(
|
|
|
|
tracer.start("BlockManager::read_block"),
|
|
|
|
))
|
|
|
|
.await
|
2023-09-05 13:04:59 +00:00
|
|
|
}
|
2021-12-15 10:26:43 +00:00
|
|
|
|
2023-09-05 13:04:59 +00:00
|
|
|
pub(crate) async fn read_block_from(
|
|
|
|
&self,
|
|
|
|
hash: &Hash,
|
|
|
|
block_path: &DataBlockPath,
|
|
|
|
) -> Result<DataBlock, Error> {
|
|
|
|
let (path, compressed) = match block_path {
|
2023-09-05 12:27:39 +00:00
|
|
|
DataBlockPath::Plain(p) => (p, false),
|
|
|
|
DataBlockPath::Compressed(p) => (p, true),
|
|
|
|
};
|
|
|
|
|
|
|
|
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?;
|
2023-09-07 10:41:36 +00:00
|
|
|
self.metrics.bytes_read.add(data.len() as u64);
|
2020-04-17 13:36:16 +00:00
|
|
|
drop(f);
|
|
|
|
|
2021-12-15 10:26:43 +00:00
|
|
|
let data = if compressed {
|
2022-07-22 16:20:27 +00:00
|
|
|
DataBlock::Compressed(data.into())
|
2021-12-15 10:26:43 +00:00
|
|
|
} else {
|
2022-07-22 16:20:27 +00:00
|
|
|
DataBlock::Plain(data.into())
|
2021-12-15 10:26:43 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
if data.verify(*hash).is_err() {
|
2022-02-16 13:23:04 +00:00
|
|
|
self.metrics.corruption_counter.add(1);
|
|
|
|
|
2023-09-05 12:27:39 +00:00
|
|
|
warn!(
|
|
|
|
"Block {:?} is corrupted. Renaming to .corrupted and resyncing.",
|
|
|
|
hash
|
|
|
|
);
|
2022-09-12 14:57:38 +00:00
|
|
|
self.lock_mutate(hash)
|
2021-10-22 10:09:03 +00:00
|
|
|
.await
|
2023-09-05 13:04:59 +00:00
|
|
|
.move_block_to_corrupted(block_path)
|
2021-10-22 10:09:03 +00:00
|
|
|
.await?;
|
2022-09-02 14:47:15 +00:00
|
|
|
self.resync.put_to_resync(hash, Duration::from_millis(0))?;
|
2023-09-05 12:27:39 +00:00
|
|
|
|
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
|
|
|
|
2022-09-02 14:52:22 +00:00
|
|
|
/// Check if this node should have a block, but don't actually have it
|
|
|
|
async fn need_block(&self, hash: &Hash) -> Result<bool, Error> {
|
2023-09-05 13:04:59 +00:00
|
|
|
let rc = self.rc.get_block_rc(hash)?;
|
|
|
|
let exists = self.find_block(hash).await.is_some();
|
|
|
|
Ok(rc.is_nonzero() && !exists)
|
2020-04-17 17:16:08 +00:00
|
|
|
}
|
|
|
|
|
2022-09-02 14:52:22 +00:00
|
|
|
/// Delete block if it is not needed anymore
|
|
|
|
pub(crate) async fn delete_if_unneeded(&self, hash: &Hash) -> Result<(), Error> {
|
2022-09-12 14:57:38 +00:00
|
|
|
self.lock_mutate(hash)
|
2022-09-02 14:52:22 +00:00
|
|
|
.await
|
|
|
|
.delete_if_unneeded(hash, self)
|
|
|
|
.await
|
|
|
|
}
|
|
|
|
|
2023-09-05 13:04:59 +00:00
|
|
|
/// Find the path where a block is currently stored
|
|
|
|
pub(crate) async fn find_block(&self, hash: &Hash) -> Option<DataBlockPath> {
|
2023-09-05 12:27:39 +00:00
|
|
|
let dirs = Some(self.data_layout.primary_block_dir(hash))
|
|
|
|
.into_iter()
|
|
|
|
.chain(self.data_layout.secondary_block_dirs(hash));
|
|
|
|
let filename = hex::encode(hash.as_ref());
|
2020-04-09 21:45:07 +00:00
|
|
|
|
2023-09-05 12:27:39 +00:00
|
|
|
for dir in dirs {
|
|
|
|
let mut path = dir;
|
|
|
|
path.push(&filename);
|
2023-05-08 17:57:10 +00:00
|
|
|
|
2023-09-05 12:27:39 +00:00
|
|
|
if self.compression_level.is_none() {
|
|
|
|
// If compression is disabled on node - check for the raw block
|
|
|
|
// first and then a compressed one (as compression may have been
|
|
|
|
// previously enabled).
|
2023-05-08 17:57:10 +00:00
|
|
|
if fs::metadata(&path).await.is_ok() {
|
2023-09-05 12:27:39 +00:00
|
|
|
return Some(DataBlockPath::Plain(path));
|
2023-05-08 17:57:10 +00:00
|
|
|
}
|
|
|
|
path.set_extension("zst");
|
2023-09-05 12:27:39 +00:00
|
|
|
if fs::metadata(&path).await.is_ok() {
|
|
|
|
return Some(DataBlockPath::Compressed(path));
|
|
|
|
}
|
|
|
|
} else {
|
2023-05-08 17:57:10 +00:00
|
|
|
path.set_extension("zst");
|
|
|
|
if fs::metadata(&path).await.is_ok() {
|
2023-09-05 12:27:39 +00:00
|
|
|
return Some(DataBlockPath::Compressed(path));
|
2023-05-08 17:57:10 +00:00
|
|
|
}
|
|
|
|
path.set_extension("");
|
2023-09-05 12:27:39 +00:00
|
|
|
if fs::metadata(&path).await.is_ok() {
|
|
|
|
return Some(DataBlockPath::Plain(path));
|
|
|
|
}
|
2023-05-08 17:57:10 +00:00
|
|
|
}
|
2021-12-15 10:26:43 +00:00
|
|
|
}
|
2023-09-05 12:27:39 +00:00
|
|
|
|
|
|
|
None
|
2021-12-15 10:26:43 +00:00
|
|
|
}
|
2022-09-12 14:57:38 +00:00
|
|
|
|
|
|
|
async fn lock_mutate(&self, hash: &Hash) -> MutexGuard<'_, BlockManagerLocked> {
|
|
|
|
let tracer = opentelemetry::global::tracer("garage");
|
2023-09-05 12:27:39 +00:00
|
|
|
let ilock = u16::from_be_bytes([hash.as_slice()[0], hash.as_slice()[1]]) as usize
|
|
|
|
% self.mutation_lock.len();
|
|
|
|
self.mutation_lock[ilock]
|
2022-09-12 14:57:38 +00:00
|
|
|
.lock()
|
|
|
|
.with_context(Context::current_with_span(
|
2023-09-05 12:27:39 +00:00
|
|
|
tracer.start(format!("Acquire mutation_lock #{}", ilock)),
|
2022-09-12 14:57:38 +00:00
|
|
|
))
|
|
|
|
.await
|
|
|
|
}
|
2020-04-17 13:36:16 +00:00
|
|
|
}
|
|
|
|
|
2021-10-14 09:50:12 +00:00
|
|
|
#[async_trait]
|
2022-07-22 16:20:27 +00:00
|
|
|
impl StreamingEndpointHandler<BlockRpc> for BlockManager {
|
|
|
|
async fn handle(self: &Arc<Self>, mut message: Req<BlockRpc>, _from: NodeID) -> Resp<BlockRpc> {
|
|
|
|
match message.msg() {
|
|
|
|
BlockRpc::PutBlock { hash, header } => Resp::new(
|
|
|
|
self.handle_put_block(*hash, *header, message.take_stream())
|
|
|
|
.await
|
2023-09-05 13:57:25 +00:00
|
|
|
.map(|()| BlockRpc::Ok),
|
2022-07-22 16:20:27 +00:00
|
|
|
),
|
2022-09-01 10:58:20 +00:00
|
|
|
BlockRpc::GetBlock(h, order_tag) => self.handle_get_block(h, *order_tag).await,
|
2022-07-22 16:20:27 +00:00
|
|
|
BlockRpc::NeedBlockQuery(h) => {
|
|
|
|
Resp::new(self.need_block(h).await.map(BlockRpc::NeedBlockReply))
|
|
|
|
}
|
|
|
|
m => Resp::new(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
|
|
|
impl BlockManagerLocked {
|
|
|
|
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,
|
2022-07-22 16:20:27 +00:00
|
|
|
) -> Result<(), Error> {
|
2021-12-15 10:26:43 +00:00
|
|
|
let compressed = data.is_compressed();
|
|
|
|
let data = data.inner_buffer();
|
|
|
|
|
2023-09-05 13:57:25 +00:00
|
|
|
let directory = mgr.data_layout.primary_block_dir(hash);
|
|
|
|
|
|
|
|
let mut tgt_path = directory.clone();
|
2023-09-05 12:37:10 +00:00
|
|
|
tgt_path.push(hex::encode(hash));
|
2023-09-05 13:04:59 +00:00
|
|
|
if compressed {
|
|
|
|
tgt_path.set_extension("zst");
|
|
|
|
}
|
2022-03-01 10:52:12 +00:00
|
|
|
|
2023-09-05 12:27:39 +00:00
|
|
|
let to_delete = match (mgr.find_block(hash).await, compressed) {
|
2023-09-05 13:04:59 +00:00
|
|
|
// If the block is stored in the wrong directory,
|
|
|
|
// write it again at the correct path and delete the old path
|
2023-09-05 12:37:10 +00:00
|
|
|
(Some(DataBlockPath::Plain(p)), false) if p != tgt_path => Some(p),
|
|
|
|
(Some(DataBlockPath::Compressed(p)), true) if p != tgt_path => Some(p),
|
|
|
|
|
2023-09-05 13:04:59 +00:00
|
|
|
// If the block is already stored not compressed but we have a compressed
|
|
|
|
// copy, write the compressed copy and delete the uncompressed one
|
2023-09-05 12:37:10 +00:00
|
|
|
(Some(DataBlockPath::Plain(plain_path)), true) => Some(plain_path),
|
|
|
|
|
2023-09-05 13:04:59 +00:00
|
|
|
// If the block is already stored compressed,
|
|
|
|
// keep the stored copy, we have nothing to do
|
2023-09-05 12:27:39 +00:00
|
|
|
(Some(DataBlockPath::Compressed(_)), _) => return Ok(()),
|
2023-09-05 12:37:10 +00:00
|
|
|
|
2023-09-05 13:04:59 +00:00
|
|
|
// If the block is already stored not compressed,
|
|
|
|
// and we don't have a compressed copy either,
|
|
|
|
// keep the stored copy, we have nothing to do
|
2023-09-05 12:27:39 +00:00
|
|
|
(Some(DataBlockPath::Plain(_)), false) => return Ok(()),
|
2023-09-05 12:37:10 +00:00
|
|
|
|
2023-09-05 13:04:59 +00:00
|
|
|
// If the block isn't stored already, just store what is given to us
|
2023-09-05 12:37:10 +00:00
|
|
|
(None, _) => None,
|
2021-12-15 10:26:43 +00:00
|
|
|
};
|
2021-10-22 10:09:03 +00:00
|
|
|
|
2023-09-05 12:37:10 +00:00
|
|
|
let mut path_tmp = tgt_path.clone();
|
2023-01-03 16:29:11 +00:00
|
|
|
let tmp_extension = format!("tmp{}", hex::encode(thread_rng().gen::<[u8; 4]>()));
|
|
|
|
path_tmp.set_extension(tmp_extension);
|
|
|
|
|
2023-09-05 13:57:25 +00:00
|
|
|
fs::create_dir_all(&directory).await?;
|
|
|
|
|
2023-01-03 16:29:11 +00:00
|
|
|
let mut delete_on_drop = DeleteOnDrop(Some(path_tmp.clone()));
|
|
|
|
|
|
|
|
let mut f = fs::File::create(&path_tmp).await?;
|
2021-10-22 10:09:03 +00:00
|
|
|
f.write_all(data).await?;
|
2023-09-07 10:41:36 +00:00
|
|
|
mgr.metrics.bytes_written.add(data.len() as u64);
|
2023-06-09 14:23:21 +00:00
|
|
|
|
|
|
|
if mgr.data_fsync {
|
|
|
|
f.sync_all().await?;
|
|
|
|
}
|
|
|
|
|
2021-10-22 10:09:03 +00:00
|
|
|
drop(f);
|
|
|
|
|
2023-09-05 12:37:10 +00:00
|
|
|
fs::rename(path_tmp, tgt_path).await?;
|
2023-01-03 16:29:11 +00:00
|
|
|
|
|
|
|
delete_on_drop.cancel();
|
|
|
|
|
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
|
|
|
|
2023-06-09 14:23:21 +00:00
|
|
|
if mgr.data_fsync {
|
|
|
|
// 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
|
|
|
|
let dir = fs::OpenOptions::new()
|
|
|
|
.read(true)
|
|
|
|
.mode(0)
|
|
|
|
.open(directory)
|
|
|
|
.await?;
|
|
|
|
dir.sync_all().await?;
|
|
|
|
drop(dir);
|
|
|
|
}
|
2022-03-01 10:52:12 +00:00
|
|
|
|
2022-07-22 16:20:27 +00:00
|
|
|
Ok(())
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
|
2023-09-05 12:27:39 +00:00
|
|
|
async fn move_block_to_corrupted(&self, block_path: &DataBlockPath) -> Result<(), Error> {
|
|
|
|
let (path, path2) = match block_path {
|
|
|
|
DataBlockPath::Plain(p) => {
|
|
|
|
let mut p2 = p.clone();
|
|
|
|
p2.set_extension("corrupted");
|
|
|
|
(p, p2)
|
|
|
|
}
|
|
|
|
DataBlockPath::Compressed(p) => {
|
|
|
|
let mut p2 = p.clone();
|
|
|
|
p2.set_extension("zst.corrupted");
|
|
|
|
(p, p2)
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
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> {
|
2023-09-05 13:04:59 +00:00
|
|
|
let rc = mgr.rc.get_block_rc(hash)?;
|
|
|
|
if rc.is_deletable() {
|
|
|
|
while let Some(path) = mgr.find_block(hash).await {
|
|
|
|
let path = match path {
|
|
|
|
DataBlockPath::Plain(p) | DataBlockPath::Compressed(p) => p,
|
|
|
|
};
|
2023-09-05 12:27:39 +00:00
|
|
|
fs::remove_file(path).await?;
|
|
|
|
mgr.metrics.delete_counter.add(1);
|
2021-12-15 10:26:43 +00:00
|
|
|
}
|
2021-10-22 10:09:03 +00:00
|
|
|
}
|
|
|
|
Ok(())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-07-22 16:20:27 +00:00
|
|
|
async fn read_stream_to_end(mut stream: ByteStream) -> Result<Bytes, Error> {
|
|
|
|
let mut parts: Vec<Bytes> = vec![];
|
|
|
|
while let Some(part) = stream.next().await {
|
|
|
|
parts.push(part.ok_or_message("error in stream")?);
|
|
|
|
}
|
|
|
|
|
|
|
|
Ok(parts
|
|
|
|
.iter()
|
|
|
|
.map(|x| &x[..])
|
|
|
|
.collect::<Vec<_>>()
|
|
|
|
.concat()
|
|
|
|
.into())
|
|
|
|
}
|
2023-01-03 16:29:11 +00:00
|
|
|
|
|
|
|
struct DeleteOnDrop(Option<PathBuf>);
|
|
|
|
|
|
|
|
impl DeleteOnDrop {
|
|
|
|
fn cancel(&mut self) {
|
|
|
|
drop(self.0.take());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
impl Drop for DeleteOnDrop {
|
|
|
|
fn drop(&mut self) {
|
|
|
|
if let Some(path) = self.0.take() {
|
|
|
|
tokio::spawn(async move {
|
|
|
|
if let Err(e) = fs::remove_file(&path).await {
|
|
|
|
debug!("DeleteOnDrop failed for {}: {}", path.display(), e);
|
|
|
|
}
|
|
|
|
});
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|