Background task runner that replaces tokio::spawn

This commit is contained in:
Alex 2020-04-11 18:51:11 +02:00
parent 4a2624b76a
commit 53289b69e5
8 changed files with 291 additions and 62 deletions

View File

@ -23,7 +23,7 @@ type BodyType = Box<dyn HttpBody<Data = Bytes, Error = Error> + Send + Unpin>;
pub async fn run_api_server(
garage: Arc<Garage>,
shutdown_signal: impl Future<Output = ()>,
) -> Result<(), hyper::Error> {
) -> Result<(), Error> {
let addr = ([0, 0, 0, 0], garage.system.config.api_port).into();
let service = make_service_fn(|conn: &AddrStream| {
@ -42,7 +42,8 @@ pub async fn run_api_server(
let graceful = server.with_graceful_shutdown(shutdown_signal);
println!("API server listening on http://{}", addr);
graceful.await
graceful.await?;
Ok(())
}
async fn handler(
@ -92,6 +93,13 @@ async fn handler_inner(
version_uuid
)))))
}
&Method::DELETE => {
let version_uuid = handle_delete(garage, &bucket, &key).await?;
Ok(Response::new(Box::new(BytesBody::from(format!(
"{:?}\n",
version_uuid
)))))
}
_ => Err(Error::BadRequest(format!("Invalid method"))),
}
}
@ -257,6 +265,27 @@ impl BodyChunker {
}
}
async fn handle_delete(garage: Arc<Garage>, bucket: &str, key: &str) -> Result<UUID, Error> {
let version_uuid = gen_uuid();
let mut object = Object {
bucket: bucket.into(),
key: key.into(),
versions: Vec::new(),
};
object.versions.push(Box::new(ObjectVersion {
uuid: version_uuid.clone(),
timestamp: now_msec(),
mime_type: "application/x-delete-marker".into(),
size: 0,
is_complete: true,
data: ObjectVersionData::DeleteMarker,
}));
garage.object_table.insert(&object).await?;
return Ok(version_uuid);
}
async fn handle_get(
garage: Arc<Garage>,
bucket: &str,

113
src/background.rs Normal file
View File

@ -0,0 +1,113 @@
use core::future::Future;
use std::pin::Pin;
use futures::future::join_all;
use std::sync::Arc;
use std::time::Duration;
use tokio::sync::Mutex;
use tokio::sync::{mpsc, watch};
use crate::error::Error;
type JobOutput = Result<(), Error>;
type Job = Pin<Box<dyn Future<Output = JobOutput> + Send>>;
pub struct BackgroundRunner {
n_runners: usize,
stop_signal: watch::Receiver<bool>,
queue_in: mpsc::UnboundedSender<(Job, bool)>,
queue_out: Mutex<mpsc::UnboundedReceiver<(Job, bool)>>,
workers: Mutex<Vec<tokio::task::JoinHandle<()>>>,
}
impl BackgroundRunner {
pub fn new(n_runners: usize, stop_signal: watch::Receiver<bool>) -> Arc<Self> {
let (queue_in, queue_out) = mpsc::unbounded_channel();
Arc::new(Self {
n_runners,
stop_signal,
queue_in,
queue_out: Mutex::new(queue_out),
workers: Mutex::new(Vec::new()),
})
}
pub async fn run(self: Arc<Self>) {
let mut workers = self.workers.lock().await;
for _i in 0..self.n_runners {
workers.push(tokio::spawn(self.clone().runner()));
}
drop(workers);
let mut stop_signal = self.stop_signal.clone();
while let Some(exit_now) = stop_signal.recv().await {
if exit_now {
let mut workers = self.workers.lock().await;
let workers_vec = workers.drain(..).collect::<Vec<_>>();
join_all(workers_vec).await;
return;
}
}
}
pub fn spawn<T>(&self, job: T)
where
T: Future<Output = JobOutput> + Send + 'static,
{
let boxed: Job = Box::pin(job);
let _: Result<_, _> = self.queue_in.clone().send((boxed, false));
}
pub fn spawn_cancellable<T>(&self, job: T)
where
T: Future<Output = JobOutput> + Send + 'static,
{
let boxed: Job = Box::pin(job);
let _: Result<_, _> = self.queue_in.clone().send((boxed, true));
}
pub async fn spawn_worker<F, T>(self: Arc<Self>, worker: F)
where
F: FnOnce(watch::Receiver<bool>) -> T + Send + 'static,
T: Future<Output = JobOutput> + Send + 'static,
{
let mut workers = self.workers.lock().await;
let stop_signal = self.stop_signal.clone();
workers.push(tokio::spawn(async move {
if let Err(e) = worker(stop_signal).await {
eprintln!("Worker stopped with error: {}", e);
}
}));
}
async fn runner(self: Arc<Self>) {
let stop_signal = self.stop_signal.clone();
loop {
let must_exit: bool = *stop_signal.borrow();
if let Some(job) = self.dequeue_job(must_exit).await {
if let Err(e) = job.await {
eprintln!("Job failed: {}", e)
}
} else {
if must_exit {
return;
}
tokio::time::delay_for(Duration::from_secs(1)).await;
}
}
}
async fn dequeue_job(&self, must_exit: bool) -> Option<Job> {
let mut queue = self.queue_out.lock().await;
while let Ok((job, cancellable)) = queue.try_recv() {
if cancellable && must_exit {
continue;
} else {
return Some(job);
}
}
None
}
}

View File

@ -63,3 +63,9 @@ impl From<sled::TransactionError<Error>> for Error {
}
}
}
impl<T> From<tokio::sync::watch::error::SendError<T>> for Error {
fn from(_e: tokio::sync::watch::error::SendError<T>) -> Error {
Error::Message(format!("Watch send error"))
}
}

View File

@ -2,6 +2,7 @@ mod data;
mod error;
mod proto;
mod background;
mod membership;
mod table;

View File

@ -8,10 +8,14 @@ use std::sync::Arc;
use std::time::Duration;
use futures::future::join_all;
use futures::select;
use futures_util::future::*;
use sha2::{Digest, Sha256};
use tokio::prelude::*;
use tokio::sync::watch;
use tokio::sync::RwLock;
use crate::background::BackgroundRunner;
use crate::data::*;
use crate::error::Error;
use crate::proto::*;
@ -29,6 +33,8 @@ pub struct System {
pub rpc_client: RpcClient,
pub members: RwLock<Members>,
pub background: Arc<BackgroundRunner>,
}
pub struct Members {
@ -181,7 +187,7 @@ fn read_network_config(metadata_dir: &PathBuf) -> Result<NetworkConfig, Error> {
}
impl System {
pub fn new(config: Config, id: UUID) -> Self {
pub fn new(config: Config, id: UUID, background: Arc<BackgroundRunner>) -> Self {
let net_config = match read_network_config(&config.metadata_dir) {
Ok(x) => x,
Err(e) => {
@ -209,24 +215,24 @@ impl System {
id,
rpc_client: RpcClient::new(),
members: RwLock::new(members),
background,
}
}
async fn save_network_config(self: Arc<Self>) {
async fn save_network_config(self: Arc<Self>) -> Result<(), Error> {
let mut path = self.config.metadata_dir.clone();
path.push("network_config");
let members = self.members.read().await;
let data =
rmp_to_vec_all_named(&members.config).expect("Error while encoding network config");
rmp_to_vec_all_named(&members.config)?;
drop(members);
let mut f = tokio::fs::File::create(path.as_path())
.await
.expect("Could not create network_config");
.await?;
f.write_all(&data[..])
.await
.expect("Could not write network_config");
.await?;
Ok(())
}
pub async fn make_ping(&self) -> Message {
@ -260,7 +266,10 @@ impl System {
.collect::<Vec<_>>();
self.clone().ping_nodes(bootstrap_peers).await;
tokio::spawn(self.ping_loop());
self.background
.clone()
.spawn_worker(|stop_signal| self.ping_loop(stop_signal).map(Ok))
.await;
}
pub async fn ping_nodes(self: Arc<Self>, peers: Vec<(SocketAddr, Option<UUID>)>) {
@ -294,10 +303,12 @@ impl System {
});
}
if is_new || members.status_hash != info.status_hash {
tokio::spawn(self.clone().pull_status(info.id.clone()));
self.background
.spawn_cancellable(self.clone().pull_status(info.id.clone()).map(Ok));
}
if is_new || members.config.version < info.config_version {
tokio::spawn(self.clone().pull_config(info.id.clone()));
self.background
.spawn_cancellable(self.clone().pull_config(info.id.clone()).map(Ok));
}
} else if let Some(id) = id_option {
let remaining_attempts = members
@ -345,10 +356,10 @@ impl System {
drop(members);
if is_new || status_hash != ping.status_hash {
tokio::spawn(self.clone().pull_status(ping.id.clone()));
self.background.spawn_cancellable(self.clone().pull_status(ping.id.clone()).map(Ok));
}
if is_new || config_version < ping.config_version {
tokio::spawn(self.clone().pull_config(ping.id.clone()));
self.background.spawn_cancellable(self.clone().pull_config(ping.id.clone()).map(Ok));
}
Ok(self.make_ping().await)
@ -405,7 +416,7 @@ impl System {
drop(members);
if to_ping.len() > 0 {
tokio::spawn(self.clone().ping_nodes(to_ping));
self.background.spawn_cancellable(self.clone().ping_nodes(to_ping).map(Ok));
}
Ok(Message::Ok)
@ -420,17 +431,18 @@ impl System {
members.config = adv.clone();
members.rebuild_ring();
tokio::spawn(
self.background.spawn_cancellable(
self.clone()
.broadcast(Message::AdvertiseConfig(adv.clone()), PING_TIMEOUT),
.broadcast(Message::AdvertiseConfig(adv.clone()), PING_TIMEOUT)
.map(Ok),
);
tokio::spawn(self.clone().save_network_config());
self.background.spawn(self.clone().save_network_config());
}
Ok(Message::Ok)
}
pub async fn ping_loop(self: Arc<Self>) {
pub async fn ping_loop(self: Arc<Self>, mut stop_signal: watch::Receiver<bool>) {
loop {
let restart_at = tokio::time::delay_for(PING_INTERVAL);
@ -445,7 +457,15 @@ impl System {
self.clone().ping_nodes(ping_addrs).await;
restart_at.await
select! {
_ = restart_at.fuse() => (),
must_exit = stop_signal.recv().fuse() => {
match must_exit {
None | Some(true) => return,
_ => (),
}
}
}
}
}

View File

@ -38,6 +38,12 @@ pub enum ObjectVersionData {
FirstBlock(Hash),
}
impl ObjectVersion {
fn cmp_key(&self) -> (u64, &UUID) {
(self.timestamp, &self.uuid)
}
}
impl Entry<String, String> for Object {
fn partition_key(&self) -> &String {
&self.bucket
@ -48,9 +54,10 @@ impl Entry<String, String> for Object {
fn merge(&mut self, other: &Self) {
for other_v in other.versions.iter() {
match self.versions.binary_search_by(|v| {
(v.timestamp, &v.uuid).cmp(&(other_v.timestamp, &other_v.uuid))
}) {
match self
.versions
.binary_search_by(|v| v.cmp_key().cmp(&other_v.cmp_key()))
{
Ok(i) => {
let mut v = &mut self.versions[i];
if other_v.size > v.size {
@ -91,7 +98,30 @@ impl TableFormat for ObjectTable {
type E = Object;
async fn updated(&self, old: Option<&Self::E>, new: &Self::E) {
//unimplemented!()
// TODO
let old = old.cloned();
let new = new.clone();
let garage = self.garage.read().await.as_ref().cloned().unwrap();
garage.clone().background.spawn(async move {
// Propagate deletion of old versions
if let Some(old_v) = old {
for v in old_v.versions.iter() {
if new
.versions
.binary_search_by(|nv| nv.cmp_key().cmp(&v.cmp_key()))
.is_err()
{
let deleted_version = Version {
uuid: v.uuid.clone(),
deleted: true,
blocks: vec![],
bucket: old_v.bucket.clone(),
key: old_v.key.clone(),
};
garage.version_table.insert(&deleted_version).await?;
}
}
}
Ok(())
});
}
}

View File

@ -90,7 +90,7 @@ async fn handler(
pub async fn run_rpc_server(
garage: Arc<Garage>,
shutdown_signal: impl Future<Output = ()>,
) -> Result<(), hyper::Error> {
) -> Result<(), Error> {
let bind_addr = ([0, 0, 0, 0], garage.system.config.rpc_port).into();
let service = make_service_fn(|conn: &AddrStream| {
@ -112,5 +112,6 @@ pub async fn run_rpc_server(
let graceful = server.with_graceful_shutdown(shutdown_signal);
println!("RPC server listening on http://{}", bind_addr);
graceful.await
graceful.await?;
Ok(())
}

View File

@ -1,13 +1,15 @@
use futures::channel::oneshot;
use futures_util::future::FutureExt;
use serde::Deserialize;
use std::collections::HashMap;
use std::io::{Read, Write};
use std::net::SocketAddr;
use std::path::PathBuf;
use std::sync::Arc;
use tokio::sync::watch;
use tokio::sync::{Mutex, RwLock};
use crate::api_server;
use crate::background::*;
use crate::data::*;
use crate::error::Error;
use crate::membership::System;
@ -15,10 +17,31 @@ use crate::proto::*;
use crate::rpc_server;
use crate::table::*;
#[derive(Deserialize, Debug)]
pub struct Config {
pub metadata_dir: PathBuf,
pub data_dir: PathBuf,
pub api_port: u16,
pub rpc_port: u16,
pub bootstrap_peers: Vec<SocketAddr>,
#[serde(default = "default_block_size")]
pub block_size: usize,
#[serde(default = "default_meta_replication_factor")]
pub meta_replication_factor: usize,
#[serde(default = "default_data_replication_factor")]
pub data_replication_factor: usize,
}
pub struct Garage {
pub db: sled::Db,
pub system: Arc<System>,
pub fs_lock: Mutex<()>,
pub background: Arc<BackgroundRunner>,
pub table_rpc_handlers: HashMap<String, Box<dyn TableRpcHandler + Sync + Send>>,
@ -28,8 +51,13 @@ pub struct Garage {
}
impl Garage {
pub async fn new(config: Config, id: UUID, db: sled::Db) -> Arc<Self> {
let system = Arc::new(System::new(config, id));
pub async fn new(
config: Config,
id: UUID,
db: sled::Db,
background: Arc<BackgroundRunner>,
) -> Arc<Self> {
let system = Arc::new(System::new(config, id, background.clone()));
let meta_rep_param = TableReplicationParams {
replication_factor: system.config.meta_replication_factor,
@ -56,6 +84,14 @@ impl Garage {
"version".to_string(),
meta_rep_param.clone(),
));
let data_rep_param = TableReplicationParams {
replication_factor: system.config.data_replication_factor,
write_quorum: (system.config.data_replication_factor + 1) / 2,
read_quorum: 1,
timeout: DEFAULT_TIMEOUT,
};
let block_ref_table = Arc::new(Table::new(
BlockRefTable {
garage: RwLock::new(None),
@ -63,13 +99,14 @@ impl Garage {
system.clone(),
&db,
"block_ref".to_string(),
meta_rep_param.clone(),
data_rep_param.clone(),
));
let mut garage = Self {
db,
system: system.clone(),
fs_lock: Mutex::new(()),
background,
table_rpc_handlers: HashMap::new(),
object_table,
version_table,
@ -105,22 +142,8 @@ fn default_block_size() -> usize {
fn default_meta_replication_factor() -> usize {
3
}
#[derive(Deserialize, Debug)]
pub struct Config {
pub metadata_dir: PathBuf,
pub data_dir: PathBuf,
pub api_port: u16,
pub rpc_port: u16,
pub bootstrap_peers: Vec<SocketAddr>,
#[serde(default = "default_block_size")]
pub block_size: usize,
#[serde(default = "default_meta_replication_factor")]
pub meta_replication_factor: usize,
fn default_data_replication_factor() -> usize {
3
}
fn read_config(config_file: PathBuf) -> Result<Config, Error> {
@ -157,19 +180,22 @@ fn gen_node_id(metadata_dir: &PathBuf) -> Result<UUID, Error> {
}
}
async fn shutdown_signal(chans: Vec<oneshot::Sender<()>>) {
async fn shutdown_signal(send_cancel: watch::Sender<bool>) -> Result<(), Error> {
// Wait for the CTRL+C signal
tokio::signal::ctrl_c()
.await
.expect("failed to install CTRL+C signal handler");
println!("Received CTRL+C, shutting down.");
for ch in chans {
ch.send(()).unwrap();
}
send_cancel.broadcast(true)?;
Ok(())
}
async fn wait_from(chan: oneshot::Receiver<()>) -> () {
chan.await.unwrap()
async fn wait_from(mut chan: watch::Receiver<bool>) -> () {
while let Some(exit_now) = chan.recv().await {
if exit_now {
return;
}
}
}
pub async fn run_server(config_file: PathBuf) -> Result<(), Error> {
@ -182,17 +208,20 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> {
let id = gen_node_id(&config.metadata_dir).expect("Unable to read or generate node ID");
println!("Node ID: {}", hex::encode(&id));
let garage = Garage::new(config, id, db).await;
let (send_cancel, watch_cancel) = watch::channel(false);
let (tx1, rx1) = oneshot::channel();
let (tx2, rx2) = oneshot::channel();
let background = BackgroundRunner::new(8, watch_cancel.clone());
let garage = Garage::new(config, id, db, background.clone()).await;
let rpc_server = rpc_server::run_rpc_server(garage.clone(), wait_from(rx1));
let api_server = api_server::run_api_server(garage.clone(), wait_from(rx2));
let rpc_server = rpc_server::run_rpc_server(garage.clone(), wait_from(watch_cancel.clone()));
let api_server = api_server::run_api_server(garage.clone(), wait_from(watch_cancel.clone()));
tokio::spawn(shutdown_signal(vec![tx1, tx2]));
tokio::spawn(garage.system.clone().bootstrap());
futures::try_join!(rpc_server, api_server)?;
futures::try_join!(
garage.system.clone().bootstrap().map(Ok),
rpc_server,
api_server,
background.run().map(Ok),
shutdown_signal(send_cancel),
)?;
Ok(())
}