aboutsummaryrefslogtreecommitdiff
path: root/src/block/manager.rs
diff options
context:
space:
mode:
Diffstat (limited to 'src/block/manager.rs')
-rw-r--r--src/block/manager.rs941
1 files changed, 311 insertions, 630 deletions
diff --git a/src/block/manager.rs b/src/block/manager.rs
index 1c04a335..7f439b96 100644
--- a/src/block/manager.rs
+++ b/src/block/manager.rs
@@ -1,29 +1,32 @@
-use std::convert::TryInto;
-use std::path::{Path, PathBuf};
+use std::path::PathBuf;
+use std::pin::Pin;
use std::sync::Arc;
use std::time::Duration;
use async_trait::async_trait;
+use bytes::Bytes;
use serde::{Deserialize, Serialize};
-use futures::future::*;
-use futures::select;
+use futures::Stream;
+use futures_util::stream::StreamExt;
use tokio::fs;
-use tokio::io::{AsyncReadExt, AsyncWriteExt};
-use tokio::sync::{watch, Mutex, Notify};
+use tokio::io::{AsyncReadExt, AsyncWriteExt, BufReader};
+use tokio::sync::{mpsc, Mutex, MutexGuard};
use opentelemetry::{
trace::{FutureExt as OtelFutureExt, TraceContextExt, Tracer},
- Context, KeyValue,
+ Context,
};
+use garage_rpc::rpc_helper::netapp::stream::{stream_asyncread, ByteStream};
+
+use garage_db as db;
+
use garage_util::data::*;
use garage_util::error::*;
use garage_util::metrics::RecordDuration;
-use garage_util::sled_counter::SledCountedTree;
-use garage_util::time::*;
-use garage_util::tranquilizer::Tranquilizer;
+use garage_rpc::rpc_helper::OrderTag;
use garage_rpc::system::System;
use garage_rpc::*;
@@ -32,24 +35,12 @@ use garage_table::replication::{TableReplication, TableShardedReplication};
use crate::block::*;
use crate::metrics::*;
use crate::rc::*;
+use crate::repair::*;
+use crate::resync::*;
/// Size under which data will be stored inlined in database instead of as files
pub const INLINE_THRESHOLD: usize = 3072;
-// Timeout for RPCs that read and write blocks to remote nodes
-const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(30);
-// Timeout for RPCs that ask other nodes whether they need a copy
-// of a given block before we delete it locally
-const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5);
-
-// The delay between the time where a resync operation fails
-// and the time when it is retried, with exponential backoff
-// (multiplied by 2, 4, 8, 16, etc. for every consecutive failure).
-const RESYNC_RETRY_DELAY: Duration = Duration::from_secs(60);
-// The minimum retry delay is 60 seconds = 1 minute
-// The maximum retry delay is 60 seconds * 2^6 = 60 seconds << 6 = 64 minutes (~1 hour)
-const RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER: u64 = 6;
-
// The delay between the moment when the reference counter
// drops to zero, and the moment where we allow ourselves
// to delete the block locally.
@@ -60,12 +51,12 @@ pub(crate) const BLOCK_GC_DELAY: Duration = Duration::from_secs(600);
pub enum BlockRpc {
Ok,
/// Message to ask for a block of data, by hash
- GetBlock(Hash),
+ GetBlock(Hash, Option<OrderTag>),
/// Message to send a block of data, either because requested, of for first delivery of new
/// block
PutBlock {
hash: Hash,
- data: DataBlock,
+ header: DataBlockHeader,
},
/// Ask other node if they should have this block, but don't actually have it
NeedBlockQuery(Hash),
@@ -85,20 +76,18 @@ pub struct BlockManager {
pub data_dir: PathBuf,
compression_level: Option<i32>,
- background_tranquility: u32,
- mutation_lock: Mutex<BlockManagerLocked>,
+ mutation_lock: [Mutex<BlockManagerLocked>; 256],
- rc: BlockRc,
+ pub(crate) rc: BlockRc,
+ pub resync: BlockResyncManager,
- resync_queue: SledCountedTree,
- resync_notify: Notify,
- resync_errors: SledCountedTree,
+ pub(crate) system: Arc<System>,
+ pub(crate) endpoint: Arc<Endpoint<BlockRpc, Self>>,
- system: Arc<System>,
- endpoint: Arc<Endpoint<BlockRpc, Self>>,
+ pub(crate) metrics: BlockManagerMetrics,
- metrics: BlockManagerMetrics,
+ tx_scrub_command: mpsc::Sender<ScrubWorkerCommand>,
}
// This custom struct contains functions that must only be ran
@@ -108,10 +97,9 @@ struct BlockManagerLocked();
impl BlockManager {
pub fn new(
- db: &sled::Db,
+ db: &db::Db,
data_dir: PathBuf,
compression_level: Option<i32>,
- background_tranquility: u32,
replication: TableShardedReplication,
system: Arc<System>,
) -> Arc<Self> {
@@ -120,215 +108,323 @@ impl BlockManager {
.expect("Unable to open block_local_rc tree");
let rc = BlockRc::new(rc);
- let resync_queue = db
- .open_tree("block_local_resync_queue")
- .expect("Unable to open block_local_resync_queue tree");
- let resync_queue = SledCountedTree::new(resync_queue);
-
- let resync_errors = db
- .open_tree("block_local_resync_errors")
- .expect("Unable to open block_local_resync_errors tree");
- let resync_errors = SledCountedTree::new(resync_errors);
+ let resync = BlockResyncManager::new(db, &system);
let endpoint = system
.netapp
- .endpoint("garage_model/block.rs/Rpc".to_string());
+ .endpoint("garage_block/manager.rs/Rpc".to_string());
- let manager_locked = BlockManagerLocked();
+ let metrics = BlockManagerMetrics::new(resync.queue.clone(), resync.errors.clone());
- let metrics = BlockManagerMetrics::new(resync_queue.clone(), resync_errors.clone());
+ let (scrub_tx, scrub_rx) = mpsc::channel(1);
let block_manager = Arc::new(Self {
replication,
data_dir,
compression_level,
- background_tranquility,
- mutation_lock: Mutex::new(manager_locked),
+ mutation_lock: [(); 256].map(|_| Mutex::new(BlockManagerLocked())),
rc,
- resync_queue,
- resync_notify: Notify::new(),
- resync_errors,
+ resync,
system,
endpoint,
metrics,
+ tx_scrub_command: scrub_tx,
});
block_manager.endpoint.set_handler(block_manager.clone());
- block_manager.clone().spawn_background_worker();
+ // Spawn a bunch of resync workers
+ for index in 0..MAX_RESYNC_WORKERS {
+ let worker = ResyncWorker::new(index, block_manager.clone());
+ block_manager.system.background.spawn_worker(worker);
+ }
+
+ // Spawn scrub worker
+ let scrub_worker = ScrubWorker::new(block_manager.clone(), scrub_rx);
+ block_manager.system.background.spawn_worker(scrub_worker);
block_manager
}
/// Ask nodes that might have a (possibly compressed) block for it
- async fn rpc_get_raw_block(&self, hash: &Hash) -> Result<DataBlock, Error> {
+ /// Return it as a stream with a header
+ async fn rpc_get_raw_block_streaming(
+ &self,
+ hash: &Hash,
+ order_tag: Option<OrderTag>,
+ ) -> Result<(DataBlockHeader, ByteStream), Error> {
let who = self.replication.read_nodes(hash);
- let resps = self
- .system
- .rpc
- .try_call_many(
- &self.endpoint,
- &who[..],
- BlockRpc::GetBlock(*hash),
- RequestStrategy::with_priority(PRIO_NORMAL)
- .with_quorum(1)
- .with_timeout(BLOCK_RW_TIMEOUT)
- .interrupt_after_quorum(true),
- )
- .await?;
+ let who = self.system.rpc.request_order(&who);
+
+ for node in who.iter() {
+ let node_id = NodeID::from(*node);
+ let rpc = self.endpoint.call_streaming(
+ &node_id,
+ BlockRpc::GetBlock(*hash, order_tag),
+ PRIO_NORMAL | PRIO_SECONDARY,
+ );
+ tokio::select! {
+ res = rpc => {
+ let res = match res {
+ Ok(res) => res,
+ Err(e) => {
+ debug!("Node {:?} returned error: {}", node, e);
+ continue;
+ }
+ };
+ let (header, stream) = match res.into_parts() {
+ (Ok(BlockRpc::PutBlock { hash: _, header }), Some(stream)) => (header, stream),
+ _ => {
+ debug!("Node {:?} returned a malformed response", node);
+ continue;
+ }
+ };
+ return Ok((header, stream));
+ }
+ _ = tokio::time::sleep(self.system.rpc.rpc_timeout()) => {
+ debug!("Node {:?} didn't return block in time, trying next.", node);
+ }
+ };
+ }
- for resp in resps {
- if let BlockRpc::PutBlock { data, .. } = resp {
- return Ok(data);
- }
+ Err(Error::Message(format!(
+ "Unable to read block {:?}: no node returned a valid block",
+ hash
+ )))
+ }
+
+ /// 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,
+ order_tag: Option<OrderTag>,
+ ) -> Result<DataBlock, Error> {
+ let who = self.replication.read_nodes(hash);
+ let who = self.system.rpc.request_order(&who);
+
+ for node in who.iter() {
+ let node_id = NodeID::from(*node);
+ let rpc = self.endpoint.call_streaming(
+ &node_id,
+ BlockRpc::GetBlock(*hash, order_tag),
+ PRIO_NORMAL | PRIO_SECONDARY,
+ );
+ tokio::select! {
+ res = rpc => {
+ let res = match res {
+ Ok(res) => res,
+ Err(e) => {
+ debug!("Node {:?} returned error: {}", node, e);
+ continue;
+ }
+ };
+ let (header, stream) = match res.into_parts() {
+ (Ok(BlockRpc::PutBlock { hash: _, header }), Some(stream)) => (header, stream),
+ _ => {
+ debug!("Node {:?} returned a malformed response", node);
+ continue;
+ }
+ };
+ match read_stream_to_end(stream).await {
+ Ok(bytes) => return Ok(DataBlock::from_parts(header, bytes)),
+ Err(e) => {
+ debug!("Error reading stream from node {:?}: {}", node, e);
+ }
+ }
+ }
+ _ = tokio::time::sleep(self.system.rpc.rpc_timeout()) => {
+ debug!("Node {:?} didn't return block in time, trying next.", node);
+ }
+ };
}
+
Err(Error::Message(format!(
- "Unable to read block {:?}: no valid blocks returned",
+ "Unable to read block {:?}: no node returned a valid block",
hash
)))
}
// ---- Public interface ----
+ /// Ask nodes that might have a block for it,
+ /// return it as a stream
+ pub async fn rpc_get_block_streaming(
+ &self,
+ hash: &Hash,
+ order_tag: Option<OrderTag>,
+ ) -> Result<
+ Pin<Box<dyn Stream<Item = Result<Bytes, std::io::Error>> + Send + Sync + 'static>>,
+ Error,
+ > {
+ let (header, stream) = self.rpc_get_raw_block_streaming(hash, order_tag).await?;
+ match header {
+ DataBlockHeader::Plain => Ok(stream),
+ 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)))
+ }
+ }
+ }
+
/// Ask nodes that might have a block for it
- pub async fn rpc_get_block(&self, hash: &Hash) -> Result<Vec<u8>, Error> {
- self.rpc_get_raw_block(hash).await?.verify_get(*hash)
+ 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)
}
/// Send block to nodes that should have it
- pub async fn rpc_put_block(&self, hash: Hash, data: Vec<u8>) -> Result<(), Error> {
+ pub async fn rpc_put_block(&self, hash: Hash, data: Bytes) -> Result<(), Error> {
let who = self.replication.write_nodes(&hash);
- let data = DataBlock::from_buffer(data, self.compression_level);
+
+ 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);
+
self.system
.rpc
.try_call_many(
&self.endpoint,
&who[..],
- BlockRpc::PutBlock { hash, data },
- RequestStrategy::with_priority(PRIO_NORMAL)
- .with_quorum(self.replication.write_quorum())
- .with_timeout(BLOCK_RW_TIMEOUT),
+ put_block_rpc,
+ RequestStrategy::with_priority(PRIO_NORMAL | PRIO_SECONDARY)
+ .with_quorum(self.replication.write_quorum()),
)
.await?;
- Ok(())
- }
- /// Launch the repair procedure on the data store
- ///
- /// This will list all blocks locally present, as well as those
- /// that are required because of refcount > 0, and will try
- /// to fix any mismatch between the two.
- pub async fn repair_data_store(&self, must_exit: &watch::Receiver<bool>) -> Result<(), Error> {
- // 1. Repair blocks from RC table.
- for (i, entry) in self.rc.rc.iter().enumerate() {
- let (hash, _) = entry?;
- let hash = Hash::try_from(&hash[..]).unwrap();
- self.put_to_resync(&hash, Duration::from_secs(0))?;
- if i & 0xFF == 0 && *must_exit.borrow() {
- return Ok(());
- }
- }
-
- // 2. Repair blocks actually on disk
- // Lists all blocks on disk and adds them to the resync queue.
- // This allows us to find blocks we are storing but don't actually need,
- // so that we can offload them if necessary and then delete them locally.
- self.for_each_file(
- (),
- move |_, hash| async move {
- self.put_to_resync(&hash, Duration::from_secs(0))
- .map_err(Into::into)
- },
- must_exit,
- )
- .await
- }
-
- /// Verify integrity of each block on disk. Use `speed_limit` to limit the load generated by
- /// this function.
- pub async fn scrub_data_store(
- &self,
- must_exit: &watch::Receiver<bool>,
- tranquility: u32,
- ) -> Result<(), Error> {
- let tranquilizer = Tranquilizer::new(30);
- self.for_each_file(
- tranquilizer,
- move |mut tranquilizer, hash| async move {
- let _ = self.read_block(&hash).await;
- tranquilizer.tranquilize(tranquility).await;
- Ok(tranquilizer)
- },
- must_exit,
- )
- .await
- }
-
- /// Get lenght of resync queue
- pub fn resync_queue_len(&self) -> usize {
- self.resync_queue.len()
+ Ok(())
}
- /// Get number of blocks that have an error
- pub fn resync_errors_len(&self) -> usize {
- self.resync_errors.len()
+ /// Get number of items in the refcount table
+ pub fn rc_len(&self) -> Result<usize, Error> {
+ Ok(self.rc.rc.len()?)
}
- /// Get number of items in the refcount table
- pub fn rc_len(&self) -> usize {
- self.rc.rc.len()
+ /// Send command to start/stop/manager scrub worker
+ pub async fn send_scrub_command(&self, cmd: ScrubWorkerCommand) {
+ let _ = self.tx_scrub_command.send(cmd).await;
}
//// ----- Managing the reference counter ----
/// Increment the number of time a block is used, putting it to resynchronization if it is
/// required, but not known
- pub fn block_incref(&self, hash: &Hash) -> Result<(), Error> {
- if self.rc.block_incref(hash)? {
+ 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.
- self.put_to_resync(hash, 2 * BLOCK_RW_TIMEOUT)?;
+ let this = self.clone();
+ tokio::spawn(async move {
+ if let Err(e) = this
+ .resync
+ .put_to_resync(&hash, 2 * this.system.rpc.rpc_timeout())
+ {
+ error!("Block {:?} could not be put in resync queue: {}.", hash, e);
+ }
+ });
}
Ok(())
}
/// Decrement the number of time a block is used
- pub fn block_decref(&self, hash: &Hash) -> Result<(), Error> {
- if self.rc.block_decref(hash)? {
+ pub fn block_decref(
+ self: &Arc<Self>,
+ tx: &mut db::Transaction,
+ hash: Hash,
+ ) -> db::TxOpResult<()> {
+ if self.rc.block_decref(tx, &hash)? {
// 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.
- self.put_to_resync(hash, BLOCK_GC_DELAY + Duration::from_secs(10))?;
+ 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))
+ {
+ error!("Block {:?} could not be put in resync queue: {}.", hash, e);
+ }
+ });
}
Ok(())
}
// ---- Reading and writing blocks locally ----
+ 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
+ }
+
/// Write a block to disk
- async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<BlockRpc, Error> {
+ pub(crate) async fn write_block(&self, hash: &Hash, data: &DataBlock) -> Result<(), Error> {
+ let tracer = opentelemetry::global::tracer("garage");
+
let write_size = data.inner_buffer().len() as u64;
- let res = self
- .mutation_lock
- .lock()
+ self.lock_mutate(hash)
.await
.write_block(hash, data, self)
.bound_record_duration(&self.metrics.block_write_duration)
+ .with_context(Context::current_with_span(
+ tracer.start("BlockManagerLocked::write_block"),
+ ))
.await?;
self.metrics.bytes_written.add(write_size);
- Ok(res)
+ Ok(())
+ }
+
+ async fn handle_get_block(&self, hash: &Hash, order_tag: Option<OrderTag>) -> Resp<BlockRpc> {
+ let block = match self.read_block(hash).await {
+ Ok(data) => data,
+ Err(e) => return Resp::new(Err(e)),
+ };
+
+ let (header, data) = block.into_parts();
+
+ let resp = Resp::new(Ok(BlockRpc::PutBlock {
+ hash: *hash,
+ header,
+ }))
+ .with_stream_from_buffer(data);
+
+ if let Some(order_tag) = order_tag {
+ resp.with_order_tag(order_tag)
+ } else {
+ resp
+ }
}
/// Read block from disk, verifying it's integrity
- async fn read_block(&self, hash: &Hash) -> Result<BlockRpc, Error> {
+ pub(crate) async fn read_block(&self, hash: &Hash) -> Result<DataBlock, Error> {
let data = self
.read_block_internal(hash)
.bound_record_duration(&self.metrics.block_read_duration)
@@ -338,7 +434,7 @@ impl BlockManager {
.bytes_read
.add(data.inner_buffer().len() as u64);
- Ok(BlockRpc::PutBlock { hash: *hash, data })
+ Ok(data)
}
async fn read_block_internal(&self, hash: &Hash) -> Result<DataBlock, Error> {
@@ -347,7 +443,8 @@ impl BlockManager {
Ok(c) => c,
Err(e) => {
// Not found but maybe we should have had it ??
- self.put_to_resync(hash, 2 * BLOCK_RW_TIMEOUT)?;
+ self.resync
+ .put_to_resync(hash, 2 * self.system.rpc.rpc_timeout())?;
return Err(Into::into(e));
}
};
@@ -361,37 +458,47 @@ impl BlockManager {
drop(f);
let data = if compressed {
- DataBlock::Compressed(data)
+ DataBlock::Compressed(data.into())
} else {
- DataBlock::Plain(data)
+ DataBlock::Plain(data.into())
};
if data.verify(*hash).is_err() {
self.metrics.corruption_counter.add(1);
- self.mutation_lock
- .lock()
+ self.lock_mutate(hash)
.await
.move_block_to_corrupted(hash, self)
.await?;
- self.put_to_resync(hash, Duration::from_millis(0))?;
+ self.resync.put_to_resync(hash, Duration::from_millis(0))?;
return Err(Error::CorruptData(*hash));
}
Ok(data)
}
- /// 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 BlockStatus { exists, needed } = self
- .mutation_lock
- .lock()
+ /// Check if this node has a block and whether it needs it
+ pub(crate) async fn check_block_status(&self, hash: &Hash) -> Result<BlockStatus, Error> {
+ self.lock_mutate(hash)
.await
.check_block_status(hash, self)
- .await?;
+ .await
+ }
+
+ /// 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 BlockStatus { exists, needed } = self.check_block_status(hash).await?;
Ok(needed.is_nonzero() && !exists)
}
+ /// Delete block if it is not needed anymore
+ pub(crate) async fn delete_if_unneeded(&self, hash: &Hash) -> Result<(), Error> {
+ self.lock_mutate(hash)
+ .await
+ .delete_if_unneeded(hash, self)
+ .await
+ }
+
/// Utility: gives the path of the directory in which a block should be found
fn block_dir(&self, hash: &Hash) -> PathBuf {
let mut path = self.data_dir.clone();
@@ -419,431 +526,38 @@ impl BlockManager {
fs::metadata(&path).await.map(|_| false).map_err(Into::into)
}
- // ---- Resync loop ----
-
- // This part manages a queue of blocks that need to be
- // "resynchronized", i.e. that need to have a check that
- // they are at present if we need them, or that they are
- // deleted once the garbage collection delay has passed.
- //
- // Here are some explanations on how the resync queue works.
- // There are two Sled trees that are used to have information
- // about the status of blocks that need to be resynchronized:
- //
- // - resync_queue: a tree that is ordered first by a timestamp
- // (in milliseconds since Unix epoch) that is the time at which
- // the resync must be done, and second by block hash.
- // The key in this tree is just:
- // concat(timestamp (8 bytes), hash (32 bytes))
- // The value is the same 32-byte hash.
- //
- // - resync_errors: a tree that indicates for each block
- // if the last resync resulted in an error, and if so,
- // the following two informations (see the ErrorCounter struct):
- // - how many consecutive resync errors for this block?
- // - when was the last try?
- // These two informations are used to implement an
- // exponential backoff retry strategy.
- // The key in this tree is the 32-byte hash of the block,
- // and the value is the encoded ErrorCounter value.
- //
- // We need to have these two trees, because the resync queue
- // is not just a queue of items to process, but a set of items
- // that are waiting a specific delay until we can process them
- // (the delay being necessary both internally for the exponential
- // backoff strategy, and exposed as a parameter when adding items
- // to the queue, e.g. to wait until the GC delay has passed).
- // This is why we need one tree ordered by time, and one
- // ordered by identifier of item to be processed (block hash).
- //
- // When the worker wants to process an item it takes from
- // resync_queue, it checks in resync_errors that if there is an
- // exponential back-off delay to await, it has passed before we
- // process the item. If not, the item in the queue is skipped
- // (but added back for later processing after the time of the
- // delay).
- //
- // An alternative that would have seemed natural is to
- // only add items to resync_queue with a processing time that is
- // after the delay, but there are several issues with this:
- // - This requires to synchronize updates to resync_queue and
- // resync_errors (with the current model, there is only one thread,
- // the worker thread, that accesses resync_errors,
- // so no need to synchronize) by putting them both in a lock.
- // This would mean that block_incref might need to take a lock
- // before doing its thing, meaning it has much more chances of
- // not completing successfully if something bad happens to Garage.
- // Currently Garage is not able to recover from block_incref that
- // doesn't complete successfully, because it is necessary to ensure
- // the consistency between the state of the block manager and
- // information in the BlockRef table.
- // - If a resync fails, we put that block in the resync_errors table,
- // and also add it back to resync_queue to be processed after
- // the exponential back-off delay,
- // but maybe the block is already scheduled to be resynced again
- // at another time that is before the exponential back-off delay,
- // and we have no way to check that easily. This means that
- // in all cases, we need to check the resync_errors table
- // in the resync loop at the time when a block is popped from
- // the resync_queue.
- // Overall, the current design is therefore simpler and more robust
- // because it tolerates inconsistencies between the resync_queue
- // and resync_errors table (items being scheduled in resync_queue
- // for times that are earlier than the exponential back-off delay
- // is a natural condition that is handled properly).
-
- fn spawn_background_worker(self: Arc<Self>) {
- // Launch a background workers for background resync loop processing
- let background = self.system.background.clone();
- tokio::spawn(async move {
- tokio::time::sleep(Duration::from_secs(10)).await;
- background.spawn_worker("block resync worker".into(), move |must_exit| {
- self.resync_loop(must_exit)
- });
- });
- }
-
- fn put_to_resync(&self, hash: &Hash, delay: Duration) -> Result<(), sled::Error> {
- let when = now_msec() + delay.as_millis() as u64;
- self.put_to_resync_at(hash, when)
- }
-
- fn put_to_resync_at(&self, hash: &Hash, when: u64) -> Result<(), sled::Error> {
- 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_waiters();
- Ok(())
- }
-
- async fn resync_loop(self: Arc<Self>, mut must_exit: watch::Receiver<bool>) {
- let mut tranquilizer = Tranquilizer::new(30);
-
- while !*must_exit.borrow() {
- match self.resync_iter(&mut must_exit).await {
- Ok(true) => {
- tranquilizer.tranquilize(self.background_tranquility).await;
- }
- Ok(false) => {
- tranquilizer.reset();
- }
- Err(e) => {
- // The errors that we have here are only Sled errors
- // We don't really know how to handle them so just ¯\_(ツ)_/¯
- // (there is kind of an assumption that Sled won't error on us,
- // if it does there is not much we can do -- TODO should we just panic?)
- error!(
- "Could not do a resync iteration: {} (this is a very bad error)",
- e
- );
- tranquilizer.reset();
- }
- }
- }
- }
-
- // The result of resync_iter is:
- // - Ok(true) -> a block was processed (successfully or not)
- // - Ok(false) -> no block was processed, but we are ready for the next iteration
- // - Err(_) -> a Sled error occurred when reading/writing from resync_queue/resync_errors
- async fn resync_iter(
- &self,
- must_exit: &mut watch::Receiver<bool>,
- ) -> Result<bool, sled::Error> {
- if let Some(first_pair_res) = self.resync_queue.iter().next() {
- let (time_bytes, hash_bytes) = first_pair_res?;
-
- let time_msec = u64::from_be_bytes(time_bytes[0..8].try_into().unwrap());
- let now = now_msec();
-
- if now >= time_msec {
- let hash = Hash::try_from(&hash_bytes[..]).unwrap();
-
- if let Some(ec) = self.resync_errors.get(hash.as_slice())? {
- let ec = ErrorCounter::decode(ec);
- if now < ec.next_try() {
- // if next retry after an error is not yet,
- // don't do resync and return early, but still
- // make sure the item is still in queue at expected time
- self.put_to_resync_at(&hash, ec.next_try())?;
- // ec.next_try() > now >= time_msec, so this remove
- // is not removing the one we added just above
- // (we want to do the remove after the insert to ensure
- // that the item is not lost if we crash in-between)
- self.resync_queue.remove(time_bytes)?;
- return Ok(false);
- }
- }
-
- let tracer = opentelemetry::global::tracer("garage");
- let trace_id = gen_uuid();
- let span = tracer
- .span_builder("Resync block")
- .with_trace_id(
- opentelemetry::trace::TraceId::from_hex(&hex::encode(
- &trace_id.as_slice()[..16],
- ))
- .unwrap(),
- )
- .with_attributes(vec![KeyValue::new("block", format!("{:?}", hash))])
- .start(&tracer);
-
- let res = self
- .resync_block(&hash)
- .with_context(Context::current_with_span(span))
- .bound_record_duration(&self.metrics.resync_duration)
- .await;
-
- self.metrics.resync_counter.add(1);
-
- if let Err(e) = &res {
- self.metrics.resync_error_counter.add(1);
- warn!("Error when resyncing {:?}: {}", hash, e);
-
- let err_counter = match self.resync_errors.get(hash.as_slice())? {
- Some(ec) => ErrorCounter::decode(ec).add1(now + 1),
- None => ErrorCounter::new(now + 1),
- };
-
- self.resync_errors
- .insert(hash.as_slice(), err_counter.encode())?;
-
- self.put_to_resync_at(&hash, err_counter.next_try())?;
- // err_counter.next_try() >= now + 1 > now,
- // the entry we remove from the queue is not
- // the entry we inserted with put_to_resync_at
- self.resync_queue.remove(time_bytes)?;
- } else {
- self.resync_errors.remove(hash.as_slice())?;
- self.resync_queue.remove(time_bytes)?;
- }
-
- Ok(true)
- } else {
- let delay = tokio::time::sleep(Duration::from_millis(time_msec - now));
- select! {
- _ = delay.fuse() => {},
- _ = self.resync_notify.notified().fuse() => {},
- _ = must_exit.changed().fuse() => {},
- }
- Ok(false)
- }
- } else {
- // Here we wait either for a notification that an item has been
- // added to the queue, or for a constant delay of 10 secs to expire.
- // The delay avoids a race condition where the notification happens
- // between the time we checked the queue and the first poll
- // to resync_notify.notified(): if that happens, we'll just loop
- // back 10 seconds later, which is fine.
- let delay = tokio::time::sleep(Duration::from_secs(10));
- select! {
- _ = delay.fuse() => {},
- _ = self.resync_notify.notified().fuse() => {},
- _ = must_exit.changed().fuse() => {},
- }
- Ok(false)
- }
- }
-
- async fn resync_block(&self, hash: &Hash) -> Result<(), Error> {
- let BlockStatus { exists, needed } = self
- .mutation_lock
+ async fn lock_mutate(&self, hash: &Hash) -> MutexGuard<'_, BlockManagerLocked> {
+ let tracer = opentelemetry::global::tracer("garage");
+ self.mutation_lock[hash.as_slice()[0] as usize]
.lock()
+ .with_context(Context::current_with_span(
+ tracer.start("Acquire mutation_lock"),
+ ))
.await
- .check_block_status(hash, self)
- .await?;
-
- if exists != needed.is_needed() || exists != needed.is_nonzero() {
- debug!(
- "Resync block {:?}: exists {}, nonzero rc {}, deletable {}",
- hash,
- exists,
- needed.is_nonzero(),
- needed.is_deletable(),
- );
- }
-
- if exists && needed.is_deletable() {
- info!("Resync block {:?}: offloading and deleting", hash);
-
- let mut who = self.replication.write_nodes(hash);
- if who.len() < self.replication.write_quorum() {
- return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string()));
- }
- who.retain(|id| *id != self.system.id);
-
- let msg = Arc::new(BlockRpc::NeedBlockQuery(*hash));
- let who_needs_fut = who.iter().map(|to| {
- self.system.rpc.call_arc(
- &self.endpoint,
- *to,
- msg.clone(),
- RequestStrategy::with_priority(PRIO_BACKGROUND)
- .with_timeout(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.err_context("NeedBlockQuery RPC")? {
- BlockRpc::NeedBlockReply(needed) => {
- if needed {
- need_nodes.push(*node);
- }
- }
- m => {
- return Err(Error::unexpected_rpc_message(m));
- }
- }
- }
-
- if !need_nodes.is_empty() {
- trace!(
- "Block {:?} needed by {} nodes, sending",
- hash,
- need_nodes.len()
- );
-
- for node in need_nodes.iter() {
- self.metrics
- .resync_send_counter
- .add(1, &[KeyValue::new("to", format!("{:?}", node))]);
- }
-
- let put_block_message = self.read_block(hash).await?;
- self.system
- .rpc
- .try_call_many(
- &self.endpoint,
- &need_nodes[..],
- put_block_message,
- RequestStrategy::with_priority(PRIO_BACKGROUND)
- .with_quorum(need_nodes.len())
- .with_timeout(BLOCK_RW_TIMEOUT),
- )
- .await
- .err_context("PutBlock RPC")?;
- }
- info!(
- "Deleting unneeded block {:?}, offload finished ({} / {})",
- hash,
- need_nodes.len(),
- who.len()
- );
-
- self.mutation_lock
- .lock()
- .await
- .delete_if_unneeded(hash, self)
- .await?;
-
- self.rc.clear_deleted_block_rc(hash)?;
- }
-
- if needed.is_nonzero() && !exists {
- info!(
- "Resync block {:?}: fetching absent but needed block (refcount > 0)",
- hash
- );
-
- let block_data = self.rpc_get_raw_block(hash).await?;
-
- self.metrics.resync_recv_counter.add(1);
-
- self.write_block(hash, &block_data).await?;
- }
-
- Ok(())
- }
-
- // ---- Utility: iteration on files in the data directory ----
-
- async fn for_each_file<F, Fut, State>(
- &self,
- state: State,
- mut f: F,
- must_exit: &watch::Receiver<bool>,
- ) -> Result<(), Error>
- where
- F: FnMut(State, Hash) -> Fut + Send,
- Fut: Future<Output = Result<State, Error>> + Send,
- State: Send,
- {
- self.for_each_file_rec(&self.data_dir, state, &mut f, must_exit)
- .await
- .map(|_| ())
- }
-
- fn for_each_file_rec<'a, F, Fut, State>(
- &'a self,
- path: &'a Path,
- mut state: State,
- f: &'a mut F,
- must_exit: &'a watch::Receiver<bool>,
- ) -> BoxFuture<'a, Result<State, Error>>
- where
- F: FnMut(State, Hash) -> Fut + Send,
- Fut: Future<Output = Result<State, Error>> + Send,
- State: Send + 'a,
- {
- async move {
- let mut ls_data_dir = fs::read_dir(path).await?;
- while let Some(data_dir_ent) = ls_data_dir.next_entry().await? {
- if *must_exit.borrow() {
- break;
- }
-
- let name = data_dir_ent.file_name();
- let name = if let Ok(n) = name.into_string() {
- n
- } else {
- continue;
- };
- let ent_type = data_dir_ent.file_type().await?;
-
- let name = name.strip_suffix(".zst").unwrap_or(&name);
- if name.len() == 2 && hex::decode(&name).is_ok() && ent_type.is_dir() {
- state = self
- .for_each_file_rec(&data_dir_ent.path(), state, f, must_exit)
- .await?;
- } else if name.len() == 64 {
- let hash_bytes = if let Ok(h) = hex::decode(&name) {
- h
- } else {
- continue;
- };
- let mut hash = [0u8; 32];
- hash.copy_from_slice(&hash_bytes[..]);
- state = f(state, hash.into()).await?;
- }
- }
- Ok(state)
- }
- .boxed()
}
}
#[async_trait]
-impl EndpointHandler<BlockRpc> for BlockManager {
- async fn handle(
- self: &Arc<Self>,
- message: &BlockRpc,
- _from: NodeID,
- ) -> Result<BlockRpc, Error> {
- match message {
- BlockRpc::PutBlock { hash, data } => self.write_block(hash, data).await,
- BlockRpc::GetBlock(h) => self.read_block(h).await,
- BlockRpc::NeedBlockQuery(h) => self.need_block(h).await.map(BlockRpc::NeedBlockReply),
- m => Err(Error::unexpected_rpc_message(m)),
+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),
+ ),
+ BlockRpc::GetBlock(h, order_tag) => self.handle_get_block(h, *order_tag).await,
+ BlockRpc::NeedBlockQuery(h) => {
+ Resp::new(self.need_block(h).await.map(BlockRpc::NeedBlockReply))
+ }
+ m => Resp::new(Err(Error::unexpected_rpc_message(m))),
}
}
}
-struct BlockStatus {
- exists: bool,
- needed: RcEntry,
+pub(crate) struct BlockStatus {
+ pub(crate) exists: bool,
+ pub(crate) needed: RcEntry,
}
impl BlockManagerLocked {
@@ -863,7 +577,7 @@ impl BlockManagerLocked {
hash: &Hash,
data: &DataBlock,
mgr: &BlockManager,
- ) -> Result<BlockRpc, Error> {
+ ) -> Result<(), Error> {
let compressed = data.is_compressed();
let data = data.inner_buffer();
@@ -874,8 +588,8 @@ impl BlockManagerLocked {
fs::create_dir_all(&directory).await?;
let to_delete = match (mgr.is_block_compressed(hash).await, compressed) {
- (Ok(true), _) => return Ok(BlockRpc::Ok),
- (Ok(false), false) => return Ok(BlockRpc::Ok),
+ (Ok(true), _) => return Ok(()),
+ (Ok(false), false) => return Ok(()),
(Ok(false), true) => {
let path_to_delete = path.clone();
path.set_extension("zst");
@@ -914,7 +628,7 @@ impl BlockManagerLocked {
dir.sync_all().await?;
drop(dir);
- Ok(BlockRpc::Ok)
+ Ok(())
}
async fn move_block_to_corrupted(&self, hash: &Hash, mgr: &BlockManager) -> Result<(), Error> {
@@ -949,49 +663,16 @@ impl BlockManagerLocked {
}
}
-/// Counts the number of errors when resyncing a block,
-/// and the time of the last try.
-/// Used to implement exponential backoff.
-#[derive(Clone, Copy, Debug)]
-struct ErrorCounter {
- errors: u64,
- last_try: u64,
-}
-
-impl ErrorCounter {
- fn new(now: u64) -> Self {
- Self {
- errors: 1,
- last_try: now,
- }
+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")?);
}
- fn decode(data: sled::IVec) -> Self {
- Self {
- errors: u64::from_be_bytes(data[0..8].try_into().unwrap()),
- last_try: u64::from_be_bytes(data[8..16].try_into().unwrap()),
- }
- }
- fn encode(&self) -> Vec<u8> {
- [
- u64::to_be_bytes(self.errors),
- u64::to_be_bytes(self.last_try),
- ]
+ Ok(parts
+ .iter()
+ .map(|x| &x[..])
+ .collect::<Vec<_>>()
.concat()
- }
-
- fn add1(self, now: u64) -> Self {
- Self {
- errors: self.errors + 1,
- last_try: now,
- }
- }
-
- fn delay_msec(&self) -> u64 {
- (RESYNC_RETRY_DELAY.as_millis() as u64)
- << std::cmp::min(self.errors - 1, RESYNC_RETRY_DELAY_MAX_BACKOFF_POWER)
- }
- fn next_try(&self) -> u64 {
- self.last_try + self.delay_msec()
- }
+ .into())
}