make actix-server self runtime hosting

This commit is contained in:
fakeshadow 2021-01-30 08:37:06 -08:00
parent df15ef40f6
commit e83eb0739a
4 changed files with 81 additions and 78 deletions

View File

@ -1,10 +1,6 @@
use std::time::Duration; use std::time::{Duration, Instant};
use std::{io, thread}; use std::{io, thread};
use actix_rt::{
time::{sleep_until, Instant},
System,
};
use log::{error, info}; use log::{error, info};
use mio::{Interest, Poll, Token as MioToken}; use mio::{Interest, Poll, Token as MioToken};
use slab::Slab; use slab::Slab;
@ -15,6 +11,8 @@ use crate::waker_queue::{WakerInterest, WakerQueue, WAKER_TOKEN};
use crate::worker::{Conn, WorkerHandle}; use crate::worker::{Conn, WorkerHandle};
use crate::Token; use crate::Token;
const DUR_ON_ERR: Duration = Duration::from_millis(500);
struct ServerSocketInfo { struct ServerSocketInfo {
// addr for socket. mainly used for logging. // addr for socket. mainly used for logging.
addr: SocketAddr, addr: SocketAddr,
@ -22,9 +20,9 @@ struct ServerSocketInfo {
// mio::Token // mio::Token
token: Token, token: Token,
lst: MioListener, lst: MioListener,
// timeout is used to mark the deadline when this socket's listener should be registered again // mark the deadline when this socket's listener should be registered again
// after an error. // after an error.
timeout: Option<Instant>, timeout_deadline: Option<Instant>,
} }
/// Accept loop would live with `ServerBuilder`. /// Accept loop would live with `ServerBuilder`.
@ -81,6 +79,9 @@ struct Accept {
srv: Server, srv: Server,
next: usize, next: usize,
backpressure: bool, backpressure: bool,
// poll time duration.
// use the nearest timeout from socket timeout settings.
timeout: Option<Duration>,
} }
/// This function defines errors that are per-connection. Which basically /// This function defines errors that are per-connection. Which basically
@ -104,13 +105,10 @@ impl Accept {
srv: Server, srv: Server,
handles: Vec<WorkerHandle>, handles: Vec<WorkerHandle>,
) { ) {
// Accept runs in its own thread and would want to spawn additional futures to current // Accept runs in its own thread
// actix system.
let sys = System::current();
thread::Builder::new() thread::Builder::new()
.name("actix-server accept loop".to_owned()) .name("actix-server accept loop".to_owned())
.spawn(move || { .spawn(move || {
System::set_current(sys);
let (mut accept, sockets) = let (mut accept, sockets) =
Accept::new_with_sockets(poll, waker, socks, handles, srv); Accept::new_with_sockets(poll, waker, socks, handles, srv);
accept.poll_with(sockets); accept.poll_with(sockets);
@ -141,7 +139,7 @@ impl Accept {
addr, addr,
token: hnd_token, token: hnd_token,
lst, lst,
timeout: None, timeout_deadline: None,
}); });
} }
@ -152,6 +150,7 @@ impl Accept {
srv, srv,
next: 0, next: 0,
backpressure: false, backpressure: false,
timeout: None,
}; };
(accept, sockets) (accept, sockets)
@ -162,9 +161,10 @@ impl Accept {
loop { loop {
self.poll self.poll
.poll(&mut events, None) .poll(&mut events, self.timeout)
.unwrap_or_else(|e| panic!("Poll error: {}", e)); .unwrap_or_else(|e| panic!("Poll error: {}", e));
// handle events
for event in events.iter() { for event in events.iter() {
let token = event.token(); let token = event.token();
match token { match token {
@ -190,12 +190,6 @@ impl Accept {
self.maybe_backpressure(&mut sockets, false); self.maybe_backpressure(&mut sockets, false);
self.handles.push(handle); self.handles.push(handle);
} }
// got timer interest and it's time to try register socket(s)
// again.
Some(WakerInterest::Timer) => {
drop(guard);
self.process_timer(&mut sockets)
}
Some(WakerInterest::Pause) => { Some(WakerInterest::Pause) => {
drop(guard); drop(guard);
sockets.iter_mut().for_each(|(_, info)| { sockets.iter_mut().for_each(|(_, info)| {
@ -234,21 +228,43 @@ impl Accept {
} }
} }
} }
// check for timeout and re-register sockets.
self.process_timeout(&mut sockets);
} }
} }
fn process_timer(&self, sockets: &mut Slab<ServerSocketInfo>) { fn process_timeout(&mut self, sockets: &mut Slab<ServerSocketInfo>) {
let now = Instant::now(); // take old timeout as it's no use after each iteration.
sockets.iter_mut().for_each(|(token, info)| { if self.timeout.take().is_some() {
// only the ServerSocketInfo have an associate timeout value was de registered. let now = Instant::now();
if let Some(inst) = info.timeout.take() { sockets.iter_mut().for_each(|(token, info)| {
if now > inst { // only the ServerSocketInfo have an associate timeout value was de registered.
self.register_logged(token, info); if let Some(inst) = info.timeout_deadline {
} else { // timeout expired register socket again.
info.timeout = Some(inst); if now >= inst {
info.timeout_deadline = None;
self.register_logged(token, info);
} else {
// still timed out. try set new timeout.
let dur = inst - now;
self.set_timeout(dur);
}
}
});
}
}
// update Accept timeout duration. would keep the smallest duration.
fn set_timeout(&mut self, dur: Duration) {
match self.timeout {
Some(timeout) => {
if timeout > dur {
self.timeout = Some(dur);
} }
} }
}); None => self.timeout = Some(dur),
}
} }
#[cfg(not(target_os = "windows"))] #[cfg(not(target_os = "windows"))]
@ -298,7 +314,7 @@ impl Accept {
if !on { if !on {
self.backpressure = false; self.backpressure = false;
for (token, info) in sockets.iter_mut() { for (token, info) in sockets.iter_mut() {
if info.timeout.is_some() { if info.timeout_deadline.is_some() {
// socket will attempt to re-register itself when its timeout completes // socket will attempt to re-register itself when its timeout completes
continue; continue;
} }
@ -396,17 +412,11 @@ impl Accept {
error!("Can not deregister server socket {}", err); error!("Can not deregister server socket {}", err);
} }
// sleep after error. write the timeout to socket info as later the poll // sleep after error. write the timeout deadline to socket info
// would need it mark which socket and when it's listener should be // as later the poll would need it mark which socket and when
// registered. // it's listener should be registered again.
info.timeout = Some(Instant::now() + Duration::from_millis(500)); info.timeout_deadline = Some(Instant::now() + DUR_ON_ERR);
self.set_timeout(DUR_ON_ERR);
// after the sleep a Timer interest is sent to Accept Poll
let waker = self.waker.clone();
System::current().worker().spawn(async move {
sleep_until(Instant::now() + Duration::from_millis(510)).await;
waker.wake(WakerInterest::Timer);
});
return; return;
} }

View File

@ -19,7 +19,7 @@ use crate::signals::{Signal, Signals};
use crate::socket::{MioListener, StdSocketAddr, StdTcpListener, ToSocketAddrs}; use crate::socket::{MioListener, StdSocketAddr, StdTcpListener, ToSocketAddrs};
use crate::socket::{MioTcpListener, MioTcpSocket}; use crate::socket::{MioTcpListener, MioTcpSocket};
use crate::waker_queue::{WakerInterest, WakerQueue}; use crate::waker_queue::{WakerInterest, WakerQueue};
use crate::worker::{self, ServerWorker, WorkerAvailability, WorkerHandle}; use crate::worker::{self, Worker, WorkerAvailability, WorkerHandle};
use crate::{join_all, Token}; use crate::{join_all, Token};
/// Server builder /// Server builder
@ -297,7 +297,7 @@ impl ServerBuilder {
let avail = WorkerAvailability::new(waker); let avail = WorkerAvailability::new(waker);
let services = self.services.iter().map(|v| v.clone_factory()).collect(); let services = self.services.iter().map(|v| v.clone_factory()).collect();
ServerWorker::start(idx, services, avail, self.shutdown_timeout) Worker::start(idx, services, avail, self.shutdown_timeout)
} }
fn handle_cmd(&mut self, item: ServerCommand) { fn handle_cmd(&mut self, item: ServerCommand) {

View File

@ -78,10 +78,6 @@ pub(crate) enum WakerInterest {
Pause, Pause,
Resume, Resume,
Stop, Stop,
/// `Timer` is an interest sent as a delayed future. When an error happens on accepting
/// connection `Accept` would deregister socket listener temporary and wake up the poll and
/// register them again after the delayed future resolve.
Timer,
/// `Worker` is an interest happen after a worker runs into faulted state(This is determined /// `Worker` is an interest happen after a worker runs into faulted state(This is determined
/// by if work can be sent to it successfully).`Accept` would be waked up and add the new /// by if work can be sent to it successfully).`Accept` would be waked up and add the new
/// `WorkerHandle`. /// `WorkerHandle`.

View File

@ -6,7 +6,6 @@ use std::task::{Context, Poll};
use std::time::Duration; use std::time::Duration;
use actix_rt::time::{sleep_until, Instant, Sleep}; use actix_rt::time::{sleep_until, Instant, Sleep};
use actix_rt::{spawn, Worker as Arbiter};
use actix_utils::counter::Counter; use actix_utils::counter::Counter;
use futures_core::future::LocalBoxFuture; use futures_core::future::LocalBoxFuture;
use log::{error, info, trace}; use log::{error, info, trace};
@ -122,10 +121,8 @@ impl WorkerAvailability {
} }
} }
/// Service worker.
///
/// Worker accepts Socket objects via unbounded channel and starts stream processing. /// Worker accepts Socket objects via unbounded channel and starts stream processing.
pub(crate) struct ServerWorker { pub(crate) struct Worker {
rx: UnboundedReceiver<WorkerCommand>, rx: UnboundedReceiver<WorkerCommand>,
rx2: UnboundedReceiver<StopCommand>, rx2: UnboundedReceiver<StopCommand>,
services: Vec<WorkerService>, services: Vec<WorkerService>,
@ -159,7 +156,7 @@ enum WorkerServiceStatus {
Stopped, Stopped,
} }
impl ServerWorker { impl Worker {
pub(crate) fn start( pub(crate) fn start(
idx: usize, idx: usize,
factories: Vec<Box<dyn InternalServiceFactory>>, factories: Vec<Box<dyn InternalServiceFactory>>,
@ -170,10 +167,9 @@ impl ServerWorker {
let (tx2, rx2) = unbounded_channel(); let (tx2, rx2) = unbounded_channel();
let avail = availability.clone(); let avail = availability.clone();
// every worker runs in it's own arbiter. // every worker runs in it's own thread.
Arbiter::new().spawn(Box::pin(async move { std::thread::spawn(move || {
availability.set(false); let mut wrk = MAX_CONNS_COUNTER.with(move |conns| Worker {
let mut wrk = MAX_CONNS_COUNTER.with(move |conns| ServerWorker {
rx, rx,
rx2, rx2,
availability, availability,
@ -184,23 +180,26 @@ impl ServerWorker {
state: WorkerState::Unavailable, state: WorkerState::Unavailable,
}); });
let fut = wrk actix_rt::Runtime::new().unwrap().block_on(async move {
.factories let fut = wrk
.iter() .factories
.enumerate() .iter()
.map(|(idx, factory)| { .enumerate()
let fut = factory.create(); .map(|(idx, factory)| {
async move { let fut = factory.create();
fut.await.map(|r| { async move {
r.into_iter().map(|(t, s)| (idx, t, s)).collect::<Vec<_>>() fut.await.map(|r| {
}) r.into_iter().map(|(t, s)| (idx, t, s)).collect::<Vec<_>>()
} })
}) }
.collect::<Vec<_>>(); })
.collect::<Vec<_>>();
spawn(async move { match join_all(fut)
let res: Result<Vec<_>, _> = join_all(fut).await.into_iter().collect(); .await
match res { .into_iter()
.collect::<Result<Vec<_>, _>>()
{
Ok(services) => { Ok(services) => {
for item in services { for item in services {
for (factory, token, service) in item { for (factory, token, service) in item {
@ -215,12 +214,12 @@ impl ServerWorker {
} }
Err(e) => { Err(e) => {
error!("Can not start worker: {:?}", e); error!("Can not start worker: {:?}", e);
Arbiter::handle().stop();
} }
} }
wrk.await wrk.await
}); })
})); });
WorkerHandle::new(idx, tx1, tx2, avail) WorkerHandle::new(idx, tx1, tx2, avail)
} }
@ -303,7 +302,7 @@ enum WorkerState {
), ),
} }
impl Future for ServerWorker { impl Future for Worker {
type Output = (); type Output = ();
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> { fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
@ -386,7 +385,6 @@ impl Future for ServerWorker {
let num = num_connections(); let num = num_connections();
if num == 0 { if num == 0 {
let _ = tx.take().unwrap().send(true); let _ = tx.take().unwrap().send(true);
Arbiter::handle().stop();
return Poll::Ready(()); return Poll::Ready(());
} }
@ -394,7 +392,6 @@ impl Future for ServerWorker {
if Pin::new(t2).poll(cx).is_ready() { if Pin::new(t2).poll(cx).is_ready() {
let _ = tx.take().unwrap().send(false); let _ = tx.take().unwrap().send(false);
self.shutdown(true); self.shutdown(true);
Arbiter::handle().stop();
return Poll::Ready(()); return Poll::Ready(());
} }