mirror of
https://git.asonix.dog/asonix/background-jobs.git
synced 2024-11-25 05:21:00 +00:00
Add queues, make fault tolerant
This commit is contained in:
parent
0cfec96d4e
commit
c0fcedf8a1
14 changed files with 392 additions and 199 deletions
|
@ -1 +1 @@
|
||||||
RUST_LOG=server_jobs_example,jobs_server_tokio=trace
|
RUST_LOG=server_jobs_example,jobs_server_tokio=info
|
||||||
|
|
|
@ -1,13 +1,18 @@
|
||||||
use failure::Error;
|
use failure::Error;
|
||||||
use jobs::ServerConfig;
|
use jobs::ServerConfig;
|
||||||
|
use server_jobs_example::queue_set;
|
||||||
|
|
||||||
fn main() -> Result<(), Error> {
|
fn main() -> Result<(), Error> {
|
||||||
dotenv::dotenv().ok();
|
dotenv::dotenv().ok();
|
||||||
env_logger::init();
|
env_logger::init();
|
||||||
|
|
||||||
let config = ServerConfig::init("127.0.0.1", 5555, 5556, 1, "example-db")?;
|
tokio::run(ServerConfig::init(
|
||||||
|
"127.0.0.1",
|
||||||
tokio::run(config.run());
|
5555,
|
||||||
|
1,
|
||||||
|
queue_set(),
|
||||||
|
"example-db",
|
||||||
|
));
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
|
@ -3,13 +3,16 @@ use jobs::{Processor, SpawnerConfig};
|
||||||
use server_jobs_example::{MyJobArguments, MyProcessor};
|
use server_jobs_example::{MyJobArguments, MyProcessor};
|
||||||
|
|
||||||
fn main() {
|
fn main() {
|
||||||
|
dotenv::dotenv().ok();
|
||||||
|
env_logger::init();
|
||||||
|
|
||||||
let (_, _, jobs) = (1..50).fold((0, 1, Vec::new()), |(x, y, mut acc), _| {
|
let (_, _, jobs) = (1..50).fold((0, 1, Vec::new()), |(x, y, mut acc), _| {
|
||||||
acc.push(MyJobArguments::new(x, y));
|
acc.push(MyJobArguments::new(x, y));
|
||||||
|
|
||||||
(y, x + y, acc)
|
(y, x + y, acc)
|
||||||
});
|
});
|
||||||
|
|
||||||
let spawner = SpawnerConfig::new("localhost", 5556);
|
let spawner = SpawnerConfig::new("localhost", 5555);
|
||||||
|
|
||||||
tokio::run(lazy(move || {
|
tokio::run(lazy(move || {
|
||||||
for job in jobs {
|
for job in jobs {
|
||||||
|
|
|
@ -1,9 +1,12 @@
|
||||||
use failure::Error;
|
use failure::Error;
|
||||||
use jobs::WorkerConfig;
|
use jobs::WorkerConfig;
|
||||||
use server_jobs_example::MyProcessor;
|
use server_jobs_example::{queue_map, MyProcessor};
|
||||||
|
|
||||||
fn main() -> Result<(), Error> {
|
fn main() -> Result<(), Error> {
|
||||||
let mut worker = WorkerConfig::init(16, "localhost", 5555, 5556)?;
|
dotenv::dotenv().ok();
|
||||||
|
env_logger::init();
|
||||||
|
|
||||||
|
let mut worker = WorkerConfig::new("localhost".to_owned(), 5555, queue_map());
|
||||||
|
|
||||||
worker.register_processor(MyProcessor);
|
worker.register_processor(MyProcessor);
|
||||||
|
|
||||||
|
|
|
@ -1,10 +1,28 @@
|
||||||
#[macro_use]
|
#[macro_use]
|
||||||
|
extern crate log;
|
||||||
|
#[macro_use]
|
||||||
extern crate serde_derive;
|
extern crate serde_derive;
|
||||||
|
|
||||||
|
use std::collections::{BTreeMap, BTreeSet};
|
||||||
|
|
||||||
use failure::Error;
|
use failure::Error;
|
||||||
use futures::{future::IntoFuture, Future};
|
use futures::{future::IntoFuture, Future};
|
||||||
use jobs::{Backoff, MaxRetries, Processor};
|
use jobs::{Backoff, MaxRetries, Processor};
|
||||||
|
|
||||||
|
pub fn queue_map() -> BTreeMap<String, usize> {
|
||||||
|
let mut map = BTreeMap::new();
|
||||||
|
map.insert("default".to_owned(), 18);
|
||||||
|
|
||||||
|
map
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn queue_set() -> BTreeSet<String> {
|
||||||
|
let mut set = BTreeSet::new();
|
||||||
|
set.insert("default".to_owned());
|
||||||
|
|
||||||
|
set
|
||||||
|
}
|
||||||
|
|
||||||
#[derive(Clone, Debug, Deserialize, Serialize)]
|
#[derive(Clone, Debug, Deserialize, Serialize)]
|
||||||
pub struct MyJobArguments {
|
pub struct MyJobArguments {
|
||||||
some_usize: usize,
|
some_usize: usize,
|
||||||
|
@ -30,6 +48,10 @@ impl Processor for MyProcessor {
|
||||||
"MyProcessor"
|
"MyProcessor"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
fn queue() -> &'static str {
|
||||||
|
"default"
|
||||||
|
}
|
||||||
|
|
||||||
fn max_retries() -> MaxRetries {
|
fn max_retries() -> MaxRetries {
|
||||||
MaxRetries::Count(1)
|
MaxRetries::Count(1)
|
||||||
}
|
}
|
||||||
|
@ -39,7 +61,7 @@ impl Processor for MyProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
fn process(&self, args: Self::Arguments) -> Box<dyn Future<Item = (), Error = Error> + Send> {
|
fn process(&self, args: Self::Arguments) -> Box<dyn Future<Item = (), Error = Error> + Send> {
|
||||||
println!("args: {:?}", args);
|
info!("args: {:?}", args);
|
||||||
|
|
||||||
Box::new(Ok(()).into_future())
|
Box::new(Ok(()).into_future())
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,7 +6,7 @@ use serde_json::Value;
|
||||||
use crate::{JobError, JobInfo, Processor};
|
use crate::{JobError, JobInfo, Processor};
|
||||||
|
|
||||||
pub type ProcessFn =
|
pub type ProcessFn =
|
||||||
Box<dyn Fn(Value) -> Box<dyn Future<Item = (), Error = JobError> + Send> + Send>;
|
Box<dyn Fn(Value) -> Box<dyn Future<Item = (), Error = JobError> + Send> + Send + Sync>;
|
||||||
|
|
||||||
pub struct Processors {
|
pub struct Processors {
|
||||||
inner: HashMap<String, ProcessFn>,
|
inner: HashMap<String, ProcessFn>,
|
||||||
|
|
|
@ -30,7 +30,7 @@ pub(crate) struct Config {
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Config {
|
impl Config {
|
||||||
fn create_server(&self) -> impl Future<Item = (), Error = ()> {
|
fn create_server(&self) -> Box<dyn Future<Item = (), Error = ()> + Send> {
|
||||||
let runner_id = self.runner_id;
|
let runner_id = self.runner_id;
|
||||||
let db_path = self.db_path.clone();
|
let db_path = self.db_path.clone();
|
||||||
let base_port = self.base_port;
|
let base_port = self.base_port;
|
||||||
|
@ -38,7 +38,7 @@ impl Config {
|
||||||
|
|
||||||
let config = Arc::new(self.clone());
|
let config = Arc::new(self.clone());
|
||||||
|
|
||||||
poll_fn(move || {
|
let fut = poll_fn(move || {
|
||||||
let runner_id = runner_id;
|
let runner_id = runner_id;
|
||||||
let db_path = db_path.clone();
|
let db_path = db_path.clone();
|
||||||
let base_port = base_port;
|
let base_port = base_port;
|
||||||
|
@ -60,6 +60,8 @@ impl Config {
|
||||||
|
|
||||||
let address = format!("tcp://{}:{}", config.ip, port);
|
let address = format!("tcp://{}:{}", config.ip, port);
|
||||||
|
|
||||||
|
info!("Creating queue {} on address {}", queue, address);
|
||||||
|
|
||||||
tokio::spawn(PushConfig::init(
|
tokio::spawn(PushConfig::init(
|
||||||
address,
|
address,
|
||||||
queue.to_owned(),
|
queue.to_owned(),
|
||||||
|
@ -71,6 +73,7 @@ impl Config {
|
||||||
StalledConfig::init(storage.clone());
|
StalledConfig::init(storage.clone());
|
||||||
|
|
||||||
let portmap_address = format!("tcp://{}:{}", config.ip, config.base_port + 1);
|
let portmap_address = format!("tcp://{}:{}", config.ip, config.base_port + 1);
|
||||||
|
info!("Creating portmap on address {}", portmap_address);
|
||||||
|
|
||||||
tokio::spawn(PortMapConfig::init(
|
tokio::spawn(PortMapConfig::init(
|
||||||
portmap_address,
|
portmap_address,
|
||||||
|
@ -79,12 +82,15 @@ impl Config {
|
||||||
));
|
));
|
||||||
|
|
||||||
let pull_address = format!("tcp://{}:{}", config.ip, config.base_port);
|
let pull_address = format!("tcp://{}:{}", config.ip, config.base_port);
|
||||||
|
info!("Creating puller on address {}", pull_address);
|
||||||
|
|
||||||
tokio::spawn(PullConfig::init(pull_address, storage, config));
|
tokio::spawn(PullConfig::init(pull_address, storage, config));
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
})
|
})
|
||||||
.map_err(|e| error!("Error starting server, {}", e))
|
.map_err(|e| error!("Error starting server, {}", e));
|
||||||
|
|
||||||
|
Box::new(fut)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -101,7 +107,7 @@ impl ServerConfig {
|
||||||
runner_id: usize,
|
runner_id: usize,
|
||||||
queues: BTreeSet<String>,
|
queues: BTreeSet<String>,
|
||||||
db_path: P,
|
db_path: P,
|
||||||
) -> impl Future<Item = (), Error = ()> {
|
) -> Box<dyn Future<Item = (), Error = ()> + Send> {
|
||||||
let context = Arc::new(Context::new());
|
let context = Arc::new(Context::new());
|
||||||
|
|
||||||
Self::init_with_context(ip, base_port, runner_id, queues, db_path, context)
|
Self::init_with_context(ip, base_port, runner_id, queues, db_path, context)
|
||||||
|
@ -114,7 +120,7 @@ impl ServerConfig {
|
||||||
queues: BTreeSet<String>,
|
queues: BTreeSet<String>,
|
||||||
db_path: P,
|
db_path: P,
|
||||||
context: Arc<Context>,
|
context: Arc<Context>,
|
||||||
) -> impl Future<Item = (), Error = ()> {
|
) -> Box<dyn Future<Item = (), Error = ()> + Send> {
|
||||||
let config = Config {
|
let config = Config {
|
||||||
ip: ip.to_owned(),
|
ip: ip.to_owned(),
|
||||||
base_port,
|
base_port,
|
||||||
|
|
|
@ -1,15 +1,15 @@
|
||||||
use std::{collections::BTreeMap, sync::Arc, time::Duration};
|
use std::{collections::BTreeMap, sync::Arc, time::Duration};
|
||||||
|
|
||||||
use failure::Error;
|
use failure::Error;
|
||||||
use futures::{future::lazy, stream::iter_ok, Future, Stream};
|
use futures::{future::lazy, Future, Stream};
|
||||||
use tokio::timer::Delay;
|
use tokio::timer::Delay;
|
||||||
use tokio_zmq::{prelude::*, Multipart, Push};
|
use tokio_zmq::{prelude::*, Multipart, Rep};
|
||||||
use zmq::Message;
|
use zmq::Message;
|
||||||
|
|
||||||
use crate::server::Config;
|
use crate::server::Config;
|
||||||
|
|
||||||
pub(crate) struct PortMapConfig {
|
pub(crate) struct PortMapConfig {
|
||||||
pusher: Push,
|
rep: Rep,
|
||||||
address: String,
|
address: String,
|
||||||
port_map: BTreeMap<String, usize>,
|
port_map: BTreeMap<String, usize>,
|
||||||
config: Arc<Config>,
|
config: Arc<Config>,
|
||||||
|
@ -27,33 +27,33 @@ impl PortMapConfig {
|
||||||
config,
|
config,
|
||||||
};
|
};
|
||||||
|
|
||||||
cfg.build()
|
cfg.build().map_err(|e| error!("Error starting rep, {}", e))
|
||||||
.map_err(|e| error!("Error starting pusher, {}", e))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
fn run(self) -> Box<dyn Future<Item = (), Error = ()> + Send> {
|
fn run(self) -> Box<dyn Future<Item = (), Error = ()> + Send> {
|
||||||
let reset = self.reset();
|
let reset = self.reset();
|
||||||
|
|
||||||
let PortMapConfig {
|
let PortMapConfig {
|
||||||
pusher,
|
rep,
|
||||||
address: _,
|
address: _,
|
||||||
port_map,
|
port_map,
|
||||||
config: _,
|
config: _,
|
||||||
} = self;
|
} = self;
|
||||||
|
|
||||||
let fut = iter_ok::<_, Error>(0..)
|
let (sink, stream) = rep.sink_stream().split();
|
||||||
.and_then(move |count| {
|
|
||||||
trace!("Pushed {} portmaps", count);
|
|
||||||
|
|
||||||
|
let fut = stream
|
||||||
|
.from_err::<Error>()
|
||||||
|
.and_then(move |_| {
|
||||||
let s = serde_json::to_string(&port_map)?;
|
let s = serde_json::to_string(&port_map)?;
|
||||||
let m = Message::from_slice(s.as_ref())?;
|
let m = Message::from_slice(s.as_ref())?;
|
||||||
|
|
||||||
Ok(Multipart::from(m))
|
Ok(Multipart::from(m))
|
||||||
})
|
})
|
||||||
.forward(pusher.sink())
|
.forward(sink)
|
||||||
.map(move |_| info!("portmap pusher shutting down"))
|
.map(move |_| info!("portmap rep shutting down"))
|
||||||
.map_err(|e| {
|
.map_err(|e| {
|
||||||
error!("Error pushing portmap, {}", e);
|
error!("Error sending portmap, {}", e);
|
||||||
|
|
||||||
tokio::spawn(reset.rebuild());
|
tokio::spawn(reset.rebuild());
|
||||||
});
|
});
|
||||||
|
@ -81,17 +81,17 @@ impl ResetPortMapConfig {
|
||||||
Delay::new(tokio::clock::now() + Duration::from_secs(5))
|
Delay::new(tokio::clock::now() + Duration::from_secs(5))
|
||||||
.from_err()
|
.from_err()
|
||||||
.and_then(move |_| self.build())
|
.and_then(move |_| self.build())
|
||||||
.map_err(|e| error!("Error restarting pusher, {}", e))
|
.map_err(|e| error!("Error restarting rep, {}", e))
|
||||||
}
|
}
|
||||||
|
|
||||||
fn build(self) -> impl Future<Item = (), Error = Error> {
|
fn build(self) -> impl Future<Item = (), Error = Error> {
|
||||||
lazy(|| {
|
lazy(|| {
|
||||||
let pusher = Push::builder(self.config.context.clone())
|
let rep = Rep::builder(self.config.context.clone())
|
||||||
.bind(&self.address)
|
.bind(&self.address)
|
||||||
.build()?;
|
.build()?;
|
||||||
|
|
||||||
let config = PortMapConfig {
|
let config = PortMapConfig {
|
||||||
pusher,
|
rep,
|
||||||
address: self.address,
|
address: self.address,
|
||||||
port_map: self.port_map,
|
port_map: self.port_map,
|
||||||
config: self.config,
|
config: self.config,
|
||||||
|
|
|
@ -44,8 +44,15 @@ impl PullConfig {
|
||||||
.puller
|
.puller
|
||||||
.stream()
|
.stream()
|
||||||
.from_err()
|
.from_err()
|
||||||
|
.map(|m| {
|
||||||
|
trace!("Handling new message");
|
||||||
|
m
|
||||||
|
})
|
||||||
.and_then(parse_job)
|
.and_then(parse_job)
|
||||||
.and_then(move |job| store_job(job, storage.clone()))
|
.and_then(move |job| {
|
||||||
|
trace!("Storing job, {:?}", job);
|
||||||
|
store_job(job, storage.clone())
|
||||||
|
})
|
||||||
.for_each(|_| Ok(()))
|
.for_each(|_| Ok(()))
|
||||||
.map(|_| info!("Puller is shutting down"))
|
.map(|_| info!("Puller is shutting down"))
|
||||||
.map_err(|e| {
|
.map_err(|e| {
|
||||||
|
|
|
@ -12,11 +12,11 @@ pub struct SpawnerConfig {
|
||||||
}
|
}
|
||||||
|
|
||||||
impl SpawnerConfig {
|
impl SpawnerConfig {
|
||||||
pub fn new(server_host: &str, queue_port: usize) -> Self {
|
pub fn new(server_host: &str, base_port: usize) -> Self {
|
||||||
let ctx = Arc::new(Context::new());
|
let ctx = Arc::new(Context::new());
|
||||||
|
|
||||||
SpawnerConfig {
|
SpawnerConfig {
|
||||||
server: format!("tcp://{}:{}", server_host, queue_port),
|
server: format!("tcp://{}:{}", server_host, base_port),
|
||||||
ctx,
|
ctx,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -31,12 +31,17 @@ impl SpawnerConfig {
|
||||||
})
|
})
|
||||||
.into_future();
|
.into_future();
|
||||||
|
|
||||||
|
debug!("Sending message to {}", self.server);
|
||||||
|
|
||||||
Push::builder(self.ctx.clone())
|
Push::builder(self.ctx.clone())
|
||||||
.connect(&self.server)
|
.connect(&self.server)
|
||||||
.build()
|
.build()
|
||||||
.into_future()
|
.into_future()
|
||||||
.from_err()
|
.from_err()
|
||||||
.join(msg)
|
.join(msg)
|
||||||
.and_then(move |(req, msg)| req.send(msg).from_err().map(|_| ()))
|
.and_then(move |(push, msg)| {
|
||||||
|
trace!("Sending");
|
||||||
|
push.send(msg).from_err().map(|_| trace!("sent"))
|
||||||
|
})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,166 +0,0 @@
|
||||||
use std::sync::Arc;
|
|
||||||
|
|
||||||
use failure::Error;
|
|
||||||
use futures::{
|
|
||||||
future::{lazy, Either, IntoFuture},
|
|
||||||
Future, Stream,
|
|
||||||
};
|
|
||||||
use jobs_core::{JobInfo, Processor, Processors};
|
|
||||||
use tokio_zmq::{prelude::*, Multipart, Pull, Push};
|
|
||||||
use zmq::{Context, Message};
|
|
||||||
|
|
||||||
struct Worker {
|
|
||||||
processors: Processors,
|
|
||||||
pull: Pull,
|
|
||||||
push: Push,
|
|
||||||
}
|
|
||||||
|
|
||||||
impl Worker {
|
|
||||||
pub fn init(
|
|
||||||
server_host: &str,
|
|
||||||
job_port: usize,
|
|
||||||
queue_port: usize,
|
|
||||||
ctx: Arc<Context>,
|
|
||||||
) -> Result<Self, Error> {
|
|
||||||
let pull = Pull::builder(ctx.clone())
|
|
||||||
.connect(&format!("tcp://{}:{}", server_host, job_port))
|
|
||||||
.build()?;
|
|
||||||
|
|
||||||
let push = Push::builder(ctx.clone())
|
|
||||||
.connect(&format!("tcp://{}:{}", server_host, queue_port))
|
|
||||||
.build()?;
|
|
||||||
|
|
||||||
let processors = Processors::new();
|
|
||||||
|
|
||||||
let worker = Worker {
|
|
||||||
processors,
|
|
||||||
push,
|
|
||||||
pull,
|
|
||||||
};
|
|
||||||
|
|
||||||
Ok(worker)
|
|
||||||
}
|
|
||||||
|
|
||||||
fn register_processor<P>(&mut self, processor: P)
|
|
||||||
where
|
|
||||||
P: Processor + Send + Sync + 'static,
|
|
||||||
{
|
|
||||||
self.processors.register_processor(processor);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
pub struct WorkerConfig {
|
|
||||||
workers: Vec<Worker>,
|
|
||||||
}
|
|
||||||
|
|
||||||
impl WorkerConfig {
|
|
||||||
pub fn init(
|
|
||||||
num_processors: usize,
|
|
||||||
server_host: &str,
|
|
||||||
job_port: usize,
|
|
||||||
queue_port: usize,
|
|
||||||
) -> Result<Self, Error> {
|
|
||||||
let ctx = Arc::new(Context::new());
|
|
||||||
|
|
||||||
let mut workers = Vec::new();
|
|
||||||
|
|
||||||
for _ in 0..num_processors {
|
|
||||||
let worker = Worker::init(server_host, job_port, queue_port, ctx.clone())?;
|
|
||||||
|
|
||||||
workers.push(worker);
|
|
||||||
}
|
|
||||||
|
|
||||||
let cfg = WorkerConfig { workers };
|
|
||||||
|
|
||||||
Ok(cfg)
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn register_processor<P>(&mut self, processor: P)
|
|
||||||
where
|
|
||||||
P: Processor + Send + Sync + 'static,
|
|
||||||
{
|
|
||||||
for worker in self.workers.iter_mut() {
|
|
||||||
worker.register_processor(processor.clone());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn run(self) -> impl Future<Item = (), Error = ()> {
|
|
||||||
let WorkerConfig { workers } = self;
|
|
||||||
|
|
||||||
lazy(|| {
|
|
||||||
for worker in workers.into_iter() {
|
|
||||||
tokio::spawn(worker_future(worker));
|
|
||||||
}
|
|
||||||
|
|
||||||
Ok(())
|
|
||||||
})
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
fn worker_future(worker: Worker) -> impl Future<Item = (), Error = ()> {
|
|
||||||
let Worker {
|
|
||||||
push,
|
|
||||||
pull,
|
|
||||||
processors,
|
|
||||||
} = worker;
|
|
||||||
|
|
||||||
pull.stream()
|
|
||||||
.from_err()
|
|
||||||
.and_then(move |multipart| wrap_processing(multipart, &processors))
|
|
||||||
.map(Some)
|
|
||||||
.or_else(|e| {
|
|
||||||
error!("Error processing job, {}", e);
|
|
||||||
Ok(None)
|
|
||||||
})
|
|
||||||
.filter_map(|item| item)
|
|
||||||
.forward(push.sink())
|
|
||||||
.map_err(|e: Error| error!("Error pushing job, {}", e))
|
|
||||||
.map(|_| ())
|
|
||||||
}
|
|
||||||
|
|
||||||
fn serialize_request(job: JobInfo) -> Result<Multipart, Error> {
|
|
||||||
let request = serde_json::to_string(&job)?;
|
|
||||||
let msg = Message::from_slice(request.as_ref())?;
|
|
||||||
|
|
||||||
Ok(msg.into())
|
|
||||||
}
|
|
||||||
|
|
||||||
fn parse_multipart(mut multipart: Multipart) -> Result<JobInfo, Error> {
|
|
||||||
let message = multipart.pop_front().ok_or(ParseError)?;
|
|
||||||
|
|
||||||
let parsed = serde_json::from_slice(&message)?;
|
|
||||||
|
|
||||||
Ok(parsed)
|
|
||||||
}
|
|
||||||
|
|
||||||
fn wrap_processing(
|
|
||||||
multipart: Multipart,
|
|
||||||
processors: &Processors,
|
|
||||||
) -> impl Future<Item = Multipart, Error = Error> {
|
|
||||||
let msg = match parse_multipart(multipart) {
|
|
||||||
Ok(msg) => msg,
|
|
||||||
Err(e) => return Either::A(Err(e).into_future()),
|
|
||||||
};
|
|
||||||
|
|
||||||
let fut = process_job(msg, processors).and_then(serialize_request);
|
|
||||||
|
|
||||||
Either::B(fut)
|
|
||||||
}
|
|
||||||
|
|
||||||
fn process_job(
|
|
||||||
job: JobInfo,
|
|
||||||
processors: &Processors,
|
|
||||||
) -> impl Future<Item = JobInfo, Error = Error> {
|
|
||||||
processors
|
|
||||||
.process_job(job.clone())
|
|
||||||
.map_err(|_| ProcessError)
|
|
||||||
.from_err()
|
|
||||||
}
|
|
||||||
|
|
||||||
#[derive(Clone, Debug, Fail)]
|
|
||||||
#[fail(display = "Error parsing job")]
|
|
||||||
struct ParseError;
|
|
||||||
|
|
||||||
#[derive(Clone, Debug, Fail)]
|
|
||||||
#[fail(display = "Error processing job")]
|
|
||||||
struct ProcessError;
|
|
172
jobs-server-tokio/src/worker/config.rs
Normal file
172
jobs-server-tokio/src/worker/config.rs
Normal file
|
@ -0,0 +1,172 @@
|
||||||
|
use std::{sync::Arc, time::Duration};
|
||||||
|
|
||||||
|
use failure::Error;
|
||||||
|
use futures::{
|
||||||
|
future::{lazy, Either, IntoFuture},
|
||||||
|
Future, Stream,
|
||||||
|
};
|
||||||
|
use jobs_core::{JobInfo, Processors};
|
||||||
|
use tokio::timer::Delay;
|
||||||
|
use tokio_zmq::{prelude::*, Multipart, Pull, Push};
|
||||||
|
use zmq::{Context, Message};
|
||||||
|
|
||||||
|
pub(crate) struct Worker {
|
||||||
|
pull: Pull,
|
||||||
|
push: Push,
|
||||||
|
push_address: String,
|
||||||
|
pull_address: String,
|
||||||
|
queue: String,
|
||||||
|
processors: Arc<Processors>,
|
||||||
|
context: Arc<Context>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl Worker {
|
||||||
|
pub(crate) fn init(
|
||||||
|
push_address: String,
|
||||||
|
pull_address: String,
|
||||||
|
queue: String,
|
||||||
|
processors: Arc<Processors>,
|
||||||
|
context: Arc<Context>,
|
||||||
|
) -> impl Future<Item = (), Error = ()> {
|
||||||
|
let cfg = ResetWorker {
|
||||||
|
push_address,
|
||||||
|
pull_address,
|
||||||
|
queue: queue.clone(),
|
||||||
|
processors,
|
||||||
|
context,
|
||||||
|
};
|
||||||
|
|
||||||
|
cfg.build()
|
||||||
|
.map_err(move |e| error!("Error starting worker for queue {}, {}", queue, e))
|
||||||
|
}
|
||||||
|
|
||||||
|
fn run(self) -> Box<dyn Future<Item = (), Error = ()> + Send> {
|
||||||
|
let reset = self.reset();
|
||||||
|
|
||||||
|
let Worker {
|
||||||
|
push,
|
||||||
|
pull,
|
||||||
|
push_address: _,
|
||||||
|
pull_address: _,
|
||||||
|
queue,
|
||||||
|
processors,
|
||||||
|
context: _,
|
||||||
|
} = self;
|
||||||
|
|
||||||
|
let fut = pull
|
||||||
|
.stream()
|
||||||
|
.from_err::<Error>()
|
||||||
|
.and_then(move |multipart| wrap_processing(multipart, &processors))
|
||||||
|
.forward(push.sink())
|
||||||
|
.map(move |_| info!("worker for queue {} is shutting down", queue))
|
||||||
|
.map_err(|e| {
|
||||||
|
error!("Error processing job, {}", e);
|
||||||
|
|
||||||
|
tokio::spawn(reset.rebuild());
|
||||||
|
});
|
||||||
|
|
||||||
|
Box::new(fut)
|
||||||
|
}
|
||||||
|
|
||||||
|
fn reset(&self) -> ResetWorker {
|
||||||
|
ResetWorker {
|
||||||
|
push_address: self.push_address.clone(),
|
||||||
|
pull_address: self.pull_address.clone(),
|
||||||
|
queue: self.queue.clone(),
|
||||||
|
processors: self.processors.clone(),
|
||||||
|
context: self.context.clone(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
struct ResetWorker {
|
||||||
|
push_address: String,
|
||||||
|
pull_address: String,
|
||||||
|
queue: String,
|
||||||
|
processors: Arc<Processors>,
|
||||||
|
context: Arc<Context>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl ResetWorker {
|
||||||
|
fn rebuild(self) -> impl Future<Item = (), Error = ()> {
|
||||||
|
let queue = self.queue.clone();
|
||||||
|
|
||||||
|
Delay::new(tokio::clock::now() + Duration::from_secs(5))
|
||||||
|
.from_err()
|
||||||
|
.and_then(move |_| self.build())
|
||||||
|
.map_err(move |e| error!("Error restarting worker for queue {}, {}", queue, e))
|
||||||
|
}
|
||||||
|
|
||||||
|
fn build(self) -> impl Future<Item = (), Error = Error> {
|
||||||
|
lazy(|| {
|
||||||
|
let push = Push::builder(self.context.clone())
|
||||||
|
.connect(&self.push_address)
|
||||||
|
.build()?;
|
||||||
|
|
||||||
|
let pull = Pull::builder(self.context.clone())
|
||||||
|
.connect(&self.pull_address)
|
||||||
|
.build()?;
|
||||||
|
|
||||||
|
let config = Worker {
|
||||||
|
push,
|
||||||
|
pull,
|
||||||
|
push_address: self.push_address,
|
||||||
|
pull_address: self.pull_address,
|
||||||
|
queue: self.queue,
|
||||||
|
processors: self.processors,
|
||||||
|
context: self.context,
|
||||||
|
};
|
||||||
|
|
||||||
|
tokio::spawn(config.run());
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
fn serialize_request(job: JobInfo) -> Result<Multipart, Error> {
|
||||||
|
let request = serde_json::to_string(&job)?;
|
||||||
|
let msg = Message::from_slice(request.as_ref())?;
|
||||||
|
|
||||||
|
Ok(msg.into())
|
||||||
|
}
|
||||||
|
|
||||||
|
fn parse_multipart(mut multipart: Multipart) -> Result<JobInfo, Error> {
|
||||||
|
let message = multipart.pop_front().ok_or(ParseError)?;
|
||||||
|
|
||||||
|
let parsed = serde_json::from_slice(&message)?;
|
||||||
|
|
||||||
|
Ok(parsed)
|
||||||
|
}
|
||||||
|
|
||||||
|
fn wrap_processing(
|
||||||
|
multipart: Multipart,
|
||||||
|
processors: &Processors,
|
||||||
|
) -> impl Future<Item = Multipart, Error = Error> {
|
||||||
|
let msg = match parse_multipart(multipart) {
|
||||||
|
Ok(msg) => msg,
|
||||||
|
Err(e) => return Either::A(Err(e).into_future()),
|
||||||
|
};
|
||||||
|
|
||||||
|
let fut = process_job(msg, processors).and_then(serialize_request);
|
||||||
|
|
||||||
|
Either::B(fut)
|
||||||
|
}
|
||||||
|
|
||||||
|
fn process_job(
|
||||||
|
job: JobInfo,
|
||||||
|
processors: &Processors,
|
||||||
|
) -> impl Future<Item = JobInfo, Error = Error> {
|
||||||
|
processors
|
||||||
|
.process_job(job.clone())
|
||||||
|
.map_err(|_| ProcessError)
|
||||||
|
.from_err()
|
||||||
|
}
|
||||||
|
|
||||||
|
#[derive(Clone, Debug, Fail)]
|
||||||
|
#[fail(display = "Error parsing job")]
|
||||||
|
struct ParseError;
|
||||||
|
|
||||||
|
#[derive(Clone, Debug, Fail)]
|
||||||
|
#[fail(display = "Error processing job")]
|
||||||
|
struct ProcessError;
|
98
jobs-server-tokio/src/worker/mod.rs
Normal file
98
jobs-server-tokio/src/worker/mod.rs
Normal file
|
@ -0,0 +1,98 @@
|
||||||
|
use std::{collections::BTreeMap, sync::Arc};
|
||||||
|
|
||||||
|
use futures::Future;
|
||||||
|
use jobs_core::{Processor, Processors};
|
||||||
|
use zmq::Context;
|
||||||
|
|
||||||
|
mod config;
|
||||||
|
mod portmap;
|
||||||
|
|
||||||
|
use self::{config::Worker, portmap::PortMap};
|
||||||
|
|
||||||
|
pub struct WorkerConfig {
|
||||||
|
processors: Processors,
|
||||||
|
queues: BTreeMap<String, usize>,
|
||||||
|
server_host: String,
|
||||||
|
base_port: usize,
|
||||||
|
context: Arc<Context>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl WorkerConfig {
|
||||||
|
pub fn new(server_host: String, base_port: usize, queues: BTreeMap<String, usize>) -> Self {
|
||||||
|
let context = Arc::new(Context::new());
|
||||||
|
|
||||||
|
Self::new_with_context(server_host, base_port, queues, context)
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn new_with_context(
|
||||||
|
server_host: String,
|
||||||
|
base_port: usize,
|
||||||
|
queues: BTreeMap<String, usize>,
|
||||||
|
context: Arc<Context>,
|
||||||
|
) -> Self {
|
||||||
|
WorkerConfig {
|
||||||
|
processors: Processors::new(),
|
||||||
|
server_host,
|
||||||
|
base_port,
|
||||||
|
queues,
|
||||||
|
context,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn register_processor<P>(&mut self, processor: P)
|
||||||
|
where
|
||||||
|
P: Processor + Send + Sync + 'static,
|
||||||
|
{
|
||||||
|
self.processors.register_processor(processor);
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn run(self) -> Box<dyn Future<Item = (), Error = ()> + Send> {
|
||||||
|
let WorkerConfig {
|
||||||
|
processors,
|
||||||
|
server_host,
|
||||||
|
base_port,
|
||||||
|
queues,
|
||||||
|
context,
|
||||||
|
} = self;
|
||||||
|
|
||||||
|
info!("Starting workers");
|
||||||
|
|
||||||
|
let processors = Arc::new(processors);
|
||||||
|
|
||||||
|
let push_address = format!("tcp://{}:{}", server_host, base_port);
|
||||||
|
let portmap_address = format!("tcp://{}:{}", server_host, base_port + 1);
|
||||||
|
|
||||||
|
info!("push address, {}", push_address);
|
||||||
|
info!("portmap address, {}", portmap_address);
|
||||||
|
|
||||||
|
let fut = PortMap::init(portmap_address, context.clone())
|
||||||
|
.and_then(move |port_map| {
|
||||||
|
info!("Got port_map, {:?}", port_map);
|
||||||
|
|
||||||
|
for (queue, num) in queues.iter() {
|
||||||
|
let port = port_map.get(queue).ok_or(MissingQueue(queue.to_owned()))?;
|
||||||
|
|
||||||
|
let pull_address = format!("tcp://{}:{}", server_host, port);
|
||||||
|
|
||||||
|
for _ in 0..*num {
|
||||||
|
tokio::spawn(Worker::init(
|
||||||
|
push_address.clone(),
|
||||||
|
pull_address.clone(),
|
||||||
|
queue.to_owned(),
|
||||||
|
processors.clone(),
|
||||||
|
context.clone(),
|
||||||
|
));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(())
|
||||||
|
})
|
||||||
|
.map_err(|e| error!("Error starting worker, {}", e));
|
||||||
|
|
||||||
|
Box::new(fut)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
#[derive(Clone, Debug, Fail)]
|
||||||
|
#[fail(display = "Could not find queue, {}", _0)]
|
||||||
|
struct MissingQueue(String);
|
38
jobs-server-tokio/src/worker/portmap.rs
Normal file
38
jobs-server-tokio/src/worker/portmap.rs
Normal file
|
@ -0,0 +1,38 @@
|
||||||
|
use std::{collections::BTreeMap, sync::Arc};
|
||||||
|
|
||||||
|
use failure::Error;
|
||||||
|
use futures::{future::lazy, Future};
|
||||||
|
use tokio_zmq::{prelude::*, Req};
|
||||||
|
use zmq::{Context, Message};
|
||||||
|
|
||||||
|
pub(crate) struct PortMap;
|
||||||
|
|
||||||
|
impl PortMap {
|
||||||
|
pub(crate) fn init(
|
||||||
|
address: String,
|
||||||
|
context: Arc<Context>,
|
||||||
|
) -> impl Future<Item = BTreeMap<String, usize>, Error = Error> {
|
||||||
|
lazy(move || {
|
||||||
|
let req = Req::builder(context.clone()).connect(&address).build()?;
|
||||||
|
|
||||||
|
Ok(req)
|
||||||
|
})
|
||||||
|
.and_then(|req| {
|
||||||
|
Message::from_slice(b"h")
|
||||||
|
.map_err(Error::from)
|
||||||
|
.map(move |msg| (req, msg.into()))
|
||||||
|
})
|
||||||
|
.and_then(|(req, msg)| req.send(msg).and_then(|req| req.recv()).from_err())
|
||||||
|
.and_then(|(mut multipart, _)| {
|
||||||
|
let msg = multipart.pop_front().ok_or(EmptyMessage)?;
|
||||||
|
|
||||||
|
let map = serde_json::from_slice(&msg)?;
|
||||||
|
|
||||||
|
Ok(map)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
#[derive(Clone, Debug, Fail)]
|
||||||
|
#[fail(display = "Message was empty")]
|
||||||
|
struct EmptyMessage;
|
Loading…
Reference in a new issue