aboutsummaryrefslogtreecommitdiff
path: root/src/util/background
diff options
context:
space:
mode:
Diffstat (limited to 'src/util/background')
-rw-r--r--src/util/background/job_worker.rs48
-rw-r--r--src/util/background/mod.rs72
-rw-r--r--src/util/background/vars.rs113
-rw-r--r--src/util/background/worker.rs85
4 files changed, 157 insertions, 161 deletions
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<Mutex<mpsc::UnboundedReceiver<(Job, bool)>>>,
- pub(crate) next_job: Option<Job>,
-}
-
-#[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<bool>) -> Result<WorkerState, Error> {
- 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<bool>) -> 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 619f5068..607cd7a3 100644
--- a/src/util/background/mod.rs
+++ b/src/util/background/mod.rs
@@ -1,27 +1,19 @@
//! Job runner for futures and async functions
-pub mod job_worker;
+pub mod vars;
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<Box<dyn Future<Output = JobOutput> + Send>>;
-
/// Job runner for futures and async functions
pub struct BackgroundRunner {
- send_job: mpsc::UnboundedSender<(Job, bool)>,
send_worker: mpsc::UnboundedSender<Box<dyn Worker>>,
worker_info: Arc<std::sync::Mutex<HashMap<usize, WorkerInfo>>>,
}
@@ -29,19 +21,27 @@ pub struct BackgroundRunner {
#[derive(Clone, Serialize, Deserialize, Debug)]
pub struct WorkerInfo {
pub name: String,
- pub info: Option<String>,
+ 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<u32>,
+ pub progress: Option<String>,
+ pub queue_length: Option<u64>,
+ pub persistent_errors: Option<u64>,
+ pub freeform: Vec<String>,
+}
+
impl BackgroundRunner {
/// Create a new BackgroundRunner
- pub fn new(
- n_runners: usize,
- stop_signal: watch::Receiver<bool>,
- ) -> (Arc<Self>, tokio::task::JoinHandle<()>) {
+ pub fn new(stop_signal: watch::Receiver<bool>) -> (Arc<Self>, tokio::task::JoinHandle<()>) {
let (send_worker, worker_out) = mpsc::unbounded_channel::<Box<dyn Worker>>();
let worker_info = Arc::new(std::sync::Mutex::new(HashMap::new()));
@@ -52,24 +52,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,
});
@@ -80,31 +63,6 @@ impl BackgroundRunner {
self.worker_info.lock().unwrap().clone()
}
- /// Spawn a task to be run in background
- pub fn spawn<T>(&self, job: T)
- where
- T: Future<Output = JobOutput> + 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<T>(&self, job: T)
- where
- T: Future<Output = JobOutput> + 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<W>(&self, worker: W)
where
W: Worker + 'static,
diff --git a/src/util/background/vars.rs b/src/util/background/vars.rs
new file mode 100644
index 00000000..7a449c95
--- /dev/null
+++ b/src/util/background/vars.rs
@@ -0,0 +1,113 @@
+use std::collections::HashMap;
+use std::str::FromStr;
+
+use crate::error::{Error, OkOrMessage};
+use crate::migrate::Migrate;
+use crate::persister::PersisterShared;
+
+pub struct BgVars {
+ vars: HashMap<&'static str, Box<dyn BgVarTrait>>,
+}
+
+impl BgVars {
+ pub fn new() -> Self {
+ Self {
+ vars: HashMap::new(),
+ }
+ }
+
+ pub fn register_rw<V, T, GF, SF>(
+ &mut self,
+ p: &PersisterShared<V>,
+ name: &'static str,
+ get_fn: GF,
+ set_fn: SF,
+ ) where
+ V: Migrate + Default + Send + Sync,
+ T: FromStr + ToString + Send + Sync + 'static,
+ GF: Fn(&PersisterShared<V>) -> T + Send + Sync + 'static,
+ SF: Fn(&PersisterShared<V>, T) -> Result<(), Error> + Send + Sync + 'static,
+ {
+ let p1 = p.clone();
+ let get_fn = move || get_fn(&p1);
+
+ let p2 = p.clone();
+ let set_fn = move |v| set_fn(&p2, v);
+
+ self.vars.insert(name, Box::new(BgVar { get_fn, set_fn }));
+ }
+
+ pub fn register_ro<V, T, GF>(&mut self, p: &PersisterShared<V>, name: &'static str, get_fn: GF)
+ where
+ V: Migrate + Default + Send + Sync,
+ T: FromStr + ToString + Send + Sync + 'static,
+ GF: Fn(&PersisterShared<V>) -> T + Send + Sync + 'static,
+ {
+ let p1 = p.clone();
+ let get_fn = move || get_fn(&p1);
+
+ let set_fn = move |_| Err(Error::Message(format!("Cannot set value of {}", name)));
+
+ self.vars.insert(name, Box::new(BgVar { get_fn, set_fn }));
+ }
+
+ pub fn get(&self, var: &str) -> Result<String, Error> {
+ Ok(self
+ .vars
+ .get(var)
+ .ok_or_message("variable does not exist")?
+ .get())
+ }
+
+ pub fn get_all(&self) -> Vec<(&'static str, String)> {
+ self.vars.iter().map(|(k, v)| (*k, v.get())).collect()
+ }
+
+ pub fn set(&self, var: &str, val: &str) -> Result<(), Error> {
+ self.vars
+ .get(var)
+ .ok_or_message("variable does not exist")?
+ .set(val)
+ }
+}
+
+impl Default for BgVars {
+ fn default() -> Self {
+ Self::new()
+ }
+}
+
+// ----
+
+trait BgVarTrait: Send + Sync + 'static {
+ fn get(&self) -> String;
+ fn set(&self, v: &str) -> Result<(), Error>;
+}
+
+struct BgVar<T, GF, SF>
+where
+ T: FromStr + ToString + Send + Sync + 'static,
+ GF: Fn() -> T + Send + Sync + 'static,
+ SF: Fn(T) -> Result<(), Error> + Sync + Send + 'static,
+{
+ get_fn: GF,
+ set_fn: SF,
+}
+
+impl<T, GF, SF> BgVarTrait for BgVar<T, GF, SF>
+where
+ T: FromStr + ToString + Sync + Send + 'static,
+ GF: Fn() -> T + Sync + Send + 'static,
+ SF: Fn(T) -> Result<(), Error> + Sync + Send + 'static,
+{
+ fn get(&self) -> String {
+ (self.get_fn)().to_string()
+ }
+
+ fn set(&self, vstr: &str) -> Result<(), Error> {
+ let value = vstr
+ .parse()
+ .map_err(|_| Error::Message(format!("invalid value: {}", vstr)))?;
+ (self.set_fn)(value)
+ }
+}
diff --git a/src/util/background/worker.rs b/src/util/background/worker.rs
index f5e3addb..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::*;
@@ -10,10 +10,14 @@ 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;
+// 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,
@@ -26,7 +30,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 +41,8 @@ impl std::fmt::Display for WorkerState {
pub trait Worker: Send {
fn name(&self) -> String;
- fn info(&self) -> Option<String> {
- None
+ fn status(&self) -> WorkerStatus {
+ Default::default()
}
/// Work: do a basic unit of work, if one is available (otherwise, should return
@@ -50,10 +54,8 @@ pub trait Worker: Send {
async fn work(&mut self, must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error>;
/// 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<bool>) -> 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,
@@ -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(),
@@ -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<bool>,
- stop_signal_worker: watch::Receiver<bool>,
worker: Box<dyn Worker>,
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!(),