aboutsummaryrefslogtreecommitdiff
path: root/src/model/block.rs
diff options
context:
space:
mode:
Diffstat (limited to 'src/model/block.rs')
-rw-r--r--src/model/block.rs241
1 files changed, 124 insertions, 117 deletions
diff --git a/src/model/block.rs b/src/model/block.rs
index 56c85c6a..0d9af38f 100644
--- a/src/model/block.rs
+++ b/src/model/block.rs
@@ -5,22 +5,20 @@ use std::time::Duration;
use arc_swap::ArcSwapOption;
use futures::future::*;
use futures::select;
-use futures::stream::*;
use serde::{Deserialize, Serialize};
use tokio::fs;
-use tokio::prelude::*;
+use tokio::io::{AsyncReadExt, AsyncWriteExt};
use tokio::sync::{watch, Mutex, Notify};
-use garage_util::data;
use garage_util::data::*;
use garage_util::error::Error;
+use garage_util::time::*;
use garage_rpc::membership::System;
use garage_rpc::rpc_client::*;
use garage_rpc::rpc_server::*;
-use garage_table::table_sharded::TableShardedReplication;
-use garage_table::TableReplication;
+use garage_table::replication::{sharded::TableShardedReplication, TableReplication};
use crate::block_ref_table::*;
@@ -28,7 +26,10 @@ use crate::garage::Garage;
pub const INLINE_THRESHOLD: usize = 3072;
+pub const BACKGROUND_WORKERS: u64 = 1;
+
const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(42);
+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);
@@ -56,14 +57,14 @@ pub struct BlockManager {
pub data_dir: PathBuf,
pub data_dir_lock: Mutex<()>,
- pub rc: sled::Tree,
+ rc: sled::Tree,
- pub resync_queue: sled::Tree,
- pub resync_notify: Notify,
+ resync_queue: sled::Tree,
+ resync_notify: Notify,
- pub system: Arc<System>,
+ system: Arc<System>,
rpc_client: Arc<RpcClient<Message>>,
- pub garage: ArcSwapOption<Garage>,
+ pub(crate) garage: ArcSwapOption<Garage>,
}
impl BlockManager {
@@ -77,7 +78,6 @@ impl BlockManager {
let rc = db
.open_tree("block_local_rc")
.expect("Unable to open block_local_rc tree");
- rc.set_merge_operator(rc_merge);
let resync_queue = db
.open_tree("block_local_resync_queue")
@@ -127,18 +127,16 @@ impl BlockManager {
}
}
- pub async fn spawn_background_worker(self: Arc<Self>) {
+ pub fn spawn_background_worker(self: Arc<Self>) {
// Launch 2 simultaneous workers for background resync loop preprocessing
- for i in 0..2usize {
+ for i in 0..BACKGROUND_WORKERS {
let bm2 = self.clone();
let background = self.system.background.clone();
tokio::spawn(async move {
- tokio::time::delay_for(Duration::from_secs(10)).await;
- background
- .spawn_worker(format!("block resync worker {}", i), move |must_exit| {
- bm2.resync_loop(must_exit)
- })
- .await;
+ 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)
+ });
});
}
}
@@ -168,7 +166,7 @@ impl BlockManager {
Ok(f) => f,
Err(e) => {
// Not found but maybe we should have had it ??
- self.put_to_resync(hash, 0)?;
+ self.put_to_resync(hash, Duration::from_millis(0))?;
return Err(Into::into(e));
}
};
@@ -176,11 +174,16 @@ impl BlockManager {
f.read_to_end(&mut data).await?;
drop(f);
- if data::sha256sum(&data[..]) != *hash {
+ if blake2sum(&data[..]) != *hash {
let _lock = self.data_dir_lock.lock().await;
- warn!("Block {:?} is corrupted. Deleting and resyncing.", hash);
- fs::remove_file(path).await?;
- self.put_to_resync(&hash, 0)?;
+ warn!(
+ "Block {:?} is corrupted. Renaming to .corrupted and resyncing.",
+ hash
+ );
+ let mut path2 = path.clone();
+ path2.set_extension(".corrupted");
+ fs::rename(path, path2).await?;
+ self.put_to_resync(&hash, Duration::from_millis(0))?;
return Err(Error::CorruptData(*hash));
}
@@ -191,7 +194,7 @@ impl BlockManager {
let needed = self
.rc
.get(hash.as_ref())?
- .map(|x| u64_from_bytes(x.as_ref()) > 0)
+ .map(|x| u64_from_be_bytes(x) > 0)
.unwrap_or(false);
if needed {
let path = self.block_path(hash);
@@ -215,84 +218,95 @@ impl BlockManager {
}
pub fn block_incref(&self, hash: &Hash) -> Result<(), Error> {
- let old_rc = self.rc.get(&hash)?;
- self.rc.merge(&hash, vec![1])?;
- if old_rc.map(|x| u64_from_bytes(&x[..]) == 0).unwrap_or(true) {
- self.put_to_resync(&hash, BLOCK_RW_TIMEOUT.as_millis() as u64)?;
+ 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 {
+ self.put_to_resync(&hash, BLOCK_RW_TIMEOUT)?;
}
Ok(())
}
pub fn block_decref(&self, hash: &Hash) -> Result<(), Error> {
- let new_rc = self.rc.merge(&hash, vec![0])?;
- if new_rc.map(|x| u64_from_bytes(&x[..]) == 0).unwrap_or(true) {
- self.put_to_resync(&hash, 0)?;
+ 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() {
+ self.put_to_resync(&hash, BLOCK_GC_TIMEOUT)?;
}
Ok(())
}
- fn put_to_resync(&self, hash: &Hash, delay_millis: u64) -> Result<(), Error> {
- let when = now_msec() + delay_millis;
+ fn put_to_resync(&self, hash: &Hash, delay: Duration) -> Result<(), Error> {
+ let when = now_msec() + delay.as_millis() as u64;
trace!("Put resync_queue: {} {:?}", when, hash);
let mut key = u64::to_be_bytes(when).to_vec();
key.extend(hash.as_ref());
self.resync_queue.insert(key, hash.as_ref())?;
- self.resync_notify.notify();
+ self.resync_notify.notify_waiters();
Ok(())
}
- async fn resync_loop(
- self: Arc<Self>,
- mut must_exit: watch::Receiver<bool>,
- ) -> Result<(), Error> {
- let mut n_failures = 0usize;
+ async fn resync_loop(self: Arc<Self>, mut must_exit: watch::Receiver<bool>) {
while !*must_exit.borrow() {
- if let Some((time_bytes, hash_bytes)) = self.resync_queue.pop_min()? {
- let time_msec = u64_from_bytes(&time_bytes[0..8]);
- let now = now_msec();
- if now >= time_msec {
- let mut hash = [0u8; 32];
- hash.copy_from_slice(hash_bytes.as_ref());
- let hash = Hash::from(hash);
-
- if let Err(e) = self.resync_iter(&hash).await {
- warn!("Failed to resync block {:?}, retrying later: {}", hash, e);
- self.put_to_resync(&hash, RESYNC_RETRY_TIMEOUT.as_millis() as u64)?;
- n_failures += 1;
- if n_failures >= 10 {
- warn!("Too many resync failures, throttling.");
- tokio::time::delay_for(Duration::from_secs(1)).await;
- }
- } else {
- n_failures = 0;
- }
- } else {
- self.resync_queue.insert(time_bytes, hash_bytes)?;
- let delay = tokio::time::delay_for(Duration::from_millis(time_msec - now));
- select! {
- _ = delay.fuse() => (),
- _ = self.resync_notify.notified().fuse() => (),
- _ = must_exit.recv().fuse() => (),
- }
+ if let Err(e) = self.resync_iter(&mut must_exit).await {
+ warn!("Error in block resync loop: {}", e);
+ select! {
+ _ = 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 {
+ let delay = tokio::time::sleep(Duration::from_millis(time_msec - now));
select! {
+ _ = delay.fuse() => (),
_ = self.resync_notify.notified().fuse() => (),
- _ = must_exit.recv().fuse() => (),
+ _ = must_exit.changed().fuse() => (),
}
}
+ } else {
+ select! {
+ _ = self.resync_notify.notified().fuse() => (),
+ _ = must_exit.changed().fuse() => (),
+ }
}
Ok(())
}
- async fn resync_iter(&self, hash: &Hash) -> Result<(), Error> {
+ async fn resync_block(&self, hash: &Hash) -> Result<(), Error> {
+ let lock = self.data_dir_lock.lock().await;
+
let path = self.block_path(hash);
let exists = fs::metadata(&path).await.is_ok();
let needed = self
.rc
.get(hash.as_ref())?
- .map(|x| u64_from_bytes(x.as_ref()) > 0)
+ .map(|x| u64_from_be_bytes(x) > 0)
.unwrap_or(false);
if exists != needed {
@@ -305,9 +319,10 @@ impl BlockManager {
if exists && !needed {
trace!("Offloading block {:?}", hash);
- let ring = self.system.ring.borrow().clone();
-
- let mut who = self.replication.replication_nodes(&hash, &ring);
+ 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")));
+ }
who.retain(|id| *id != self.system.id);
let msg = Arc::new(Message::NeedBlockQuery(*hash));
@@ -340,17 +355,17 @@ impl BlockManager {
need_nodes.len()
);
- let put_block_message = Arc::new(self.read_block(hash).await?);
- let put_resps = join_all(need_nodes.iter().map(|to| {
- self.rpc_client
- .call_arc(*to, put_block_message.clone(), BLOCK_RW_TIMEOUT)
- }))
- .await;
- for resp in put_resps {
- resp?;
- }
+ let put_block_message = self.read_block(hash).await?;
+ self.rpc_client
+ .try_call_many(
+ &need_nodes[..],
+ put_block_message,
+ RequestStrategy::with_quorum(need_nodes.len())
+ .with_timeout(BLOCK_RW_TIMEOUT),
+ )
+ .await?;
}
- trace!(
+ info!(
"Deleting block {:?}, offload finished ({} / {})",
hash,
need_nodes.len(),
@@ -358,10 +373,11 @@ impl BlockManager {
);
fs::remove_file(path).await?;
- self.resync_queue.remove(&hash)?;
}
if needed && !exists {
+ 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.
@@ -373,7 +389,7 @@ impl BlockManager {
}
pub async fn rpc_get_block(&self, hash: &Hash) -> Result<Vec<u8>, Error> {
- let who = self.replication.read_nodes(&hash, &self.system);
+ let who = self.replication.read_nodes(&hash);
let resps = self
.rpc_client
.try_call_many(
@@ -397,12 +413,12 @@ impl BlockManager {
}
pub async fn rpc_put_block(&self, hash: Hash, data: Vec<u8>) -> Result<(), Error> {
- let who = self.replication.write_nodes(&hash, &self.system);
+ let who = self.replication.write_nodes(&hash);
self.rpc_client
.try_call_many(
&who[..],
Message::PutBlock(PutBlockMessage { hash, data }),
- RequestStrategy::with_quorum(self.replication.write_quorum(&self.system))
+ RequestStrategy::with_quorum(self.replication.write_quorum())
.with_timeout(BLOCK_RW_TIMEOUT),
)
.await?;
@@ -414,15 +430,15 @@ impl BlockManager {
let garage = self.garage.load_full().unwrap();
let mut last_hash = None;
let mut i = 0usize;
- for entry in garage.block_ref_table.store.iter() {
+ for entry in garage.block_ref_table.data.store.iter() {
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 {
+ if !block_ref.deleted.get() {
last_hash = Some(block_ref.block);
- self.put_to_resync(&block_ref.block, 0)?;
+ self.put_to_resync(&block_ref.block, Duration::from_secs(0))?;
}
i += 1;
if i & 0xFF == 0 && *must_exit.borrow() {
@@ -447,8 +463,12 @@ impl BlockManager {
// so that we can offload them if necessary and then delete them locally.
async move {
let mut ls_data_dir = fs::read_dir(path).await?;
- while let Some(data_dir_ent) = ls_data_dir.next().await {
- let data_dir_ent = data_dir_ent?;
+ loop {
+ let data_dir_ent = ls_data_dir.next_entry().await?;
+ let data_dir_ent = match data_dir_ent {
+ Some(x) => x,
+ None => break,
+ };
let name = data_dir_ent.file_name();
let name = match name.into_string() {
Ok(x) => x,
@@ -466,7 +486,7 @@ impl BlockManager {
};
let mut hash = [0u8; 32];
hash.copy_from_slice(&hash_bytes[..]);
- self.put_to_resync(&hash.into(), 0)?;
+ self.put_to_resync(&hash.into(), Duration::from_secs(0))?;
}
if *must_exit.borrow() {
@@ -477,32 +497,19 @@ impl BlockManager {
}
.boxed()
}
+
+ pub fn resync_queue_len(&self) -> usize {
+ self.resync_queue.len()
+ }
+
+ pub fn rc_len(&self) -> usize {
+ self.rc.len()
+ }
}
-fn u64_from_bytes(bytes: &[u8]) -> u64 {
- assert!(bytes.len() == 8);
+fn u64_from_be_bytes<T: AsRef<[u8]>>(bytes: T) -> u64 {
+ assert!(bytes.as_ref().len() == 8);
let mut x8 = [0u8; 8];
- x8.copy_from_slice(bytes);
+ x8.copy_from_slice(bytes.as_ref());
u64::from_be_bytes(x8)
}
-
-fn rc_merge(_key: &[u8], old: Option<&[u8]>, new: &[u8]) -> Option<Vec<u8>> {
- let old = old.map(u64_from_bytes).unwrap_or(0);
- assert!(new.len() == 1);
- let new = match new[0] {
- 0 => {
- if old > 0 {
- old - 1
- } else {
- 0
- }
- }
- 1 => old + 1,
- _ => unreachable!(),
- };
- if new == 0 {
- None
- } else {
- Some(u64::to_be_bytes(new).to_vec())
- }
-}