garage/src/model/block.rs

662 lines
18 KiB
Rust
Raw Normal View History

2020-04-09 21:45:07 +00:00
use std::path::PathBuf;
use std::sync::Arc;
use std::time::Duration;
2020-04-09 21:45:07 +00:00
use arc_swap::ArcSwapOption;
use futures::future::*;
use futures::select;
2020-04-18 17:21:34 +00:00
use serde::{Deserialize, Serialize};
2020-04-09 21:45:07 +00:00
use tokio::fs;
2021-03-15 21:36:41 +00:00
use tokio::io::{AsyncReadExt, AsyncWriteExt};
use tokio::sync::{watch, Mutex, Notify};
2021-04-07 00:13:26 +00:00
use zstd::stream::{decode_all as zstd_decode, Encoder};
2020-04-09 21:45:07 +00:00
2020-04-24 10:10:01 +00:00
use garage_util::data::*;
use garage_util::error::Error;
2021-03-15 15:21:41 +00:00
use garage_util::time::*;
2020-04-24 10:10:01 +00:00
use garage_rpc::membership::System;
use garage_rpc::rpc_client::*;
use garage_rpc::rpc_server::*;
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
2020-04-24 10:10:01 +00:00
use crate::block_ref_table::*;
2020-04-23 17:05:46 +00:00
2020-04-24 10:10:01 +00:00
use crate::garage::Garage;
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;
2021-03-15 18:51:16 +00:00
pub const BACKGROUND_WORKERS: u64 = 1;
const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(42);
2021-03-15 14:26:29 +00:00
const BLOCK_GC_TIMEOUT: Duration = Duration::from_secs(60);
const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5);
const RESYNC_RETRY_TIMEOUT: Duration = Duration::from_secs(10);
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)]
pub enum Message {
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-04-07 00:13:26 +00:00
PutBlock {
hash: Hash,
data: BlockData,
},
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-04-06 15:51:19 +00:00
/// A possibly compressed block of data
#[derive(Debug, Serialize, Deserialize)]
2021-04-06 15:51:19 +00:00
pub enum BlockData {
Plain(#[serde(with = "serde_bytes")] Vec<u8>),
Compressed(#[serde(with = "serde_bytes")] Vec<u8>),
}
impl BlockData {
pub fn is_compressed(&self) -> bool {
match self {
BlockData::Plain(_) => false,
BlockData::Compressed(_) => true,
}
}
2021-04-07 00:13:26 +00:00
pub fn buffer(&self) -> &Vec<u8> {
match self {
BlockData::Plain(b) => b,
BlockData::Compressed(b) => b,
}
}
}
2020-04-18 17:21:34 +00:00
impl RpcMessage for Message {}
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,
2021-03-26 20:53:28 +00:00
/// Lock to prevent concurrent edition of the directory
pub data_dir_lock: Mutex<()>,
2021-03-15 18:51:16 +00:00
rc: sled::Tree,
2021-03-15 18:51:16 +00:00
resync_queue: sled::Tree,
resync_notify: Notify,
2021-03-15 18:51:16 +00:00
system: Arc<System>,
2020-04-18 17:21:34 +00:00
rpc_client: Arc<RpcClient<Message>>,
2021-03-15 18:51:16 +00:00
pub(crate) garage: ArcSwapOption<Garage>,
2020-04-09 21:45:07 +00:00
}
2020-04-11 21:00:26 +00:00
impl BlockManager {
2020-04-18 17:21:34 +00:00
pub fn new(
db: &sled::Db,
data_dir: PathBuf,
2020-04-23 17:05:46 +00:00
replication: TableShardedReplication,
2020-04-18 17:21:34 +00:00
system: Arc<System>,
rpc_server: &mut RpcServer,
) -> Arc<Self> {
let rc = db
.open_tree("block_local_rc")
2020-04-11 21:00:26 +00:00
.expect("Unable to open block_local_rc tree");
let resync_queue = db
.open_tree("block_local_resync_queue")
.expect("Unable to open block_local_resync_queue tree");
2020-04-18 17:21:34 +00:00
let rpc_path = "block_manager";
let rpc_client = system.rpc_client::<Message>(rpc_path);
let block_manager = Arc::new(Self {
2020-04-23 17:05:46 +00:00
replication,
data_dir,
data_dir_lock: Mutex::new(()),
2020-04-11 21:00:26 +00:00
rc,
resync_queue,
resync_notify: Notify::new(),
system,
2020-04-18 17:21:34 +00:00
rpc_client,
garage: ArcSwapOption::from(None),
2020-04-18 17:21:34 +00:00
});
block_manager
.clone()
.register_handler(rpc_server, rpc_path.into());
block_manager
}
fn register_handler(self: Arc<Self>, rpc_server: &mut RpcServer, path: String) {
let self2 = self.clone();
2020-04-18 17:21:34 +00:00
rpc_server.add_handler::<Message, _, _>(path, move |msg, _addr| {
let self2 = self2.clone();
async move { self2.handle(&msg).await }
2020-04-18 17:21:34 +00:00
});
let self2 = self.clone();
self.rpc_client
.set_local_handler(self.system.id, move |msg| {
let self2 = self2.clone();
async move { self2.handle(&msg).await }
});
}
async fn handle(self: Arc<Self>, msg: &Message) -> Result<Message, Error> {
match msg {
2021-04-06 15:51:19 +00:00
Message::PutBlock { hash, data } => self.write_block(&hash, &data).await,
Message::GetBlock(h) => self.read_block(h).await,
Message::NeedBlockQuery(h) => self.need_block(h).await.map(Message::NeedBlockReply),
2020-11-08 14:04:30 +00:00
_ => Err(Error::BadRPC(format!("Unexpected RPC message"))),
}
}
pub fn spawn_background_worker(self: Arc<Self>) {
2021-03-26 20:53:28 +00:00
// Launch 2 simultaneous workers for background resync loop preprocessing <= TODO actually this
// launches only one worker with current value of BACKGROUND_WORKERS
2021-03-15 18:51:16 +00:00
for i in 0..BACKGROUND_WORKERS {
let bm2 = self.clone();
let background = self.system.background.clone();
tokio::spawn(async move {
2021-03-15 21:36:41 +00:00
tokio::time::sleep(Duration::from_secs(10 * (i + 1))).await;
background.spawn_worker(format!("block resync worker {}", i), move |must_exit| {
bm2.resync_loop(must_exit)
});
});
}
2020-04-11 21:00:26 +00:00
}
2021-03-26 20:53:28 +00:00
/// Write a block to disk
2021-04-06 15:51:19 +00:00
pub async fn write_block(&self, hash: &Hash, data: &BlockData) -> Result<Message, Error> {
2021-04-07 00:13:26 +00:00
let mut path = self.block_dir(hash);
let _lock = self.data_dir_lock.lock().await;
let clean_plain = match self.is_block_compressed(hash).await {
Ok(true) => return Ok(Message::Ok),
Ok(false) if !data.is_compressed() => return Ok(Message::Ok), // we have a plain block, and the provided block is not compressed either
Ok(false) => true,
Err(_) => false,
};
2020-04-11 21:00:26 +00:00
fs::create_dir_all(&path).await?;
path.push(hex::encode(hash));
2021-04-07 00:13:26 +00:00
if data.is_compressed() {
path.set_extension("zst");
}
2021-04-07 00:13:26 +00:00
let buffer = data.buffer();
let mut f = fs::File::create(path.clone()).await?;
2021-04-06 15:51:19 +00:00
f.write_all(&buffer).await?;
2021-04-14 21:37:41 +00:00
drop(f);
if clean_plain {
path.set_extension("");
fs::remove_file(path).await?;
}
2020-04-11 21:00:26 +00:00
Ok(Message::Ok)
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-03-17 19:37:31 +00:00
pub async fn read_block(&self, hash: &Hash) -> Result<Message, Error> {
let mut path = self.block_path(hash);
2020-04-09 21:45:07 +00:00
2021-03-17 19:37:31 +00:00
let mut data = vec![];
let block = match self.is_block_compressed(hash).await {
Ok(false) => {
let f = fs::File::open(&path).await;
f.map(|f| (f, false)).map_err(Into::into)
}
Ok(true) => {
2021-04-07 00:13:26 +00:00
path.set_extension("zst");
2021-03-17 19:37:31 +00:00
let f = fs::File::open(&path).await;
f.map(|f| (f, true)).map_err(Into::into)
}
Err(e) => Err(e),
};
let (mut f, compressed) = match block {
Ok(ok) => ok,
e => {
2020-04-17 16:38:11 +00:00
// Not found but maybe we should have had it ??
2021-03-15 13:46:37 +00:00
self.put_to_resync(hash, Duration::from_millis(0))?;
2021-03-17 19:37:31 +00:00
e?
2020-04-17 16:38:11 +00:00
}
};
2020-04-11 21:00:26 +00:00
f.read_to_end(&mut data).await?;
drop(f);
let sum_ok = if compressed {
2021-04-07 00:13:26 +00:00
zstd_check_checksum(&data[..])
2021-03-17 19:37:31 +00:00
} else {
blake2sum(&data[..]) == *hash
2021-03-17 19:37:31 +00:00
};
if !sum_ok {
let _lock = self.data_dir_lock.lock().await;
2021-03-15 14:26:29 +00:00
warn!(
"Block {:?} is corrupted. Renaming to .corrupted and resyncing.",
hash
);
2021-03-15 13:46:37 +00:00
let mut path2 = path.clone();
2021-03-17 19:37:31 +00:00
path2.set_extension("corrupted");
2021-03-15 13:46:37 +00:00
fs::rename(path, path2).await?;
self.put_to_resync(&hash, Duration::from_millis(0))?;
return Err(Error::CorruptData(*hash));
}
2020-04-11 21:00:26 +00:00
2021-03-17 19:37:31 +00:00
if compressed {
2021-04-06 15:51:19 +00:00
Ok(Message::PutBlock {
2021-03-17 19:37:31 +00:00
hash: *hash,
2021-04-06 15:51:19 +00:00
data: BlockData::Compressed(data),
})
2021-03-17 19:37:31 +00:00
} else {
2021-04-06 15:51:19 +00:00
Ok(Message::PutBlock {
hash: *hash,
data: BlockData::Plain(data),
})
2021-03-17 19:37:31 +00:00
}
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> {
let needed = self
.rc
.get(hash.as_ref())?
2021-03-15 17:27:26 +00:00
.map(|x| u64_from_be_bytes(x) > 0)
.unwrap_or(false);
if needed {
2021-03-17 19:37:31 +00:00
let exists = self.is_block_compressed(hash).await.is_ok();
Ok(!exists)
} else {
Ok(false)
}
}
2021-03-17 19:37:31 +00:00
async fn is_block_compressed(&self, hash: &Hash) -> Result<bool, Error> {
let mut path = self.block_path(hash);
2021-04-07 00:13:26 +00:00
path.set_extension("zst");
2021-03-17 19:37:31 +00:00
if fs::metadata(&path).await.is_ok() {
return Ok(true);
2021-03-17 19:37:31 +00:00
}
path.set_extension("");
fs::metadata(&path).await.map(|_| false).map_err(Into::into)
2021-03-17 19:37:31 +00:00
}
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
}
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-03-26 20:53:28 +00:00
/// Increment the number of time a block is used, putting it to resynchronization if it is
/// required, but not known
2020-04-11 21:00:26 +00:00
pub fn block_incref(&self, hash: &Hash) -> Result<(), Error> {
2021-03-15 17:27:26 +00:00
let old_rc = self.rc.fetch_and_update(&hash, |old| {
let old_v = old.map(u64_from_be_bytes).unwrap_or(0);
Some(u64::to_be_bytes(old_v + 1).to_vec())
})?;
let old_rc = old_rc.map(u64_from_be_bytes).unwrap_or(0);
if old_rc == 0 {
2021-03-15 13:46:37 +00:00
self.put_to_resync(&hash, BLOCK_RW_TIMEOUT)?;
}
2020-04-11 21:00:26 +00:00
Ok(())
}
2020-04-09 21:45:07 +00:00
2021-03-26 20:53:28 +00:00
/// Decrement the number of time a block is used
pub fn block_decref(&self, hash: &Hash) -> Result<(), Error> {
2021-03-15 17:27:26 +00:00
let new_rc = self.rc.update_and_fetch(&hash, |old| {
let old_v = old.map(u64_from_be_bytes).unwrap_or(0);
if old_v > 1 {
Some(u64::to_be_bytes(old_v - 1).to_vec())
} else {
None
}
})?;
if new_rc.is_none() {
2021-03-15 14:26:29 +00:00
self.put_to_resync(&hash, BLOCK_GC_TIMEOUT)?;
}
Ok(())
}
2021-03-15 13:46:37 +00:00
fn put_to_resync(&self, hash: &Hash, delay: Duration) -> Result<(), Error> {
let when = now_msec() + delay.as_millis() as u64;
2020-04-21 12:54:55 +00:00
trace!("Put resync_queue: {} {:?}", when, hash);
let mut key = u64::to_be_bytes(when).to_vec();
key.extend(hash.as_ref());
self.resync_queue.insert(key, hash.as_ref())?;
2021-03-15 21:36:41 +00:00
self.resync_notify.notify_waiters();
Ok(())
}
async fn resync_loop(self: Arc<Self>, mut must_exit: watch::Receiver<bool>) {
while !*must_exit.borrow() {
if let Err(e) = self.resync_iter(&mut must_exit).await {
warn!("Error in block resync loop: {}", e);
select! {
2021-03-16 15:51:15 +00:00
_ = tokio::time::sleep(Duration::from_secs(1)).fuse() => (),
_ = must_exit.changed().fuse() => (),
}
}
}
}
async fn resync_iter(&self, must_exit: &mut watch::Receiver<bool>) -> Result<(), Error> {
if let Some(first_item) = self.resync_queue.iter().next() {
let (time_bytes, hash_bytes) = first_item?;
let time_msec = u64_from_be_bytes(&time_bytes[0..8]);
let now = now_msec();
if now >= time_msec {
let hash = Hash::try_from(&hash_bytes[..]).unwrap();
let res = self.resync_block(&hash).await;
if let Err(e) = &res {
warn!("Error when resyncing {:?}: {}", hash, e);
self.put_to_resync(&hash, RESYNC_RETRY_TIMEOUT)?;
}
self.resync_queue.remove(&time_bytes)?;
res?; // propagate error to delay main loop
} else {
2021-03-15 21:36:41 +00:00
let delay = tokio::time::sleep(Duration::from_millis(time_msec - now));
select! {
_ = delay.fuse() => (),
_ = self.resync_notify.notified().fuse() => (),
2021-03-15 21:36:41 +00:00
_ = must_exit.changed().fuse() => (),
}
2020-04-11 21:00:26 +00:00
}
} else {
select! {
_ = self.resync_notify.notified().fuse() => (),
2021-03-15 21:36:41 +00:00
_ = must_exit.changed().fuse() => (),
}
2020-04-11 21:00:26 +00:00
}
Ok(())
2020-04-11 21:00:26 +00:00
}
async fn resync_block(&self, hash: &Hash) -> Result<(), Error> {
2021-03-15 13:46:37 +00:00
let lock = self.data_dir_lock.lock().await;
2020-04-17 17:20:17 +00:00
let path = self.block_path(hash);
2021-03-17 19:37:31 +00:00
let exists = self.is_block_compressed(hash).await.is_ok();
let needed = self
.rc
.get(hash.as_ref())?
2021-03-15 17:27:26 +00:00
.map(|x| u64_from_be_bytes(x) > 0)
.unwrap_or(false);
2020-04-21 12:54:55 +00:00
if exists != needed {
info!(
"Resync block {:?}: exists {}, needed {}",
hash, exists, needed
);
}
if exists && !needed {
2021-02-24 10:58:03 +00:00
trace!("Offloading block {:?}", hash);
2021-03-16 10:14:27 +00:00
let mut who = self.replication.write_nodes(&hash);
if who.len() < self.replication.write_quorum() {
return Err(Error::Message(format!("Not trying to offload block because we don't have a quorum of nodes to write to")));
}
2021-02-24 10:58:03 +00:00
who.retain(|id| *id != self.system.id);
let msg = Arc::new(Message::NeedBlockQuery(*hash));
let who_needs_fut = who.iter().map(|to| {
self.rpc_client
.call_arc(*to, msg.clone(), NEED_BLOCK_QUERY_TIMEOUT)
});
let who_needs_resps = join_all(who_needs_fut).await;
let mut need_nodes = vec![];
for (node, needed) in who.iter().zip(who_needs_resps.into_iter()) {
match needed? {
Message::NeedBlockReply(needed) => {
if needed {
need_nodes.push(*node);
}
}
2021-02-24 10:58:03 +00:00
_ => {
return Err(Error::Message(format!(
"Unexpected response to NeedBlockQuery RPC"
)));
}
}
2021-02-24 10:58:03 +00:00
}
if need_nodes.len() > 0 {
trace!(
"Block {:?} needed by {} nodes, sending",
hash,
need_nodes.len()
);
let put_block_message = self.read_block(hash).await?;
2021-03-12 20:52:19 +00:00
self.rpc_client
.try_call_many(
&need_nodes[..],
put_block_message,
2021-03-12 20:52:19 +00:00
RequestStrategy::with_quorum(need_nodes.len())
.with_timeout(BLOCK_RW_TIMEOUT),
)
.await?;
}
2021-03-16 10:14:27 +00:00
info!(
"Deleting block {:?}, offload finished ({} / {})",
hash,
need_nodes.len(),
who.len()
);
2021-02-24 10:58:03 +00:00
fs::remove_file(path).await?;
}
if needed && !exists {
2021-03-15 13:46:37 +00:00
drop(lock);
// TODO find a way to not do this if they are sending it to us
// Let's suppose this isn't an issue for now with the BLOCK_RW_TIMEOUT delay
// between the RC being incremented and this part being called.
2021-04-06 15:51:19 +00:00
let block = self.rpc_get_raw_block(&hash).await?;
self.write_block(hash, &block).await?;
}
Ok(())
}
2020-04-18 17:21:34 +00:00
2021-04-06 15:51:19 +00:00
async fn rpc_get_raw_block(&self, hash: &Hash) -> Result<BlockData, Error> {
2021-03-16 10:14:27 +00:00
let who = self.replication.read_nodes(&hash);
let resps = self
.rpc_client
.try_call_many(
&who[..],
Message::GetBlock(*hash),
RequestStrategy::with_quorum(1)
.with_timeout(BLOCK_RW_TIMEOUT)
.interrupt_after_quorum(true),
)
.await?;
for resp in resps {
2021-03-17 19:37:31 +00:00
match resp {
2021-04-06 15:51:19 +00:00
Message::PutBlock { data, .. } => return Ok(data),
2021-03-17 19:37:31 +00:00
_ => {}
2020-04-18 17:21:34 +00:00
}
}
Err(Error::Message(format!(
"Unable to read block {:?}: no valid blocks returned",
hash
)))
}
2021-03-17 19:37:31 +00:00
/// 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
2021-04-06 15:51:19 +00:00
.and_then(|data| match data {
BlockData::Plain(data) => Ok(data),
BlockData::Compressed(data) => {
2021-03-17 19:37:31 +00:00
zstd_decode(&data[..]).map_err(|_| Error::CorruptData(*hash))
}
})
}
2021-03-26 20:53:28 +00:00
/// Send block to nodes that should have it
2020-04-18 17:21:34 +00:00
pub async fn rpc_put_block(&self, hash: Hash, data: Vec<u8>) -> Result<(), Error> {
2021-03-17 19:37:31 +00:00
let garage = self.garage.load_full().unwrap();
2021-04-14 21:37:41 +00:00
let compressed = if garage.config.enable_compression {
zstd_encode(&data[..], garage.config.compression_level).ok()
} else {
None
};
// If compressed data is not less than 7/8 of the size of the original data, i.e. if we
// don't gain a significant margin by compressing, then we store the plain data instead
// so that we don't lose time decompressing it on reads.
2021-04-27 14:58:44 +00:00
let block_data =
if compressed.is_some() && compressed.as_ref().unwrap().len() < (data.len() * 7) / 8 {
BlockData::Compressed(compressed.unwrap())
} else {
BlockData::Plain(data)
};
2021-04-14 21:37:41 +00:00
let message = Message::PutBlock {
2021-04-27 14:58:44 +00:00
hash,
data: block_data,
2021-03-17 19:37:31 +00:00
};
2021-03-16 10:14:27 +00:00
let who = self.replication.write_nodes(&hash);
2020-04-18 17:21:34 +00:00
self.rpc_client
.try_call_many(
&who[..],
2021-03-17 19:37:31 +00:00
message,
2021-03-16 10:14:27 +00:00
RequestStrategy::with_quorum(self.replication.write_quorum())
.with_timeout(BLOCK_RW_TIMEOUT),
2020-04-18 17:21:34 +00:00
)
.await?;
Ok(())
}
2020-04-19 20:36:36 +00:00
pub async fn repair_data_store(&self, must_exit: &watch::Receiver<bool>) -> Result<(), Error> {
2020-04-19 20:36:36 +00:00
// 1. Repair blocks from RC table
let garage = self.garage.load_full().unwrap();
let mut last_hash = None;
let mut i = 0usize;
2021-03-11 15:54:15 +00:00
for entry in garage.block_ref_table.data.store.iter() {
2020-04-19 20:36:36 +00:00
let (_k, v_bytes) = entry?;
let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(v_bytes.as_ref())?;
if Some(&block_ref.block) == last_hash.as_ref() {
continue;
}
if !block_ref.deleted.get() {
last_hash = Some(block_ref.block);
2021-03-15 13:46:37 +00:00
self.put_to_resync(&block_ref.block, Duration::from_secs(0))?;
2020-04-19 20:36:36 +00:00
}
i += 1;
if i & 0xFF == 0 && *must_exit.borrow() {
return Ok(());
}
}
// 2. Repair blocks actually on disk
self.repair_aux_read_dir_rec(&self.data_dir, must_exit)
.await?;
2020-04-19 20:36:36 +00:00
2021-02-24 10:58:03 +00:00
Ok(())
}
fn repair_aux_read_dir_rec<'a>(
&'a self,
path: &'a PathBuf,
must_exit: &'a watch::Receiver<bool>,
) -> BoxFuture<'a, Result<(), Error>> {
2021-02-25 09:53:33 +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.
2021-02-24 10:58:03 +00:00
async move {
let mut ls_data_dir = fs::read_dir(path).await?;
2021-03-15 21:36:41 +00:00
loop {
let data_dir_ent = ls_data_dir.next_entry().await?;
let data_dir_ent = match data_dir_ent {
Some(x) => x,
None => break,
};
2021-02-24 10:58:03 +00:00
let name = data_dir_ent.file_name();
let name = match name.into_string() {
2020-04-19 20:36:36 +00:00
Ok(x) => x,
Err(_) => continue,
};
2021-02-24 10:58:03 +00:00
let ent_type = data_dir_ent.file_type().await?;
let mut iter_name = name.split('.');
let name = iter_name
.next()
.expect("split always contain at least one item");
let corrupted = iter_name
.last()
.map(|ext| ext == "corrupted")
.unwrap_or(false);
2021-02-24 10:58:03 +00:00
if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() {
self.repair_aux_read_dir_rec(&data_dir_ent.path(), must_exit)
.await?;
} else if name.len() == 64 && !corrupted {
2021-02-24 10:58:03 +00:00
let hash_bytes = match hex::decode(&name) {
Ok(h) => h,
Err(_) => continue,
};
let mut hash = [0u8; 32];
hash.copy_from_slice(&hash_bytes[..]);
2021-03-15 14:26:29 +00:00
self.put_to_resync(&hash.into(), Duration::from_secs(0))?;
2020-04-19 20:36:36 +00:00
}
if *must_exit.borrow() {
2021-02-24 10:58:03 +00:00
break;
2020-04-19 20:36:36 +00:00
}
}
2021-02-24 10:58:03 +00:00
Ok(())
}
.boxed()
2020-04-19 20:36:36 +00:00
}
2021-03-15 18:51:16 +00:00
2021-03-26 20:53:28 +00:00
/// Get lenght of resync queue
2021-03-15 18:51:16 +00:00
pub fn resync_queue_len(&self) -> usize {
self.resync_queue.len()
}
pub fn rc_len(&self) -> usize {
self.rc.len()
}
}
2021-03-15 17:27:26 +00:00
fn u64_from_be_bytes<T: AsRef<[u8]>>(bytes: T) -> u64 {
assert!(bytes.as_ref().len() == 8);
let mut x8 = [0u8; 8];
2021-03-15 17:27:26 +00:00
x8.copy_from_slice(bytes.as_ref());
u64::from_be_bytes(x8)
2020-04-11 21:00:26 +00:00
}
2021-04-07 00:13:26 +00:00
fn zstd_check_checksum<R: std::io::Read>(source: R) -> bool {
zstd::stream::copy_decode(source, std::io::sink()).is_ok()
}
fn zstd_encode<R: std::io::Read>(mut source: R, level: i32) -> std::io::Result<Vec<u8>> {
let mut result = Vec::<u8>::new();
let mut encoder = Encoder::new(&mut result, level)?;
encoder.include_checksum(true)?;
std::io::copy(&mut source, &mut encoder)?;
encoder.finish()?;
Ok(result)
}