From de9d6cddf709e686ada3d1e71de7b31d7704b8b5 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 12 Dec 2022 17:16:49 +0100 Subject: Prettier worker list table; remove useless CLI log messages --- src/util/background/mod.rs | 13 ++++++++++++- src/util/background/worker.rs | 12 ++++++------ 2 files changed, 18 insertions(+), 7 deletions(-) (limited to 'src/util') diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index 619f5068..fd9258b8 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -29,13 +29,24 @@ pub struct BackgroundRunner { #[derive(Clone, Serialize, Deserialize, Debug)] pub struct WorkerInfo { pub name: String, - pub info: Option, + pub status: WorkerStatus, pub state: WorkerState, pub errors: usize, pub consecutive_errors: usize, pub last_error: Option<(String, u64)>, } +/// WorkerStatus is a struct returned by the worker with a bunch of canonical +/// fields to indicate their status to CLI users. All fields are optional. +#[derive(Clone, Serialize, Deserialize, Debug, Default)] +pub struct WorkerStatus { + pub tranquility: Option, + pub progress: Option, + pub queue_length: Option, + pub persistent_errors: Option, + pub freeform: Vec, +} + impl BackgroundRunner { /// Create a new BackgroundRunner pub fn new( diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index f5e3addb..7e9da7f8 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -10,7 +10,7 @@ use serde::{Deserialize, Serialize}; use tokio::select; use tokio::sync::{mpsc, watch}; -use crate::background::WorkerInfo; +use crate::background::{WorkerInfo, WorkerStatus}; use crate::error::Error; use crate::time::now_msec; @@ -26,7 +26,7 @@ impl std::fmt::Display for WorkerState { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { WorkerState::Busy => write!(f, "Busy"), - WorkerState::Throttled(t) => write!(f, "Thr:{:.3}", t), + WorkerState::Throttled(_) => write!(f, "Busy*"), WorkerState::Idle => write!(f, "Idle"), WorkerState::Done => write!(f, "Done"), } @@ -37,8 +37,8 @@ impl std::fmt::Display for WorkerState { pub trait Worker: Send { fn name(&self) -> String; - fn info(&self) -> Option { - None + fn status(&self) -> WorkerStatus { + Default::default() } /// Work: do a basic unit of work, if one is available (otherwise, should return @@ -119,7 +119,7 @@ impl WorkerProcessor { match wi.get_mut(&worker.task_id) { Some(i) => { i.state = worker.state; - i.info = worker.worker.info(); + i.status = worker.worker.status(); i.errors = worker.errors; i.consecutive_errors = worker.consecutive_errors; if worker.last_error.is_some() { @@ -130,7 +130,7 @@ impl WorkerProcessor { wi.insert(worker.task_id, WorkerInfo { name: worker.worker.name(), state: worker.state, - info: worker.worker.info(), + status: worker.worker.status(), errors: worker.errors, consecutive_errors: worker.consecutive_errors, last_error: worker.last_error.take(), -- cgit v1.2.3 From d6040e32a610a792d1e5365a7643eb99fbb5a217 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 13 Dec 2022 15:43:22 +0100 Subject: cli: prettier table in garage stats --- src/util/formater.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) (limited to 'src/util') diff --git a/src/util/formater.rs b/src/util/formater.rs index 95324f9a..2ea53ebb 100644 --- a/src/util/formater.rs +++ b/src/util/formater.rs @@ -1,4 +1,4 @@ -pub fn format_table(data: Vec) { +pub fn format_table_to_string(data: Vec) -> String { let data = data .iter() .map(|s| s.split('\t').collect::>()) @@ -24,5 +24,9 @@ pub fn format_table(data: Vec) { out.push('\n'); } - print!("{}", out); + out +} + +pub fn format_table(data: Vec) { + print!("{}", format_table_to_string(data)); } -- cgit v1.2.3 From d56c472712df7c064387429a5af73d3bc0eb438d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 14 Dec 2022 12:51:16 +0100 Subject: Refactor background runner and get rid of job worker --- src/util/background/job_worker.rs | 48 ----------------------------- src/util/background/mod.rs | 64 ++++++++------------------------------- 2 files changed, 13 insertions(+), 99 deletions(-) delete mode 100644 src/util/background/job_worker.rs (limited to 'src/util') diff --git a/src/util/background/job_worker.rs b/src/util/background/job_worker.rs deleted file mode 100644 index 2568ea11..00000000 --- a/src/util/background/job_worker.rs +++ /dev/null @@ -1,48 +0,0 @@ -//! Job worker: a generic worker that just processes incoming -//! jobs one by one - -use std::sync::Arc; - -use async_trait::async_trait; -use tokio::sync::{mpsc, Mutex}; - -use crate::background::worker::*; -use crate::background::*; - -pub(crate) struct JobWorker { - pub(crate) index: usize, - pub(crate) job_chan: Arc>>, - pub(crate) next_job: Option, -} - -#[async_trait] -impl Worker for JobWorker { - fn name(&self) -> String { - format!("Job worker #{}", self.index) - } - - async fn work(&mut self, _must_exit: &mut watch::Receiver) -> Result { - match self.next_job.take() { - None => return Ok(WorkerState::Idle), - Some(job) => { - job.await?; - Ok(WorkerState::Busy) - } - } - } - - async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerState { - loop { - match self.job_chan.lock().await.recv().await { - Some((job, cancellable)) => { - if cancellable && *must_exit.borrow() { - continue; - } - self.next_job = Some(job); - return WorkerState::Busy; - } - None => return WorkerState::Done, - } - } - } -} diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index fd9258b8..0bb4fb67 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -1,27 +1,23 @@ //! Job runner for futures and async functions -pub mod job_worker; pub mod worker; use core::future::Future; use std::collections::HashMap; -use std::pin::Pin; use std::sync::Arc; use serde::{Deserialize, Serialize}; -use tokio::sync::{mpsc, watch, Mutex}; +use tokio::sync::{mpsc, watch}; use crate::error::Error; use worker::WorkerProcessor; pub use worker::{Worker, WorkerState}; pub(crate) type JobOutput = Result<(), Error>; -pub(crate) type Job = Pin + Send>>; /// Job runner for futures and async functions pub struct BackgroundRunner { - send_job: mpsc::UnboundedSender<(Job, bool)>, send_worker: mpsc::UnboundedSender>, worker_info: Arc>>, } @@ -49,10 +45,7 @@ pub struct WorkerStatus { impl BackgroundRunner { /// Create a new BackgroundRunner - pub fn new( - n_runners: usize, - stop_signal: watch::Receiver, - ) -> (Arc, tokio::task::JoinHandle<()>) { + pub fn new(stop_signal: watch::Receiver) -> (Arc, tokio::task::JoinHandle<()>) { let (send_worker, worker_out) = mpsc::unbounded_channel::>(); let worker_info = Arc::new(std::sync::Mutex::new(HashMap::new())); @@ -63,24 +56,7 @@ impl BackgroundRunner { worker_processor.run().await; }); - let (send_job, queue_out) = mpsc::unbounded_channel(); - let queue_out = Arc::new(Mutex::new(queue_out)); - - for i in 0..n_runners { - let queue_out = queue_out.clone(); - - send_worker - .send(Box::new(job_worker::JobWorker { - index: i, - job_chan: queue_out.clone(), - next_job: None, - })) - .ok() - .unwrap(); - } - let bgrunner = Arc::new(Self { - send_job, send_worker, worker_info, }); @@ -91,31 +67,6 @@ impl BackgroundRunner { self.worker_info.lock().unwrap().clone() } - /// Spawn a task to be run in background - pub fn spawn(&self, job: T) - where - T: Future + Send + 'static, - { - let boxed: Job = Box::pin(job); - self.send_job - .send((boxed, false)) - .ok() - .expect("Could not put job in queue"); - } - - /// Spawn a task to be run in background. It may get discarded before running if spawned while - /// the runner is stopping - pub fn spawn_cancellable(&self, job: T) - where - T: Future + Send + 'static, - { - let boxed: Job = Box::pin(job); - self.send_job - .send((boxed, true)) - .ok() - .expect("Could not put job in queue"); - } - pub fn spawn_worker(&self, worker: W) where W: Worker + 'static, @@ -126,3 +77,14 @@ impl BackgroundRunner { .expect("Could not put worker in queue"); } } + +pub fn spawn(job: T) +where + T: Future + Send + 'static, +{ + tokio::spawn(async move { + if let Err(e) = job.await { + error!("{}", e); + } + }); +} -- cgit v1.2.3 From dfc131850a09e7ceacfa98315adbef156e07e9ca Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 14 Dec 2022 15:25:29 +0100 Subject: Simplified and more aggressive worker exit logic --- src/util/background/worker.rs | 73 ++++++++++++++----------------------------- 1 file changed, 23 insertions(+), 50 deletions(-) (limited to 'src/util') diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs index 7e9da7f8..8165e2cb 100644 --- a/src/util/background/worker.rs +++ b/src/util/background/worker.rs @@ -1,6 +1,6 @@ use std::collections::HashMap; use std::sync::Arc; -use std::time::{Duration, Instant}; +use std::time::Duration; use async_trait::async_trait; use futures::future::*; @@ -14,6 +14,10 @@ use crate::background::{WorkerInfo, WorkerStatus}; use crate::error::Error; use crate::time::now_msec; +// All workers that haven't exited for this time after an exit signal was recieved +// will be interrupted in the middle of whatever they are doing. +const EXIT_DEADLINE: Duration = Duration::from_secs(8); + #[derive(PartialEq, Copy, Clone, Serialize, Deserialize, Debug)] pub enum WorkerState { Busy, @@ -50,10 +54,8 @@ pub trait Worker: Send { async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result; /// Wait for work: await for some task to become available. This future can be interrupted in - /// the middle for any reason. This future doesn't have to await on must_exit.changed(), we - /// are doing it for you. Therefore it only receives a read refernce to must_exit which allows - /// it to check if we are exiting. - async fn wait_for_work(&mut self, must_exit: &watch::Receiver) -> WorkerState; + /// the middle for any reason, for example if an interrupt signal was recieved. + async fn wait_for_work(&mut self) -> WorkerState; } pub(crate) struct WorkerProcessor { @@ -93,11 +95,9 @@ impl WorkerProcessor { let task_id = next_task_id; next_task_id += 1; let stop_signal = self.stop_signal.clone(); - let stop_signal_worker = self.stop_signal.clone(); let mut worker = WorkerHandler { task_id, stop_signal, - stop_signal_worker, worker: new_worker, state: WorkerState::Busy, errors: 0, @@ -153,26 +153,14 @@ impl WorkerProcessor { } // We are exiting, drain everything - let drain_half_time = Instant::now() + Duration::from_secs(5); let drain_everything = async move { - while let Some(mut worker) = workers.next().await { - if worker.state == WorkerState::Done { - info!( - "Worker {} (TID {}) exited", - worker.worker.name(), - worker.task_id - ); - } else if Instant::now() > drain_half_time { - warn!("Worker {} (TID {}) interrupted between two iterations in state {:?} (this should be fine)", worker.worker.name(), worker.task_id, worker.state); - } else { - workers.push( - async move { - worker.step().await; - worker - } - .boxed(), - ); - } + while let Some(worker) = workers.next().await { + info!( + "Worker {} (TID {}) exited (last state: {:?})", + worker.worker.name(), + worker.task_id, + worker.state + ); } }; @@ -180,7 +168,7 @@ impl WorkerProcessor { _ = drain_everything => { info!("All workers exited peacefully \\o/"); } - _ = tokio::time::sleep(Duration::from_secs(9)) => { + _ = tokio::time::sleep(EXIT_DEADLINE) => { error!("Some workers could not exit in time, we are cancelling some things in the middle"); } } @@ -190,7 +178,6 @@ impl WorkerProcessor { struct WorkerHandler { task_id: usize, stop_signal: watch::Receiver, - stop_signal_worker: watch::Receiver, worker: Box, state: WorkerState, errors: usize, @@ -225,33 +212,19 @@ impl WorkerHandler { }, WorkerState::Throttled(delay) => { // Sleep for given delay and go back to busy state - if !*self.stop_signal.borrow() { - select! { - _ = tokio::time::sleep(Duration::from_secs_f32(delay)) => (), - _ = self.stop_signal.changed() => (), + select! { + _ = tokio::time::sleep(Duration::from_secs_f32(delay)) => { + self.state = WorkerState::Busy; } + _ = self.stop_signal.changed() => (), } - self.state = WorkerState::Busy; } WorkerState::Idle => { - if *self.stop_signal.borrow() { - select! { - new_st = self.worker.wait_for_work(&self.stop_signal_worker) => { - self.state = new_st; - } - _ = tokio::time::sleep(Duration::from_secs(1)) => { - // stay in Idle state - } - } - } else { - select! { - new_st = self.worker.wait_for_work(&self.stop_signal_worker) => { - self.state = new_st; - } - _ = self.stop_signal.changed() => { - // stay in Idle state - } + select! { + new_st = self.worker.wait_for_work() => { + self.state = new_st; } + _ = self.stop_signal.changed() => (), } } WorkerState::Done => unreachable!(), -- cgit v1.2.3 From 510b62010871e9133a98f625b85f07a7e50f6f23 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 14 Dec 2022 16:08:05 +0100 Subject: Get rid of background::spawn --- src/util/background/mod.rs | 16 ---------------- 1 file changed, 16 deletions(-) (limited to 'src/util') diff --git a/src/util/background/mod.rs b/src/util/background/mod.rs index 0bb4fb67..41b48e93 100644 --- a/src/util/background/mod.rs +++ b/src/util/background/mod.rs @@ -2,20 +2,15 @@ pub mod worker; -use core::future::Future; - use std::collections::HashMap; use std::sync::Arc; use serde::{Deserialize, Serialize}; use tokio::sync::{mpsc, watch}; -use crate::error::Error; use worker::WorkerProcessor; pub use worker::{Worker, WorkerState}; -pub(crate) type JobOutput = Result<(), Error>; - /// Job runner for futures and async functions pub struct BackgroundRunner { send_worker: mpsc::UnboundedSender>, @@ -77,14 +72,3 @@ impl BackgroundRunner { .expect("Could not put worker in queue"); } } - -pub fn spawn(job: T) -where - T: Future + Send + 'static, -{ - tokio::spawn(async move { - if let Err(e) = job.await { - error!("{}", e); - } - }); -} -- cgit v1.2.3 From 67755695254fb20bcb535d3484d692babb853d33 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 2 Jan 2023 14:15:33 +0100 Subject: Bump everything to v0.8.1 --- src/util/Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'src/util') diff --git a/src/util/Cargo.toml b/src/util/Cargo.toml index 8e978fc2..11640027 100644 --- a/src/util/Cargo.toml +++ b/src/util/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "garage_util" -version = "0.8.0" +version = "0.8.1" authors = ["Alex Auvolat "] edition = "2018" license = "AGPL-3.0" @@ -14,7 +14,7 @@ path = "lib.rs" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -garage_db = { version = "0.8.0", path = "../db" } +garage_db = { version = "0.8.1", path = "../db" } arc-swap = "1.0" async-trait = "0.1" -- cgit v1.2.3 From cdb2a591e9d393d24ab5c49bb905b0589b193299 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 3 Jan 2023 14:44:47 +0100 Subject: Refactor how things are migrated --- src/util/Cargo.toml | 1 + src/util/lib.rs | 1 + src/util/migrate.rs | 75 +++++++++++++++++++++++++++++++++++++++++++++++++++ src/util/persister.rs | 38 +++++++++++++++++--------- 4 files changed, 103 insertions(+), 12 deletions(-) create mode 100644 src/util/migrate.rs (limited to 'src/util') diff --git a/src/util/Cargo.toml b/src/util/Cargo.toml index 11640027..32e9c851 100644 --- a/src/util/Cargo.toml +++ b/src/util/Cargo.toml @@ -23,6 +23,7 @@ bytes = "1.0" digest = "0.10" err-derive = "0.3" git-version = "0.3.4" +hexdump = "0.1" xxhash-rust = { version = "0.8", default-features = false, features = ["xxh3"] } hex = "0.4" lazy_static = "1.4" diff --git a/src/util/lib.rs b/src/util/lib.rs index 264cc192..fd3d5c7b 100644 --- a/src/util/lib.rs +++ b/src/util/lib.rs @@ -11,6 +11,7 @@ pub mod data; pub mod error; pub mod formater; pub mod metrics; +pub mod migrate; pub mod persister; pub mod time; pub mod token_bucket; diff --git a/src/util/migrate.rs b/src/util/migrate.rs new file mode 100644 index 00000000..199c68f6 --- /dev/null +++ b/src/util/migrate.rs @@ -0,0 +1,75 @@ +use serde::{Deserialize, Serialize}; + +pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { + /// A sequence of bytes to add at the beginning of the serialized + /// string, to identify that the data is of this version. + const MARKER: &'static [u8] = b""; + + /// The previous version of this data type, from which items of this version + /// can be migrated. Set `type Previous = NoPrevious` to indicate that this datatype + /// is the initial schema and cannot be migrated. + type Previous: Migrate; + + /// This function must be filled in by implementors to migrate from a previons iteration + /// of the data format. + fn migrate(previous: Self::Previous) -> Self; + + fn decode(bytes: &[u8]) -> Option { + if bytes.len() >= Self::MARKER.len() && &bytes[..Self::MARKER.len()] == Self::MARKER { + if let Ok(value) = + rmp_serde::decode::from_read_ref::<_, Self>(&bytes[Self::MARKER.len()..]) + { + return Some(value); + } + } + + Self::Previous::decode(bytes).map(Self::migrate) + } + + fn encode(&self) -> Result, rmp_serde::encode::Error> { + let mut wr = Vec::with_capacity(128); + wr.extend_from_slice(Self::MARKER); + let mut se = rmp_serde::Serializer::new(&mut wr) + .with_struct_map() + .with_string_variants(); + self.serialize(&mut se)?; + Ok(wr) + } +} + +pub trait InitialFormat: Serialize + for<'de> Deserialize<'de> + 'static { + /// A sequence of bytes to add at the beginning of the serialized + /// string, to identify that the data is of this version. + const MARKER: &'static [u8] = b""; +} + +// ---- + +impl Migrate for T { + const MARKER: &'static [u8] = ::MARKER; + + type Previous = NoPrevious; + + fn migrate(_previous: Self::Previous) -> Self { + unreachable!(); + } +} + +#[derive(Serialize, Deserialize)] +pub struct NoPrevious; + +impl Migrate for NoPrevious { + type Previous = NoPrevious; + + fn migrate(_previous: Self::Previous) -> Self { + unreachable!(); + } + + fn decode(_bytes: &[u8]) -> Option { + None + } + + fn encode(&self) -> Result, rmp_serde::encode::Error> { + unreachable!() + } +} diff --git a/src/util/persister.rs b/src/util/persister.rs index 9e1a1910..4b9adf51 100644 --- a/src/util/persister.rs +++ b/src/util/persister.rs @@ -3,21 +3,16 @@ use std::path::{Path, PathBuf}; use tokio::io::{AsyncReadExt, AsyncWriteExt}; -use serde::{Deserialize, Serialize}; - -use crate::data::*; use crate::error::Error; +use crate::migrate::Migrate; -pub struct Persister Deserialize<'de>> { +pub struct Persister { path: PathBuf, _marker: std::marker::PhantomData, } -impl Persister -where - T: Serialize + for<'de> Deserialize<'de>, -{ +impl Persister { pub fn new(base_dir: &Path, file_name: &str) -> Self { let mut path = base_dir.to_path_buf(); path.push(file_name); @@ -27,18 +22,37 @@ where } } + fn decode(&self, bytes: &[u8]) -> Result { + match T::decode(bytes) { + Some(v) => Ok(v), + None => { + error!( + "Unable to decode persisted data file {}", + self.path.display() + ); + for line in hexdump::hexdump_iter(bytes) { + debug!("{}", line); + } + Err(Error::Message(format!( + "Unable to decode persisted data file {}", + self.path.display() + ))) + } + } + } + pub fn load(&self) -> Result { let mut file = std::fs::OpenOptions::new().read(true).open(&self.path)?; let mut bytes = vec![]; file.read_to_end(&mut bytes)?; - let value = rmp_serde::decode::from_read_ref(&bytes[..])?; + let value = self.decode(&bytes[..])?; Ok(value) } pub fn save(&self, t: &T) -> Result<(), Error> { - let bytes = rmp_to_vec_all_named(t)?; + let bytes = t.encode()?; let mut file = std::fs::OpenOptions::new() .write(true) @@ -57,12 +71,12 @@ where let mut bytes = vec![]; file.read_to_end(&mut bytes).await?; - let value = rmp_serde::decode::from_read_ref(&bytes[..])?; + let value = self.decode(&bytes[..])?; Ok(value) } pub async fn save_async(&self, t: &T) -> Result<(), Error> { - let bytes = rmp_to_vec_all_named(t)?; + let bytes = t.encode()?; let mut file = tokio::fs::File::create(&self.path).await?; file.write_all(&bytes[..]).await?; -- cgit v1.2.3 From 8d5505514f950dc1ca1249a3385c9913b5b5e8e0 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 3 Jan 2023 15:27:36 +0100 Subject: Make it explicit when using nonversioned encoding --- src/util/data.rs | 15 --------------- src/util/encode.rs | 26 ++++++++++++++++++++++++++ src/util/lib.rs | 1 + src/util/migrate.rs | 15 +++++++-------- 4 files changed, 34 insertions(+), 23 deletions(-) create mode 100644 src/util/encode.rs (limited to 'src/util') diff --git a/src/util/data.rs b/src/util/data.rs index 7715c2cc..b2a52e25 100644 --- a/src/util/data.rs +++ b/src/util/data.rs @@ -141,21 +141,6 @@ pub fn gen_uuid() -> Uuid { rand::thread_rng().gen::<[u8; 32]>().into() } -// RMP serialization with names of fields and variants - -/// Serialize to MessagePack -pub fn rmp_to_vec_all_named(val: &T) -> Result, rmp_serde::encode::Error> -where - T: Serialize + ?Sized, -{ - let mut wr = Vec::with_capacity(128); - let mut se = rmp_serde::Serializer::new(&mut wr) - .with_struct_map() - .with_string_variants(); - val.serialize(&mut se)?; - Ok(wr) -} - /// Serialize to JSON, truncating long result pub fn debug_serialize(x: T) -> String { match serde_json::to_string(&x) { diff --git a/src/util/encode.rs b/src/util/encode.rs new file mode 100644 index 00000000..724e482a --- /dev/null +++ b/src/util/encode.rs @@ -0,0 +1,26 @@ +use serde::{Deserialize, Serialize}; + +/// Serialize to MessagePacki, without versionning +/// (see garage_util::migrate for functions that manage versionned +/// data formats) +pub fn nonversioned_encode(val: &T) -> Result, rmp_serde::encode::Error> +where + T: Serialize + ?Sized, +{ + let mut wr = Vec::with_capacity(128); + let mut se = rmp_serde::Serializer::new(&mut wr) + .with_struct_map() + .with_string_variants(); + val.serialize(&mut se)?; + Ok(wr) +} + +/// Deserialize from MessagePacki, without versionning +/// (see garage_util::migrate for functions that manage versionned +/// data formats) +pub fn nonversioned_decode(bytes: &[u8]) -> Result +where + T: for<'de> Deserialize<'de> + ?Sized, +{ + rmp_serde::decode::from_read_ref::<_, T>(bytes) +} diff --git a/src/util/lib.rs b/src/util/lib.rs index fd3d5c7b..be82061f 100644 --- a/src/util/lib.rs +++ b/src/util/lib.rs @@ -8,6 +8,7 @@ pub mod background; pub mod config; pub mod crdt; pub mod data; +pub mod encode; pub mod error; pub mod formater; pub mod metrics; diff --git a/src/util/migrate.rs b/src/util/migrate.rs index 199c68f6..f6028bf4 100644 --- a/src/util/migrate.rs +++ b/src/util/migrate.rs @@ -3,7 +3,7 @@ use serde::{Deserialize, Serialize}; pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { /// A sequence of bytes to add at the beginning of the serialized /// string, to identify that the data is of this version. - const MARKER: &'static [u8] = b""; + const VERSION_MARKER: &'static [u8] = b""; /// The previous version of this data type, from which items of this version /// can be migrated. Set `type Previous = NoPrevious` to indicate that this datatype @@ -15,10 +15,9 @@ pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { fn migrate(previous: Self::Previous) -> Self; fn decode(bytes: &[u8]) -> Option { - if bytes.len() >= Self::MARKER.len() && &bytes[..Self::MARKER.len()] == Self::MARKER { - if let Ok(value) = - rmp_serde::decode::from_read_ref::<_, Self>(&bytes[Self::MARKER.len()..]) - { + let marker_len = Self::VERSION_MARKER.len(); + if bytes.len() >= marker_len && &bytes[..marker_len] == Self::VERSION_MARKER { + if let Ok(value) = rmp_serde::decode::from_read_ref::<_, Self>(&bytes[marker_len..]) { return Some(value); } } @@ -28,7 +27,7 @@ pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { fn encode(&self) -> Result, rmp_serde::encode::Error> { let mut wr = Vec::with_capacity(128); - wr.extend_from_slice(Self::MARKER); + wr.extend_from_slice(Self::VERSION_MARKER); let mut se = rmp_serde::Serializer::new(&mut wr) .with_struct_map() .with_string_variants(); @@ -40,13 +39,13 @@ pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { pub trait InitialFormat: Serialize + for<'de> Deserialize<'de> + 'static { /// A sequence of bytes to add at the beginning of the serialized /// string, to identify that the data is of this version. - const MARKER: &'static [u8] = b""; + const VERSION_MARKER: &'static [u8] = b""; } // ---- impl Migrate for T { - const MARKER: &'static [u8] = ::MARKER; + const VERSION_MARKER: &'static [u8] = ::VERSION_MARKER; type Previous = NoPrevious; -- cgit v1.2.3 From a54b67740d08e3fabeb1652a1bed14d78fea4b74 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 3 Jan 2023 15:29:29 +0100 Subject: move debug_serialize to garage_util::encode --- src/util/data.rs | 16 ---------------- src/util/encode.rs | 16 ++++++++++++++++ src/util/error.rs | 1 + 3 files changed, 17 insertions(+), 16 deletions(-) (limited to 'src/util') diff --git a/src/util/data.rs b/src/util/data.rs index b2a52e25..3f61e301 100644 --- a/src/util/data.rs +++ b/src/util/data.rs @@ -140,19 +140,3 @@ pub fn fasthash(data: &[u8]) -> FastHash { pub fn gen_uuid() -> Uuid { rand::thread_rng().gen::<[u8; 32]>().into() } - -/// Serialize to JSON, truncating long result -pub fn debug_serialize(x: T) -> String { - match serde_json::to_string(&x) { - Ok(ss) => { - if ss.len() > 100 { - // TODO this can panic if 100 is not a codepoint boundary, but inside a 2 Bytes - // (or more) codepoint - ss[..100].to_string() - } else { - ss - } - } - Err(e) => format!("", e), - } -} diff --git a/src/util/encode.rs b/src/util/encode.rs index 724e482a..1cd3198f 100644 --- a/src/util/encode.rs +++ b/src/util/encode.rs @@ -24,3 +24,19 @@ where { rmp_serde::decode::from_read_ref::<_, T>(bytes) } + +/// Serialize to JSON, truncating long result +pub fn debug_serialize(x: T) -> String { + match serde_json::to_string(&x) { + Ok(ss) => { + if ss.len() > 100 { + // TODO this can panic if 100 is not a codepoint boundary, but inside a 2 Bytes + // (or more) codepoint + ss[..100].to_string() + } else { + ss + } + } + Err(e) => format!("", e), + } +} diff --git a/src/util/error.rs b/src/util/error.rs index 9995c746..3fcee71d 100644 --- a/src/util/error.rs +++ b/src/util/error.rs @@ -7,6 +7,7 @@ use err_derive::Error; use serde::{de::Visitor, Deserialize, Deserializer, Serialize, Serializer}; use crate::data::*; +use crate::encode::debug_serialize; /// Regroup all Garage errors #[derive(Debug, Error)] -- cgit v1.2.3 From 33f25d26c7a81f7dc7ae3ab4fd2faa49fb053ceb Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 3 Jan 2023 15:53:13 +0100 Subject: fix doc and add tests for migrate.rs --- src/util/migrate.rs | 97 +++++++++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 91 insertions(+), 6 deletions(-) (limited to 'src/util') diff --git a/src/util/migrate.rs b/src/util/migrate.rs index f6028bf4..b7d6edc1 100644 --- a/src/util/migrate.rs +++ b/src/util/migrate.rs @@ -1,19 +1,21 @@ use serde::{Deserialize, Serialize}; +/// Indicates that this type has an encoding that can be migrated from +/// a previous version upon upgrades of Garage. pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { /// A sequence of bytes to add at the beginning of the serialized /// string, to identify that the data is of this version. const VERSION_MARKER: &'static [u8] = b""; /// The previous version of this data type, from which items of this version - /// can be migrated. Set `type Previous = NoPrevious` to indicate that this datatype - /// is the initial schema and cannot be migrated. + /// can be migrated. type Previous: Migrate; - /// This function must be filled in by implementors to migrate from a previons iteration - /// of the data format. + /// The migration function that transforms a value decoded in the old format + /// to an up-to-date value. fn migrate(previous: Self::Previous) -> Self; + /// Decode an encoded version of this type, going through a migration if necessary. fn decode(bytes: &[u8]) -> Option { let marker_len = Self::VERSION_MARKER.len(); if bytes.len() >= marker_len && &bytes[..marker_len] == Self::VERSION_MARKER { @@ -25,6 +27,7 @@ pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { Self::Previous::decode(bytes).map(Self::migrate) } + /// Encode this type with optionnal version marker fn encode(&self) -> Result, rmp_serde::encode::Error> { let mut wr = Vec::with_capacity(128); wr.extend_from_slice(Self::VERSION_MARKER); @@ -36,14 +39,13 @@ pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { } } +/// Indicates that this type has no previous encoding version to be migrated from. pub trait InitialFormat: Serialize + for<'de> Deserialize<'de> + 'static { /// A sequence of bytes to add at the beginning of the serialized /// string, to identify that the data is of this version. const VERSION_MARKER: &'static [u8] = b""; } -// ---- - impl Migrate for T { const VERSION_MARKER: &'static [u8] = ::VERSION_MARKER; @@ -54,6 +56,7 @@ impl Migrate for T { } } +/// Internal type used by InitialFormat, not meant for general use. #[derive(Serialize, Deserialize)] pub struct NoPrevious; @@ -72,3 +75,85 @@ impl Migrate for NoPrevious { unreachable!() } } + +#[cfg(test)] +mod test { + use super::*; + + #[derive(Serialize, Deserialize, PartialEq, Eq, Debug)] + struct V1 { + a: usize, + b: String, + } + impl InitialFormat for V1 {} + + #[derive(Serialize, Deserialize, PartialEq, Eq, Debug)] + struct V2 { + a: usize, + b: Vec, + c: String, + } + impl Migrate for V2 { + const VERSION_MARKER: &'static [u8] = b"GtestV2"; + type Previous = V1; + fn migrate(prev: V1) -> V2 { + V2 { + a: prev.a, + b: vec![prev.b], + c: String::new(), + } + } + } + + #[test] + fn test_v1() { + let x = V1 { + a: 12, + b: "hello".into(), + }; + let x_enc = x.encode().unwrap(); + let y = V1::decode(&x_enc).unwrap(); + assert_eq!(x, y); + } + + #[test] + fn test_v2() { + let x = V2 { + a: 12, + b: vec!["hello".into(), "world".into()], + c: "plop".into(), + }; + let x_enc = x.encode().unwrap(); + assert_eq!(&x_enc[..V2::VERSION_MARKER.len()], V2::VERSION_MARKER); + let y = V2::decode(&x_enc).unwrap(); + assert_eq!(x, y); + } + + #[test] + fn test_migrate() { + let x = V1 { + a: 12, + b: "hello".into(), + }; + let x_enc = x.encode().unwrap(); + + let xx = V1::decode(&x_enc).unwrap(); + assert_eq!(x, xx); + + let y = V2::decode(&x_enc).unwrap(); + assert_eq!( + y, + V2 { + a: 12, + b: vec!["hello".into()], + c: "".into(), + } + ); + + let y_enc = y.encode().unwrap(); + assert_eq!(&y_enc[..V2::VERSION_MARKER.len()], V2::VERSION_MARKER); + + let z = V2::decode(&y_enc).unwrap(); + assert_eq!(y, z); + } +} -- cgit v1.2.3 From c106304b9cd325238742be4366877ed7316e7e28 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 3 Jan 2023 16:00:19 +0100 Subject: more idiomatic and shorter --- src/util/migrate.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'src/util') diff --git a/src/util/migrate.rs b/src/util/migrate.rs index b7d6edc1..00d81181 100644 --- a/src/util/migrate.rs +++ b/src/util/migrate.rs @@ -18,7 +18,7 @@ pub trait Migrate: Serialize + for<'de> Deserialize<'de> + 'static { /// Decode an encoded version of this type, going through a migration if necessary. fn decode(bytes: &[u8]) -> Option { let marker_len = Self::VERSION_MARKER.len(); - if bytes.len() >= marker_len && &bytes[..marker_len] == Self::VERSION_MARKER { + if bytes.get(..marker_len) == Some(Self::VERSION_MARKER) { if let Ok(value) = rmp_serde::decode::from_read_ref::<_, Self>(&bytes[marker_len..]) { return Some(value); } -- cgit v1.2.3 From 1d5bdc17a46648eb3494ff629d0d360d0217c1e2 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 3 Jan 2023 16:04:06 +0100 Subject: use impossible enum type --- src/util/migrate.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'src/util') diff --git a/src/util/migrate.rs b/src/util/migrate.rs index 00d81181..1229fd9c 100644 --- a/src/util/migrate.rs +++ b/src/util/migrate.rs @@ -58,7 +58,7 @@ impl Migrate for T { /// Internal type used by InitialFormat, not meant for general use. #[derive(Serialize, Deserialize)] -pub struct NoPrevious; +pub enum NoPrevious {} impl Migrate for NoPrevious { type Previous = NoPrevious; -- cgit v1.2.3