garage/src/block/manager.rs

865 lines
24 KiB
Rust
Raw Normal View History

use std::convert::TryInto;
use std::path::PathBuf;
use std::sync::Arc;
2022-02-22 13:52:41 +00:00
use std::time::Duration;
2020-04-09 21:45:07 +00:00
use arc_swap::{ArcSwap, 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;
use rand::prelude::*;
use serde::{Deserialize, Serialize};
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};
use tokio::sync::{mpsc, Mutex, MutexGuard, Semaphore};
use opentelemetry::{
trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
Context,
};
2020-04-09 21:45:07 +00:00
use garage_net::stream::{read_stream_to_end, stream_asyncread, ByteStream};
2022-07-22 16:20:27 +00:00
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
2022-06-08 08:01:44 +00:00
use garage_db as db;
use garage_util::background::{vars, BackgroundRunner};
use garage_util::config::Config;
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;
use garage_util::persister::{Persister, PersisterShared};
use garage_util::time::msec_to_rfc3339;
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::*;
use crate::layout::*;
use crate::metrics::*;
2022-03-15 11:23:33 +00:00
use crate::rc::*;
use crate::repair::*;
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
pub const INLINE_THRESHOLD: usize = 3072;
// 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);
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
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
PutBlock {
hash: Hash,
2022-07-22 16:20:27 +00:00
header: DataBlockHeader,
},
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),
}
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,
/// Data layout
pub(crate) data_layout: ArcSwap<DataLayout>,
/// Data layout persister
pub(crate) data_layout_persister: Persister<DataLayout>,
2023-06-09 14:23:21 +00:00
data_fsync: bool,
2022-03-15 11:04:12 +00:00
compression_level: Option<i32>,
disable_scrub: bool,
2023-09-05 13:39:21 +00:00
mutation_lock: Vec<Mutex<BlockManagerLocked>>,
pub rc: BlockRc,
pub resync: BlockResyncManager,
pub(crate) system: Arc<System>,
pub(crate) endpoint: Arc<Endpoint<BlockRpc, Self>>,
buffer_kb_semaphore: Arc<Semaphore>,
2022-02-16 13:23:04 +00:00
pub(crate) metrics: BlockManagerMetrics,
pub scrub_persister: PersisterShared<ScrubWorkerPersisted>,
tx_scrub_command: ArcSwapOption<mpsc::Sender<ScrubWorkerCommand>>,
2020-04-09 21:45:07 +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;
// 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(
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
2022-06-08 08:01:44 +00:00
db: &db::Db,
config: &Config,
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
let data_layout_persister: Persister<DataLayout> =
Persister::new(&system.metadata_dir, "data_layout");
let mut data_layout = match data_layout_persister.load() {
Ok(layout) => layout
.update(&config.data_dir)
.ok_or_message("invalid data_dir config")?,
Err(_) => {
DataLayout::initialize(&config.data_dir).ok_or_message("invalid data_dir config")?
}
};
data_layout.check_markers()?;
data_layout_persister
.save(&data_layout)
.expect("cannot save data_layout");
2023-09-05 13:39:21 +00:00
// Open metadata tables
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);
let resync = BlockResyncManager::new(db, &system);
let endpoint = system
.netapp
First implementation of K2V (#293) **Specification:** View spec at [this URL](https://git.deuxfleurs.fr/Deuxfleurs/garage/src/branch/k2v/doc/drafts/k2v-spec.md) - [x] Specify the structure of K2V triples - [x] Specify the DVVS format used for causality detection - [x] Specify the K2V index (just a counter of number of values per partition key) - [x] Specify single-item endpoints: ReadItem, InsertItem, DeleteItem - [x] Specify index endpoint: ReadIndex - [x] Specify multi-item endpoints: InsertBatch, ReadBatch, DeleteBatch - [x] Move to JSON objects instead of tuples - [x] Specify endpoints for polling for updates on single values (PollItem) **Implementation:** - [x] Table for K2V items, causal contexts - [x] Indexing mechanism and table for K2V index - [x] Make API handlers a bit more generic - [x] K2V API endpoint - [x] K2V API router - [x] ReadItem - [x] InsertItem - [x] DeleteItem - [x] PollItem - [x] ReadIndex - [x] InsertBatch - [x] ReadBatch - [x] DeleteBatch **Testing:** - [x] Just a simple Python script that does some requests to check visually that things are going right (does not contain parsing of results or assertions on returned values) - [x] Actual tests: - [x] Adapt testing framework - [x] Simple test with InsertItem + ReadItem - [x] Test with several Insert/Read/DeleteItem + ReadIndex - [x] Test all combinations of return formats for ReadItem - [x] Test with ReadBatch, InsertBatch, DeleteBatch - [x] Test with PollItem - [x] Test error codes - [ ] Fix most broken stuff - [x] test PollItem broken randomly - [x] when invalid causality tokens are given, errors should be 4xx not 5xx **Improvements:** - [x] Descending range queries - [x] Specify - [x] Implement - [x] Add test - [x] Batch updates to index counter - [x] Put K2V behind `k2v` feature flag Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/293 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
2022-05-10 11:16:57 +00:00
.endpoint("garage_block/manager.rs/Rpc".to_string());
2020-04-18 17:21:34 +00:00
let buffer_kb_semaphore = Arc::new(Semaphore::new(config.block_ram_buffer_max / 1024));
let metrics = BlockManagerMetrics::new(
config.compression_level,
rc.rc_table.clone(),
resync.queue.clone(),
resync.errors.clone(),
buffer_kb_semaphore.clone(),
);
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,
data_layout: ArcSwap::new(Arc::new(data_layout)),
data_layout_persister,
data_fsync: config.data_fsync,
disable_scrub: config.disable_scrub,
compression_level: config.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,
resync,
system,
2021-10-14 09:50:12 +00:00
endpoint,
buffer_kb_semaphore,
2022-02-16 13:23:04 +00:00
metrics,
scrub_persister,
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());
block_manager.scrub_persister.set_with(|_| ()).unwrap();
2023-09-05 13:39:21 +00:00
Ok(block_manager)
}
pub fn spawn_workers(self: &Arc<Self>, bg: &BackgroundRunner) {
// Spawn a bunch of resync workers
for index in 0..MAX_RESYNC_WORKERS {
let worker = ResyncWorker::new(index, self.clone());
bg.spawn_worker(worker);
}
// Spawn scrub worker
if !self.disable_scrub {
let (scrub_tx, scrub_rx) = mpsc::channel(1);
self.tx_scrub_command.store(Some(Arc::new(scrub_tx)));
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);
if !self.disable_scrub {
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))
});
vars.register_ro(&self.scrub_persister, "scrub-next-run", |p| {
p.get_with(|x| msec_to_rfc3339(x.time_next_run_scrub))
});
vars.register_ro(&self.scrub_persister, "scrub-corruptions_detected", |p| {
p.get_with(|x| x.corruptions_detected)
});
}
}
/// Initialization: set how block references are recalculated
/// for repair operations
pub fn set_recalc_rc(&self, recalc: Vec<CalculateRefcount>) {
self.rc.recalc_rc.store(Some(Arc::new(recalc)));
}
/// 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,
priority: RequestPriority,
order_tag: Option<OrderTag>,
) -> Result<DataBlockStream, Error> {
self.rpc_get_raw_block_internal(
hash,
priority,
order_tag,
|stream| async move { Ok(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
pub(crate) async fn rpc_get_raw_block(
&self,
hash: &Hash,
priority: RequestPriority,
order_tag: Option<OrderTag>,
) -> Result<DataBlock, Error> {
self.rpc_get_raw_block_internal(hash, priority, order_tag, |block_stream| async move {
let (header, stream) = block_stream.into_parts();
2023-09-05 13:39:21 +00:00
read_stream_to_end(stream)
.await
.err_context("error in block data stream")
.map(|data| DataBlock::from_parts(header, data.into_bytes()))
2023-09-05 13:39:21 +00:00
})
.await
}
async fn rpc_get_raw_block_internal<F, Fut, T>(
&self,
hash: &Hash,
priority: RequestPriority,
2023-09-05 13:39:21 +00:00
order_tag: Option<OrderTag>,
f: F,
) -> Result<T, Error>
where
F: Fn(DataBlockStream) -> Fut,
2023-09-05 13:39:21 +00:00
Fut: futures::Future<Output = Result<T, Error>>,
{
let who = self
.system
.rpc_helper()
.block_read_nodes_of(hash, self.system.rpc_helper());
2022-07-22 16:20:27 +00:00
for node in who.iter() {
let node_id = NodeID::from(*node);
let rpc = self.endpoint.call_streaming(
&node_id,
BlockRpc::GetBlock(*hash, order_tag),
priority,
);
2022-07-22 16:20:27 +00:00
tokio::select! {
res = rpc => {
let res = match res {
Ok(res) => res,
Err(e) => {
debug!("Get block {:?}: node {:?} could not be contacted: {}", hash, node, e);
2022-07-22 16:20:27 +00:00
continue;
}
};
let block_stream = match res.into_parts() {
(Ok(BlockRpc::PutBlock { hash: _, header }), Some(stream)) => DataBlockStream::from_parts(header, stream),
(Ok(_), _) => {
debug!("Get block {:?}: node {:?} returned a malformed response", hash, node);
2022-07-22 16:20:27 +00:00
continue;
}
(Err(e), _) => {
debug!("Get block {:?}: node {:?} returned error: {}", hash, node, e);
continue;
}
2022-07-22 16:20:27 +00:00
};
match f(block_stream).await {
2023-09-05 13:39:21 +00:00
Ok(ret) => return Ok(ret),
2022-07-22 16:20:27 +00:00
Err(e) => {
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
_ = tokio::time::sleep(self.system.rpc_helper().rpc_timeout()) => {
debug!("Get block {:?}: node {:?} didn't return block in time, trying next.", hash, node);
2022-07-22 16:20:27 +00:00
}
};
}
2022-07-22 16:20:27 +00:00
let err = Error::MissingBlock(*hash);
debug!("{}", err);
Err(err)
2020-04-11 21:00:26 +00:00
}
// ---- Public interface ----
/// Ask nodes that might have a block for it, return it as a stream
2022-07-22 16:20:27 +00:00
pub async fn rpc_get_block_streaming(
&self,
hash: &Hash,
order_tag: Option<OrderTag>,
) -> Result<ByteStream, Error> {
let block_stream = self
.rpc_get_raw_block_streaming(hash, PRIO_NORMAL | PRIO_SECONDARY, order_tag)
.await?;
let (header, stream) = block_stream.into_parts();
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)))
}
}
}
/// Send block to nodes that should have it
pub async fn rpc_put_block(
&self,
hash: Hash,
data: Bytes,
2024-02-23 15:49:50 +00:00
prevent_compression: bool,
order_tag: Option<OrderTag>,
) -> Result<(), Error> {
let who = self.replication.write_sets(&hash);
2022-07-22 16:20:27 +00:00
2024-02-23 15:49:50 +00:00
let compression_level = self.compression_level.filter(|_| !prevent_compression);
let (header, bytes) = DataBlock::from_buffer(data, compression_level)
2022-07-22 16:20:27 +00:00
.await
.into_parts();
let permit = self
.buffer_kb_semaphore
.clone()
.acquire_many_owned((bytes.len() / 1024).try_into().unwrap())
.await
.ok_or_message("could not reserve space for buffer of data to send to remote nodes")?;
2022-07-22 16:20:27 +00:00
let put_block_rpc =
Req::new(BlockRpc::PutBlock { hash, header })?.with_stream_from_buffer(bytes);
let put_block_rpc = if let Some(tag) = order_tag {
put_block_rpc.with_order_tag(tag)
} else {
put_block_rpc
};
2022-07-22 16:20:27 +00:00
self.system
.rpc_helper()
.try_write_many_sets(
&self.endpoint,
2023-11-15 14:40:44 +00:00
who.as_ref(),
2022-07-22 16:20:27 +00:00
put_block_rpc,
RequestStrategy::with_priority(PRIO_NORMAL | PRIO_SECONDARY)
.with_drop_on_completion(permit)
.with_quorum(self.replication.write_quorum()),
)
.await?;
2022-07-22 16:20:27 +00:00
Ok(())
}
2020-04-11 21:00:26 +00:00
/// Get number of items in the refcount table
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
2022-06-08 08:01:44 +00:00
pub fn rc_len(&self) -> Result<usize, Error> {
Ok(self.rc.rc_table.len()?)
}
/// Send command to start/stop/manager scrub worker
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(())
}
/// 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> {
2024-03-08 13:59:56 +00:00
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)
}
//// ----- Managing the reference counter ----
/// Increment the number of time a block is used, putting it to resynchronization if it is
/// required, but not known
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
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)? {
// 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.
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
2022-06-08 08:01:44 +00:00
let this = self.clone();
tokio::spawn(async move {
if let Err(e) = this
.resync
.put_to_resync(&hash, 2 * this.system.rpc_helper().rpc_timeout())
{
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
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
}
Ok(())
}
2020-04-09 21:45:07 +00:00
/// Decrement the number of time a block is used
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
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.
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
2022-06-08 08:01:44 +00:00
let this = self.clone();
tokio::spawn(async move {
if let Err(e) = this
.resync
.put_to_resync(&hash, BLOCK_GC_DELAY + Duration::from_secs(10))
Abstract database behind generic interface and implement alternative drivers (#322) - [x] Design interface - [x] Implement Sled backend - [x] Re-implement the SledCountedTree hack ~~on Sled backend~~ on all backends (i.e. over the abstraction) - [x] Convert Garage code to use generic interface - [x] Proof-read converted Garage code - [ ] Test everything well - [x] Implement sqlite backend - [x] Implement LMDB backend - [ ] (Implement Persy backend?) - [ ] (Implement other backends? (like RocksDB, ...)) - [x] Implement backend choice in config file and garage server module - [x] Add CLI for converting between DB formats - Exploit the new interface to put more things in transactions - [x] `.updated()` trigger on Garage tables Fix #284 **Bugs** - [x] When exporting sqlite, trees iterate empty?? - [x] LMDB doesn't work **Known issues for various back-ends** - Sled: - Eats all my RAM and also all my disk space - `.len()` has to traverse the whole table - Is actually quite slow on some operations - And is actually pretty bad code... - Sqlite: - Requires a lock to be taken on all operations. The lock is also taken when iterating on a table with `.iter()`, and the lock isn't released until the iterator is dropped. This means that we must be VERY carefull to not do anything else inside a `.iter()` loop or else we will have a deadlock! Most such cases have been eliminated from the Garage codebase, but there might still be some that remain. If your Garage-over-Sqlite seems to hang/freeze, this is the reason. - (adapter uses a bunch of unsafe code) - Heed (LMDB): - Not suited for 32-bit machines as it has to map the whole DB in memory. - (adpater uses a tiny bit of unsafe code) **My recommendation:** avoid 32-bit machines and use LMDB as much as possible. **Converting databases** is actually quite easy. For example from Sled to LMDB: ```bash cd src/db cargo run --features cli --bin convert -- -i path/to/garage/meta/db -a sled -o path/to/garage/meta/db.lmdb -b lmdb ``` Then, just add this to your `config.toml`: ```toml db_engine = "lmdb" ``` Co-authored-by: Alex Auvolat <alex@adnab.me> Reviewed-on: https://git.deuxfleurs.fr/Deuxfleurs/garage/pulls/322 Co-authored-by: Alex <alex@adnab.me> Co-committed-by: Alex <alex@adnab.me>
2022-06-08 08:01:44 +00:00
{
error!("Block {:?} could not be put in resync queue: {}.", hash, e);
}
});
}
Ok(())
}
2020-04-11 21:00:26 +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?.into_bytes();
2022-07-22 16:20:27 +00:00
let data = DataBlock::from_parts(header, bytes);
self.write_block(&hash, &data).await
}
/// Write a block to disk
pub(crate) async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<(), Error> {
let tracer = opentelemetry::global::tracer("garage");
2022-09-12 14:57:38 +00:00
self.lock_mutate(hash)
.await
.write_block(hash, data, self)
2022-02-22 12:53:59 +00:00
.bound_record_duration(&self.metrics.block_write_duration)
.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
}
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
let resp = Resp::new(Ok(BlockRpc::PutBlock {
2022-07-22 16:20:27 +00:00
hash: *hash,
header,
}))
.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> {
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_helper().rpc_timeout())?;
return Err(Error::Message(format!(
"block {:?} not found on node",
hash
)));
}
2020-04-17 16:38:11 +00:00
}
}
.bound_record_duration(&self.metrics.block_read_duration)
.with_context(Context::current_with_span(
tracer.start("BlockManager::read_block"),
))
.await
}
pub(crate) async fn read_block_from(
&self,
hash: &Hash,
block_path: &DataBlockPath,
) -> Result<DataBlock, Error> {
let (header, path) = block_path.as_parts_ref();
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?;
self.metrics.bytes_read.add(data.len() as u64);
drop(f);
let data = DataBlock::from_parts(header, data.into());
if data.verify(*hash).is_err() {
2022-02-16 13:23:04 +00:00
self.metrics.corruption_counter.add(1);
warn!(
"Block {:?} is corrupted. Renaming to .corrupted and resyncing.",
hash
);
2022-09-12 14:57:38 +00:00
self.lock_mutate(hash)
.await
.move_block_to_corrupted(block_path)
.await?;
self.resync.put_to_resync(hash, Duration::from_millis(0))?;
return Err(Error::CorruptData(*hash));
}
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
/// Check if this node should have a block, but don't actually have it
async fn need_block(&self, hash: &Hash) -> Result<bool, Error> {
let rc = self.rc.get_block_rc(hash)?;
let exists = self.find_block(hash).await.is_some();
Ok(rc.is_nonzero() && !exists)
}
/// 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)
.await
.delete_if_unneeded(hash, self)
.await
}
/// Find the path where a block is currently stored
pub(crate) async fn find_block(&self, hash: &Hash) -> Option<DataBlockPath> {
let data_layout = self.data_layout.load_full();
let dirs = Some(data_layout.primary_block_dir(hash))
.into_iter()
.chain(data_layout.secondary_block_dirs(hash));
let filename = hex::encode(hash.as_ref());
2020-04-09 21:45:07 +00:00
for dir in dirs {
let mut path = dir;
path.push(&filename);
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).
if fs::metadata(&path).await.is_ok() {
return Some(DataBlockPath::plain(path));
}
path.set_extension("zst");
if fs::metadata(&path).await.is_ok() {
return Some(DataBlockPath::compressed(path));
}
} else {
path.set_extension("zst");
if fs::metadata(&path).await.is_ok() {
return Some(DataBlockPath::compressed(path));
}
path.set_extension("");
if fs::metadata(&path).await.is_ok() {
return Some(DataBlockPath::plain(path));
}
}
}
None
}
2022-09-12 14:57:38 +00:00
/// Rewrite a block at the primary location for its path and delete the old path.
/// Returns the number of bytes read/written
pub(crate) async fn fix_block_location(
&self,
hash: &Hash,
wrong_path: DataBlockPath,
) -> Result<usize, Error> {
self.lock_mutate(hash)
.await
.fix_block_location(hash, wrong_path, self)
.await
}
2022-09-12 14:57:38 +00:00
async fn lock_mutate(&self, hash: &Hash) -> MutexGuard<'_, BlockManagerLocked> {
let tracer = opentelemetry::global::tracer("garage");
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(
tracer.start(format!("Acquire mutation_lock #{}", ilock)),
2022-09-12 14:57:38 +00:00
))
.await
}
}
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
.map(|()| BlockRpc::Ok),
2022-07-22 16:20:27 +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-14 09:50:12 +00:00
}
}
impl BlockManagerLocked {
async fn write_block(
&self,
hash: &Hash,
data: &DataBlock,
mgr: &BlockManager,
) -> Result<(), Error> {
let existing_path = mgr.find_block(hash).await;
self.write_block_inner(hash, data, mgr, existing_path).await
}
async fn write_block_inner(
&self,
hash: &Hash,
data: &DataBlock,
mgr: &BlockManager,
existing_path: Option<DataBlockPath>,
2022-07-22 16:20:27 +00:00
) -> Result<(), Error> {
2024-02-23 11:26:35 +00:00
let (header, data) = data.as_parts_ref();
let compressed = header.is_compressed();
let directory = mgr.data_layout.load().primary_block_dir(hash);
let mut tgt_path = directory.clone();
tgt_path.push(hex::encode(hash));
if compressed {
tgt_path.set_extension("zst");
}
let existing_info = existing_path.map(|x| x.into_parts());
let to_delete = match (existing_info, compressed) {
// If the block is stored in the wrong directory,
// write it again at the correct path and delete the old path
(Some((DataBlockHeader::Plain, p)), false) if p != tgt_path => Some(p),
(Some((DataBlockHeader::Compressed, p)), true) if p != tgt_path => Some(p),
// If the block is already stored not compressed but we have a compressed
// copy, write the compressed copy and delete the uncompressed one
(Some((DataBlockHeader::Plain, plain_path)), true) => Some(plain_path),
// If the block is already stored compressed,
// keep the stored copy, we have nothing to do
(Some((DataBlockHeader::Compressed, _)), _) => return Ok(()),
// 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
(Some((DataBlockHeader::Plain, _)), false) => return Ok(()),
// If the block isn't stored already, just store what is given to us
(None, _) => None,
};
assert!(to_delete.as_ref() != Some(&tgt_path));
let mut path_tmp = tgt_path.clone();
let tmp_extension = format!("tmp{}", hex::encode(thread_rng().gen::<[u8; 4]>()));
path_tmp.set_extension(tmp_extension);
fs::create_dir_all(&directory).await?;
let mut delete_on_drop = DeleteOnDrop(Some(path_tmp.clone()));
let mut f = fs::File::create(&path_tmp).await?;
f.write_all(data).await?;
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?;
}
drop(f);
fs::rename(path_tmp, tgt_path).await?;
delete_on_drop.cancel();
if let Some(to_delete) = to_delete {
fs::remove_file(to_delete).await?;
}
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-07-22 16:20:27 +00:00
Ok(())
}
async fn move_block_to_corrupted(&self, block_path: &DataBlockPath) -> Result<(), Error> {
let (header, path) = block_path.as_parts_ref();
let mut path2 = path.clone();
if header.is_compressed() {
path2.set_extension("zst.corrupted");
} else {
path2.set_extension("corrupted");
}
fs::rename(path, path2).await?;
Ok(())
}
async fn delete_if_unneeded(&self, hash: &Hash, mgr: &BlockManager) -> Result<(), Error> {
let rc = mgr.rc.get_block_rc(hash)?;
if rc.is_deletable() {
while let Some(path) = mgr.find_block(hash).await {
let (_header, path) = path.as_parts_ref();
fs::remove_file(path).await?;
mgr.metrics.delete_counter.add(1);
}
}
Ok(())
}
async fn fix_block_location(
&self,
hash: &Hash,
wrong_path: DataBlockPath,
mgr: &BlockManager,
) -> Result<usize, Error> {
let data = mgr.read_block_from(hash, &wrong_path).await?;
self.write_block_inner(hash, &data, mgr, Some(wrong_path))
.await?;
2024-02-23 11:26:35 +00:00
Ok(data.as_parts_ref().1.len())
}
}
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);
}
});
}
}
}