From b7202db8fd989cd8fe534b2f2c74c8205f867094 Mon Sep 17 00:00:00 2001 From: fakeshadow <24548779@qq.com> Date: Tue, 29 Dec 2020 07:44:53 +0800 Subject: [PATCH 1/8] update actix-server and actix-testing to tokio 1.0 (#239) --- actix-server/CHANGES.md | 6 + actix-server/Cargo.toml | 26 +- actix-server/src/accept.rs | 500 +++++++++++++----------------- actix-server/src/builder.rs | 207 ++++++------- actix-server/src/config.rs | 61 ++-- actix-server/src/lib.rs | 102 ++++++ actix-server/src/server.rs | 32 +- actix-server/src/service.rs | 43 ++- actix-server/src/signals.rs | 86 +++-- actix-server/src/socket.rs | 326 +++++++++++-------- actix-server/src/waker_queue.rs | 89 ++++++ actix-server/src/worker.rs | 335 +++++++++----------- actix-server/tests/test_server.rs | 27 +- actix-testing/Cargo.toml | 4 +- actix-testing/src/lib.rs | 5 +- 15 files changed, 1008 insertions(+), 841 deletions(-) create mode 100644 actix-server/src/waker_queue.rs diff --git a/actix-server/CHANGES.md b/actix-server/CHANGES.md index 5e28fe0b..15d7d596 100644 --- a/actix-server/CHANGES.md +++ b/actix-server/CHANGES.md @@ -3,6 +3,12 @@ ## Unreleased - 2020-xx-xx * Added explicit info log message on accept queue pause. [#215] * Prevent double registration of sockets when back-pressure is resolved. [#223] +* Update `mio` dependency to `0.7.3`. +* Remove `socket2` dependency. +* `ServerBuilder::backlog` would accept `u32` instead of `i32`. +* Remove `AcceptNotify` type and pass `WakerQueue` to `Worker` for wake up the `Accept`'s `Poll`. +* Convert `mio::net::TcpStream` to `actix_rt::net::TcpStream`(`UnixStream` for uds) using `FromRawFd` and `IntoRawFd`(`FromRawSocket` and `IntoRawSocket` on windows). +* Remove `AsyncRead` and `AsyncWrite` trait bound for `socket::FromStream` trait. [#215]: https://github.com/actix/actix-net/pull/215 [#223]: https://github.com/actix/actix-net/pull/223 diff --git a/actix-server/Cargo.toml b/actix-server/Cargo.toml index 1a67f61c..34fb3775 100644 --- a/actix-server/Cargo.toml +++ b/actix-server/Cargo.toml @@ -20,25 +20,21 @@ path = "src/lib.rs" default = [] [dependencies] -actix-service = "1.0.6" -actix-rt = "1.1.1" -actix-codec = "0.3.0" -actix-utils = "2.0.0" +actix-codec = "0.4.0-beta.1" +actix-rt = "2.0.0-beta.1" +actix-service = "2.0.0-beta.1" +actix-utils = "3.0.0-beta.1" +futures-core = { version = "0.3.7", default-features = false } log = "0.4" +mio = { version = "0.7.6", features = ["os-poll", "net"] } num_cpus = "1.13" -mio = "0.6.19" -socket2 = "0.3" -futures-channel = { version = "0.3.4", default-features = false } -futures-util = { version = "0.3.4", default-features = false, features = ["sink"] } slab = "0.4" - -# unix domain sockets -# FIXME: Remove it and use mio own uds feature once mio 0.7 is released -mio-uds = { version = "0.6.7" } +tokio = { version = "1", features = ["sync"] } [dev-dependencies] -bytes = "0.5" -env_logger = "0.7" actix-testing = "1.0.0" -tokio = { version = "0.2", features = ["io-util"] } +bytes = "1" +env_logger = "0.7" +futures-util = { version = "0.3.7", default-features = false, features = ["sink"] } +tokio = { version = "1", features = ["io-util"] } diff --git a/actix-server/src/accept.rs b/actix-server/src/accept.rs index bef175d8..bf895f06 100644 --- a/actix-server/src/accept.rs +++ b/actix-server/src/accept.rs @@ -1,120 +1,86 @@ -use std::sync::mpsc as sync_mpsc; use std::time::Duration; use std::{io, thread}; -use actix_rt::time::{delay_until, Instant}; +use actix_rt::time::{sleep_until, Instant}; use actix_rt::System; use log::{error, info}; +use mio::{Interest, Poll, Token as MioToken}; use slab::Slab; use crate::server::Server; -use crate::socket::{SocketAddr, SocketListener, StdListener}; -use crate::worker::{Conn, WorkerClient}; +use crate::socket::{MioListener, SocketAddr}; +use crate::waker_queue::{WakerInterest, WakerQueue, WAKER_TOKEN}; +use crate::worker::{Conn, WorkerHandle}; use crate::Token; -pub(crate) enum Command { - Pause, - Resume, - Stop, - Worker(WorkerClient), -} - struct ServerSocketInfo { + // addr for socket. mainly used for logging. addr: SocketAddr, + // be ware this is the crate token for identify socket and should not be confused with + // mio::Token token: Token, - sock: SocketListener, + lst: MioListener, + // timeout is used to mark the deadline when this socket's listener should be registered again + // after an error. timeout: Option, } -#[derive(Clone)] -pub(crate) struct AcceptNotify(mio::SetReadiness); - -impl AcceptNotify { - pub(crate) fn new(ready: mio::SetReadiness) -> Self { - AcceptNotify(ready) - } - - pub(crate) fn notify(&self) { - let _ = self.0.set_readiness(mio::Ready::readable()); - } -} - -impl Default for AcceptNotify { - fn default() -> Self { - AcceptNotify::new(mio::Registration::new2().1) - } -} - +/// Accept loop would live with `ServerBuilder`. +/// +/// It's tasked with construct `Poll` instance and `WakerQueue` which would be distributed to +/// `Accept` and `Worker`. +/// +/// It would also listen to `ServerCommand` and push interests to `WakerQueue`. pub(crate) struct AcceptLoop { - cmd_reg: Option, - cmd_ready: mio::SetReadiness, - notify_reg: Option, - notify_ready: mio::SetReadiness, - tx: sync_mpsc::Sender, - rx: Option>, srv: Option, + poll: Option, + waker: WakerQueue, } impl AcceptLoop { - pub fn new(srv: Server) -> AcceptLoop { - let (tx, rx) = sync_mpsc::channel(); - let (cmd_reg, cmd_ready) = mio::Registration::new2(); - let (notify_reg, notify_ready) = mio::Registration::new2(); + pub fn new(srv: Server) -> Self { + let poll = Poll::new().unwrap_or_else(|e| panic!("Can not create `mio::Poll`: {}", e)); + let waker = WakerQueue::new(poll.registry()) + .unwrap_or_else(|e| panic!("Can not create `mio::Waker`: {}", e)); - AcceptLoop { - tx, - cmd_ready, - cmd_reg: Some(cmd_reg), - notify_ready, - notify_reg: Some(notify_reg), - rx: Some(rx), + Self { srv: Some(srv), + poll: Some(poll), + waker, } } - pub fn send(&self, msg: Command) { - let _ = self.tx.send(msg); - let _ = self.cmd_ready.set_readiness(mio::Ready::readable()); + pub(crate) fn waker_owned(&self) -> WakerQueue { + self.waker.clone() } - pub fn get_notify(&self) -> AcceptNotify { - AcceptNotify::new(self.notify_ready.clone()) + pub fn wake(&self, i: WakerInterest) { + self.waker.wake(i); } pub(crate) fn start( &mut self, - socks: Vec<(Token, StdListener)>, - workers: Vec, + socks: Vec<(Token, MioListener)>, + handles: Vec, ) { let srv = self.srv.take().expect("Can not re-use AcceptInfo"); + let poll = self.poll.take().unwrap(); + let waker = self.waker.clone(); - Accept::start( - self.rx.take().expect("Can not re-use AcceptInfo"), - self.cmd_reg.take().expect("Can not re-use AcceptInfo"), - self.notify_reg.take().expect("Can not re-use AcceptInfo"), - socks, - srv, - workers, - ); + Accept::start(poll, waker, socks, srv, handles); } } +/// poll instance of the server. struct Accept { - poll: mio::Poll, - rx: sync_mpsc::Receiver, - sockets: Slab, - workers: Vec, + poll: Poll, + waker: WakerQueue, + handles: Vec, srv: Server, - timer: (mio::Registration, mio::SetReadiness), next: usize, backpressure: bool, } -const DELTA: usize = 100; -const CMD: mio::Token = mio::Token(0); -const TIMER: mio::Token = mio::Token(1); -const NOTIFY: mio::Token = mio::Token(2); - /// This function defines errors that are per-connection. Which basically /// means that if we get this error from `accept()` system call it means /// next connection might be ready to be accepted. @@ -129,326 +95,290 @@ fn connection_error(e: &io::Error) -> bool { } impl Accept { - #![allow(clippy::too_many_arguments)] pub(crate) fn start( - rx: sync_mpsc::Receiver, - cmd_reg: mio::Registration, - notify_reg: mio::Registration, - socks: Vec<(Token, StdListener)>, + poll: Poll, + waker: WakerQueue, + socks: Vec<(Token, MioListener)>, srv: Server, - workers: Vec, + handles: Vec, ) { + // Accept runs in its own thread and would want to spawn additional futures to current + // actix system. let sys = System::current(); - - // start accept thread - let _ = thread::Builder::new() + thread::Builder::new() .name("actix-server accept loop".to_owned()) .spawn(move || { System::set_current(sys); - let mut accept = Accept::new(rx, socks, workers, srv); - - // Start listening for incoming commands - if let Err(err) = accept.poll.register( - &cmd_reg, - CMD, - mio::Ready::readable(), - mio::PollOpt::edge(), - ) { - panic!("Can not register Registration: {}", err); - } - - // Start listening for notify updates - if let Err(err) = accept.poll.register( - ¬ify_reg, - NOTIFY, - mio::Ready::readable(), - mio::PollOpt::edge(), - ) { - panic!("Can not register Registration: {}", err); - } - - accept.poll(); - }); + let (mut accept, sockets) = + Accept::new_with_sockets(poll, waker, socks, handles, srv); + accept.poll_with(sockets); + }) + .unwrap(); } - fn new( - rx: sync_mpsc::Receiver, - socks: Vec<(Token, StdListener)>, - workers: Vec, + fn new_with_sockets( + poll: Poll, + waker: WakerQueue, + socks: Vec<(Token, MioListener)>, + handles: Vec, srv: Server, - ) -> Accept { - // Create a poll instance - let poll = match mio::Poll::new() { - Ok(poll) => poll, - Err(err) => panic!("Can not create mio::Poll: {}", err), - }; - - // Start accept + ) -> (Accept, Slab) { let mut sockets = Slab::new(); - for (hnd_token, lst) in socks.into_iter() { + for (hnd_token, mut lst) in socks.into_iter() { let addr = lst.local_addr(); - let server = lst.into_listener(); let entry = sockets.vacant_entry(); let token = entry.key(); // Start listening for incoming connections - if let Err(err) = poll.register( - &server, - mio::Token(token + DELTA), - mio::Ready::readable(), - mio::PollOpt::edge(), - ) { - panic!("Can not register io: {}", err); - } + poll.registry() + .register(&mut lst, MioToken(token), Interest::READABLE) + .unwrap_or_else(|e| panic!("Can not register io: {}", e)); entry.insert(ServerSocketInfo { addr, token: hnd_token, - sock: server, + lst, timeout: None, }); } - // Timer - let (tm, tmr) = mio::Registration::new2(); - if let Err(err) = - poll.register(&tm, TIMER, mio::Ready::readable(), mio::PollOpt::edge()) - { - panic!("Can not register Registration: {}", err); - } - - Accept { + let accept = Accept { poll, - rx, - sockets, - workers, + waker, + handles, srv, next: 0, - timer: (tm, tmr), backpressure: false, - } + }; + + (accept, sockets) } - fn poll(&mut self) { - // Create storage for events + fn poll_with(&mut self, mut sockets: Slab) { let mut events = mio::Events::with_capacity(128); loop { - if let Err(err) = self.poll.poll(&mut events, None) { - panic!("Poll error: {}", err); - } + self.poll + .poll(&mut events, None) + .unwrap_or_else(|e| panic!("Poll error: {}", e)); for event in events.iter() { let token = event.token(); match token { - CMD => { - if !self.process_cmd() { - return; + // This is a loop because interests for command from previous version was + // a loop that would try to drain the command channel. It's yet unknown + // if it's necessary/good practice to actively drain the waker queue. + WAKER_TOKEN => 'waker: loop { + // take guard with every iteration so no new interest can be added + // until the current task is done. + let mut guard = self.waker.guard(); + match guard.pop_front() { + // worker notify it becomes available. we may want to recover + // from backpressure. + Some(WakerInterest::WorkerAvailable) => { + drop(guard); + self.maybe_backpressure(&mut sockets, false); + } + // a new worker thread is made and it's handle would be added + // to Accept + Some(WakerInterest::Worker(handle)) => { + drop(guard); + // maybe we want to recover from a backpressure. + self.maybe_backpressure(&mut sockets, false); + 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) => { + drop(guard); + sockets.iter_mut().for_each(|(_, info)| { + match self.deregister(info) { + Ok(_) => info!( + "Paused accepting connections on {}", + info.addr + ), + Err(e) => { + error!("Can not deregister server socket {}", e) + } + } + }); + } + Some(WakerInterest::Resume) => { + drop(guard); + sockets.iter_mut().for_each(|(token, info)| { + self.register_logged(token, info); + }); + } + Some(WakerInterest::Stop) => { + return self.deregister_all(&mut sockets); + } + // waker queue is drained. + None => { + // Reset the WakerQueue before break so it does not grow + // infinitely. + WakerQueue::reset(&mut guard); + break 'waker; + } } - } - TIMER => self.process_timer(), - NOTIFY => self.backpressure(false), + }, _ => { let token = usize::from(token); - if token < DELTA { - continue; - } - self.accept(token - DELTA); + self.accept(&mut sockets, token); } } } } } - fn process_timer(&mut self) { + fn process_timer(&self, sockets: &mut Slab) { let now = Instant::now(); - for (token, info) in self.sockets.iter_mut() { + sockets.iter_mut().for_each(|(token, info)| { + // only the ServerSocketInfo have an associate timeout value was de registered. if let Some(inst) = info.timeout.take() { if now > inst { - if let Err(err) = self.poll.register( - &info.sock, - mio::Token(token + DELTA), - mio::Ready::readable(), - mio::PollOpt::edge(), - ) { - error!("Can not register server socket {}", err); - } else { - info!("Resume accepting connections on {}", info.addr); - } + self.register_logged(token, info); } else { info.timeout = Some(inst); } } - } - } - - fn process_cmd(&mut self) -> bool { - loop { - match self.rx.try_recv() { - Ok(cmd) => match cmd { - Command::Pause => { - for (_, info) in self.sockets.iter_mut() { - if let Err(err) = self.poll.deregister(&info.sock) { - error!("Can not deregister server socket {}", err); - } else { - info!("Paused accepting connections on {}", info.addr); - } - } - } - Command::Resume => { - for (token, info) in self.sockets.iter() { - if let Err(err) = self.register(token, info) { - error!("Can not resume socket accept process: {}", err); - } else { - info!( - "Accepting connections on {} has been resumed", - info.addr - ); - } - } - } - Command::Stop => { - for (_, info) in self.sockets.iter() { - let _ = self.poll.deregister(&info.sock); - } - return false; - } - Command::Worker(worker) => { - self.backpressure(false); - self.workers.push(worker); - } - }, - Err(err) => match err { - sync_mpsc::TryRecvError::Empty => break, - sync_mpsc::TryRecvError::Disconnected => { - for (_, info) in self.sockets.iter() { - let _ = self.poll.deregister(&info.sock); - } - return false; - } - }, - } - } - true + }); } #[cfg(not(target_os = "windows"))] - fn register(&self, token: usize, info: &ServerSocketInfo) -> io::Result<()> { - self.poll.register( - &info.sock, - mio::Token(token + DELTA), - mio::Ready::readable(), - mio::PollOpt::edge(), - ) + fn register(&self, token: usize, info: &mut ServerSocketInfo) -> io::Result<()> { + self.poll + .registry() + .register(&mut info.lst, MioToken(token), Interest::READABLE) } #[cfg(target_os = "windows")] - fn register(&self, token: usize, info: &ServerSocketInfo) -> io::Result<()> { + fn register(&self, token: usize, info: &mut ServerSocketInfo) -> io::Result<()> { // On windows, calling register without deregister cause an error. // See https://github.com/actix/actix-web/issues/905 // Calling reregister seems to fix the issue. self.poll - .register( - &info.sock, - mio::Token(token + DELTA), - mio::Ready::readable(), - mio::PollOpt::edge(), - ) + .registry() + .register(&mut info.lst, mio::Token(token), Interest::READABLE) .or_else(|_| { - self.poll.reregister( - &info.sock, - mio::Token(token + DELTA), - mio::Ready::readable(), - mio::PollOpt::edge(), + self.poll.registry().reregister( + &mut info.lst, + mio::Token(token), + Interest::READABLE, ) }) } - fn backpressure(&mut self, on: bool) { + fn register_logged(&self, token: usize, info: &mut ServerSocketInfo) { + match self.register(token, info) { + Ok(_) => info!("Resume accepting connections on {}", info.addr), + Err(e) => error!("Can not register server socket {}", e), + } + } + + fn deregister(&self, info: &mut ServerSocketInfo) -> io::Result<()> { + self.poll.registry().deregister(&mut info.lst) + } + + fn deregister_all(&self, sockets: &mut Slab) { + sockets.iter_mut().for_each(|(_, info)| { + info!("Accepting connections on {} has been paused", info.addr); + let _ = self.deregister(info); + }); + } + + fn maybe_backpressure(&mut self, sockets: &mut Slab, on: bool) { if self.backpressure { if !on { self.backpressure = false; - for (token, info) in self.sockets.iter() { + for (token, info) in sockets.iter_mut() { if info.timeout.is_some() { // socket will attempt to re-register itself when its timeout completes continue; } - - if let Err(err) = self.register(token, info) { - error!("Can not resume socket accept process: {}", err); - } else { - info!("Accepting connections on {} has been resumed", info.addr); - } + self.register_logged(token, info); } } } else if on { self.backpressure = true; - for (_, info) in self.sockets.iter() { - let _ = self.poll.deregister(&info.sock); - info!("Accepting connections on {} has been paused", info.addr); - } + self.deregister_all(sockets); } } - fn accept_one(&mut self, mut msg: Conn) { + fn accept_one(&mut self, sockets: &mut Slab, mut msg: Conn) { if self.backpressure { - while !self.workers.is_empty() { - match self.workers[self.next].send(msg) { - Ok(_) => (), + while !self.handles.is_empty() { + match self.handles[self.next].send(msg) { + Ok(_) => { + self.set_next(); + break; + } Err(tmp) => { - self.srv.worker_faulted(self.workers[self.next].idx); + // worker lost contact and could be gone. a message is sent to + // `ServerBuilder` future to notify it a new worker should be made. + // after that remove the fault worker. + self.srv.worker_faulted(self.handles[self.next].idx); msg = tmp; - self.workers.swap_remove(self.next); - if self.workers.is_empty() { + self.handles.swap_remove(self.next); + if self.handles.is_empty() { error!("No workers"); return; - } else if self.workers.len() <= self.next { + } else if self.handles.len() <= self.next { self.next = 0; } continue; } } - self.next = (self.next + 1) % self.workers.len(); - break; } } else { let mut idx = 0; - while idx < self.workers.len() { + while idx < self.handles.len() { idx += 1; - if self.workers[self.next].available() { - match self.workers[self.next].send(msg) { + if self.handles[self.next].available() { + match self.handles[self.next].send(msg) { Ok(_) => { - self.next = (self.next + 1) % self.workers.len(); + self.set_next(); return; } + // worker lost contact and could be gone. a message is sent to + // `ServerBuilder` future to notify it a new worker should be made. + // after that remove the fault worker and enter backpressure if necessary. Err(tmp) => { - self.srv.worker_faulted(self.workers[self.next].idx); + self.srv.worker_faulted(self.handles[self.next].idx); msg = tmp; - self.workers.swap_remove(self.next); - if self.workers.is_empty() { + self.handles.swap_remove(self.next); + if self.handles.is_empty() { error!("No workers"); - self.backpressure(true); + self.maybe_backpressure(sockets, true); return; - } else if self.workers.len() <= self.next { + } else if self.handles.len() <= self.next { self.next = 0; } continue; } } } - self.next = (self.next + 1) % self.workers.len(); + self.set_next(); } // enable backpressure - self.backpressure(true); - self.accept_one(msg); + self.maybe_backpressure(sockets, true); + self.accept_one(sockets, msg); } } - fn accept(&mut self, token: usize) { + // set next worker handle that would accept work. + fn set_next(&mut self) { + self.next = (self.next + 1) % self.handles.len(); + } + + fn accept(&mut self, sockets: &mut Slab, token: usize) { loop { - let msg = if let Some(info) = self.sockets.get_mut(token) { - match info.sock.accept() { + let msg = if let Some(info) = sockets.get_mut(token) { + match info.lst.accept() { Ok(Some((io, addr))) => Conn { io, token: info.token, @@ -458,18 +388,22 @@ impl Accept { Err(ref e) if e.kind() == io::ErrorKind::WouldBlock => return, Err(ref e) if connection_error(e) => continue, Err(e) => { + // deregister listener temporary error!("Error accepting connection: {}", e); - if let Err(err) = self.poll.deregister(&info.sock) { + if let Err(err) = self.deregister(info) { error!("Can not deregister server socket {}", err); } - // sleep after error + // sleep after error. write the timeout to socket info as later the poll + // would need it mark which socket and when it's listener should be + // registered. info.timeout = Some(Instant::now() + Duration::from_millis(500)); - let r = self.timer.1.clone(); + // after the sleep a Timer interest is sent to Accept Poll + let waker = self.waker.clone(); System::current().arbiter().send(Box::pin(async move { - delay_until(Instant::now() + Duration::from_millis(510)).await; - let _ = r.set_readiness(mio::Ready::readable()); + sleep_until(Instant::now() + Duration::from_millis(510)).await; + waker.wake(WakerInterest::Timer); })); return; } @@ -478,7 +412,7 @@ impl Accept { return; }; - self.accept_one(msg); + self.accept_one(sockets, msg); } } } diff --git a/actix-server/src/builder.rs b/actix-server/src/builder.rs index 64a45df9..51dd0eda 100644 --- a/actix-server/src/builder.rs +++ b/actix-server/src/builder.rs @@ -1,36 +1,35 @@ +use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; use std::time::Duration; -use std::{io, mem, net}; +use std::{io, mem}; use actix_rt::net::TcpStream; -use actix_rt::time::{delay_until, Instant}; +use actix_rt::time::{sleep_until, Instant}; use actix_rt::{spawn, System}; -use futures_channel::mpsc::{unbounded, UnboundedReceiver}; -use futures_channel::oneshot; -use futures_util::future::ready; -use futures_util::stream::FuturesUnordered; -use futures_util::{future::Future, ready, stream::Stream, FutureExt, StreamExt}; use log::{error, info}; -use socket2::{Domain, Protocol, Socket, Type}; +use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; +use tokio::sync::oneshot; -use crate::accept::{AcceptLoop, AcceptNotify, Command}; +use crate::accept::AcceptLoop; use crate::config::{ConfiguredService, ServiceConfig}; use crate::server::{Server, ServerCommand}; use crate::service::{InternalServiceFactory, ServiceFactory, StreamNewService}; use crate::signals::{Signal, Signals}; -use crate::socket::StdListener; -use crate::worker::{self, Worker, WorkerAvailability, WorkerClient}; -use crate::Token; +use crate::socket::{MioListener, StdSocketAddr, StdTcpListener, ToSocketAddrs}; +use crate::socket::{MioTcpListener, MioTcpSocket}; +use crate::waker_queue::{WakerInterest, WakerQueue}; +use crate::worker::{self, Worker, WorkerAvailability, WorkerHandle}; +use crate::{join_all, Token}; /// Server builder pub struct ServerBuilder { threads: usize, token: Token, - backlog: i32, - workers: Vec<(usize, WorkerClient)>, + backlog: u32, + handles: Vec<(usize, WorkerHandle)>, services: Vec>, - sockets: Vec<(Token, String, StdListener)>, + sockets: Vec<(Token, String, MioListener)>, accept: AcceptLoop, exit: bool, shutdown_timeout: Duration, @@ -49,13 +48,13 @@ impl Default for ServerBuilder { impl ServerBuilder { /// Create new Server builder instance pub fn new() -> ServerBuilder { - let (tx, rx) = unbounded(); + let (tx, rx) = unbounded_channel(); let server = Server::new(tx); ServerBuilder { threads: num_cpus::get(), - token: Token(0), - workers: Vec::new(), + token: Token::default(), + handles: Vec::new(), services: Vec::new(), sockets: Vec::new(), accept: AcceptLoop::new(server.clone()), @@ -89,7 +88,7 @@ impl ServerBuilder { /// Generally set in the 64-2048 range. Default value is 2048. /// /// This method should be called before `bind()` method call. - pub fn backlog(mut self, num: i32) -> Self { + pub fn backlog(mut self, num: u32) -> Self { self.backlog = num; self } @@ -147,7 +146,7 @@ impl ServerBuilder { for (name, lst) in cfg.services { let token = self.token.next(); srv.stream(token, name.clone(), lst.local_addr()?); - self.sockets.push((token, name, StdListener::Tcp(lst))); + self.sockets.push((token, name, MioListener::Tcp(lst))); } self.services.push(Box::new(srv)); } @@ -160,7 +159,7 @@ impl ServerBuilder { pub fn bind>(mut self, name: N, addr: U, factory: F) -> io::Result where F: ServiceFactory, - U: net::ToSocketAddrs, + U: ToSocketAddrs, { let sockets = bind_addr(addr, self.backlog)?; @@ -173,12 +172,12 @@ impl ServerBuilder { lst.local_addr()?, )); self.sockets - .push((token, name.as_ref().to_string(), StdListener::Tcp(lst))); + .push((token, name.as_ref().to_string(), MioListener::Tcp(lst))); } Ok(self) } - #[cfg(all(unix))] + #[cfg(unix)] /// Add new unix domain service to the server. pub fn bind_uds(self, name: N, addr: U, factory: F) -> io::Result where @@ -186,8 +185,6 @@ impl ServerBuilder { N: AsRef, U: AsRef, { - use std::os::unix::net::UnixListener; - // The path must not exist when we try to bind. // Try to remove it to avoid bind error. if let Err(e) = std::fs::remove_file(addr.as_ref()) { @@ -197,26 +194,27 @@ impl ServerBuilder { } } - let lst = UnixListener::bind(addr)?; + let lst = crate::socket::StdUnixListener::bind(addr)?; self.listen_uds(name, lst, factory) } - #[cfg(all(unix))] + #[cfg(unix)] /// Add new unix domain service to the server. /// Useful when running as a systemd service and /// a socket FD can be acquired using the systemd crate. pub fn listen_uds>( mut self, name: N, - lst: std::os::unix::net::UnixListener, + lst: crate::socket::StdUnixListener, factory: F, ) -> io::Result where F: ServiceFactory, { - use std::net::{IpAddr, Ipv4Addr, SocketAddr}; + use std::net::{IpAddr, Ipv4Addr}; + lst.set_nonblocking(true)?; let token = self.token.next(); - let addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 8080); + let addr = StdSocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 8080); self.services.push(StreamNewService::create( name.as_ref().to_string(), token, @@ -224,7 +222,7 @@ impl ServerBuilder { addr, )); self.sockets - .push((token, name.as_ref().to_string(), StdListener::Uds(lst))); + .push((token, name.as_ref().to_string(), MioListener::from(lst))); Ok(self) } @@ -232,21 +230,25 @@ impl ServerBuilder { pub fn listen>( mut self, name: N, - lst: net::TcpListener, + lst: StdTcpListener, factory: F, ) -> io::Result where F: ServiceFactory, { + lst.set_nonblocking(true)?; + let addr = lst.local_addr()?; + let token = self.token.next(); self.services.push(StreamNewService::create( name.as_ref().to_string(), token, factory, - lst.local_addr()?, + addr, )); + self.sockets - .push((token, name.as_ref().to_string(), StdListener::Tcp(lst))); + .push((token, name.as_ref().to_string(), MioListener::from(lst))); Ok(self) } @@ -263,12 +265,12 @@ impl ServerBuilder { info!("Starting {} workers", self.threads); // start workers - let workers = (0..self.threads) + let handles = (0..self.threads) .map(|idx| { - let worker = self.start_worker(idx, self.accept.get_notify()); - self.workers.push((idx, worker.clone())); + let handle = self.start_worker(idx, self.accept.waker_owned()); + self.handles.push((idx, handle.clone())); - worker + handle }) .collect(); @@ -281,7 +283,7 @@ impl ServerBuilder { .into_iter() .map(|t| (t.0, t.2)) .collect(), - workers, + handles, ); // handle signals @@ -296,10 +298,9 @@ impl ServerBuilder { } } - fn start_worker(&self, idx: usize, notify: AcceptNotify) -> WorkerClient { - let avail = WorkerAvailability::new(notify); - let services: Vec> = - self.services.iter().map(|v| v.clone_factory()).collect(); + fn start_worker(&self, idx: usize, waker: WakerQueue) -> WorkerHandle { + let avail = WorkerAvailability::new(waker); + let services = self.services.iter().map(|v| v.clone_factory()).collect(); Worker::start(idx, services, avail, self.shutdown_timeout) } @@ -307,11 +308,11 @@ impl ServerBuilder { fn handle_cmd(&mut self, item: ServerCommand) { match item { ServerCommand::Pause(tx) => { - self.accept.send(Command::Pause); + self.accept.wake(WakerInterest::Pause); let _ = tx.send(()); } ServerCommand::Resume(tx) => { - self.accept.send(Command::Resume); + self.accept.wake(WakerInterest::Resume); let _ = tx.send(()); } ServerCommand::Signal(sig) => { @@ -355,50 +356,41 @@ impl ServerBuilder { let exit = self.exit; // stop accept thread - self.accept.send(Command::Stop); + self.accept.wake(WakerInterest::Stop); let notify = std::mem::take(&mut self.notify); // stop workers - if !self.workers.is_empty() && graceful { - spawn( - self.workers - .iter() - .map(move |worker| worker.1.stop(graceful)) - .collect::>() - .collect::>() - .then(move |_| { - if let Some(tx) = completion { - let _ = tx.send(()); - } - for tx in notify { - let _ = tx.send(()); - } - if exit { - spawn( - async { - delay_until( - Instant::now() + Duration::from_millis(300), - ) - .await; - System::current().stop(); - } - .boxed(), - ); - } - ready(()) - }), - ) + if !self.handles.is_empty() && graceful { + let iter = self + .handles + .iter() + .map(move |worker| worker.1.stop(graceful)) + .collect(); + + let fut = join_all(iter); + + spawn(async move { + let _ = fut.await; + if let Some(tx) = completion { + let _ = tx.send(()); + } + for tx in notify { + let _ = tx.send(()); + } + if exit { + spawn(async { + sleep_until(Instant::now() + Duration::from_millis(300)).await; + System::current().stop(); + }); + } + }) } else { // we need to stop system if server was spawned if self.exit { - spawn( - delay_until(Instant::now() + Duration::from_millis(300)).then( - |_| { - System::current().stop(); - ready(()) - }, - ), - ); + spawn(async { + sleep_until(Instant::now() + Duration::from_millis(300)).await; + System::current().stop(); + }); } if let Some(tx) = completion { let _ = tx.send(()); @@ -410,9 +402,9 @@ impl ServerBuilder { } ServerCommand::WorkerFaulted(idx) => { let mut found = false; - for i in 0..self.workers.len() { - if self.workers[i].0 == idx { - self.workers.swap_remove(i); + for i in 0..self.handles.len() { + if self.handles[i].0 == idx { + self.handles.swap_remove(i); found = true; break; } @@ -421,10 +413,10 @@ impl ServerBuilder { if found { error!("Worker has died {:?}, restarting", idx); - let mut new_idx = self.workers.len(); + let mut new_idx = self.handles.len(); 'found: loop { - for i in 0..self.workers.len() { - if self.workers[i].0 == new_idx { + for i in 0..self.handles.len() { + if self.handles[i].0 == new_idx { new_idx += 1; continue 'found; } @@ -432,9 +424,9 @@ impl ServerBuilder { break; } - let worker = self.start_worker(new_idx, self.accept.get_notify()); - self.workers.push((new_idx, worker.clone())); - self.accept.send(Command::Worker(worker)); + let handle = self.start_worker(new_idx, self.accept.waker_owned()); + self.handles.push((new_idx, handle.clone())); + self.accept.wake(WakerInterest::Worker(handle)); } } } @@ -446,20 +438,18 @@ impl Future for ServerBuilder { fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { - match ready!(Pin::new(&mut self.cmd).poll_next(cx)) { - Some(it) => self.as_mut().get_mut().handle_cmd(it), - None => { - return Poll::Pending; - } + match Pin::new(&mut self.cmd).poll_recv(cx) { + Poll::Ready(Some(it)) => self.as_mut().get_mut().handle_cmd(it), + _ => return Poll::Pending, } } } } -pub(super) fn bind_addr( +pub(super) fn bind_addr( addr: S, - backlog: i32, -) -> io::Result> { + backlog: u32, +) -> io::Result> { let mut err = None; let mut succ = false; let mut sockets = Vec::new(); @@ -487,14 +477,13 @@ pub(super) fn bind_addr( } } -fn create_tcp_listener(addr: net::SocketAddr, backlog: i32) -> io::Result { - let domain = match addr { - net::SocketAddr::V4(_) => Domain::ipv4(), - net::SocketAddr::V6(_) => Domain::ipv6(), +fn create_tcp_listener(addr: StdSocketAddr, backlog: u32) -> io::Result { + let socket = match addr { + StdSocketAddr::V4(_) => MioTcpSocket::new_v4()?, + StdSocketAddr::V6(_) => MioTcpSocket::new_v6()?, }; - let socket = Socket::new(domain, Type::stream(), Some(Protocol::tcp()))?; - socket.set_reuse_address(true)?; - socket.bind(&addr.into())?; - socket.listen(backlog)?; - Ok(socket.into_tcp_listener()) + + socket.set_reuseaddr(true)?; + socket.bind(addr)?; + socket.listen(backlog) } diff --git a/actix-server/src/config.rs b/actix-server/src/config.rs index a1315a72..20270a2f 100644 --- a/actix-server/src/config.rs +++ b/actix-server/src/config.rs @@ -1,5 +1,6 @@ use std::collections::HashMap; -use std::{fmt, io, net}; +use std::future::Future; +use std::{fmt, io}; use actix_rt::net::TcpStream; use actix_service::{ @@ -7,23 +8,23 @@ use actix_service::{ ServiceFactory as BaseServiceFactory, }; use actix_utils::counter::CounterGuard; -use futures_util::future::{ok, Future, FutureExt, LocalBoxFuture}; +use futures_core::future::LocalBoxFuture; use log::error; -use super::builder::bind_addr; -use super::service::{BoxedServerService, InternalServiceFactory, StreamService}; -use super::Token; -use crate::socket::StdStream; +use crate::builder::bind_addr; +use crate::service::{BoxedServerService, InternalServiceFactory, StreamService}; +use crate::socket::{MioStream, MioTcpListener, StdSocketAddr, StdTcpListener, ToSocketAddrs}; +use crate::{ready, Token}; pub struct ServiceConfig { - pub(crate) services: Vec<(String, net::TcpListener)>, + pub(crate) services: Vec<(String, MioTcpListener)>, pub(crate) apply: Option>, pub(crate) threads: usize, - pub(crate) backlog: i32, + pub(crate) backlog: u32, } impl ServiceConfig { - pub(super) fn new(threads: usize, backlog: i32) -> ServiceConfig { + pub(super) fn new(threads: usize, backlog: u32) -> ServiceConfig { ServiceConfig { threads, backlog, @@ -43,24 +44,20 @@ impl ServiceConfig { /// Add new service to server pub fn bind>(&mut self, name: N, addr: U) -> io::Result<&mut Self> where - U: net::ToSocketAddrs, + U: ToSocketAddrs, { let sockets = bind_addr(addr, self.backlog)?; for lst in sockets { - self.listen(name.as_ref(), lst); + self._listen(name.as_ref(), lst); } Ok(self) } /// Add new service to server - pub fn listen>(&mut self, name: N, lst: net::TcpListener) -> &mut Self { - if self.apply.is_none() { - self.apply = Some(Box::new(not_configured)); - } - self.services.push((name.as_ref().to_string(), lst)); - self + pub fn listen>(&mut self, name: N, lst: StdTcpListener) -> &mut Self { + self._listen(name, MioTcpListener::from_std(lst)) } /// Register service configuration function. This function get called @@ -72,11 +69,19 @@ impl ServiceConfig { self.apply = Some(Box::new(f)); Ok(()) } + + fn _listen>(&mut self, name: N, lst: MioTcpListener) -> &mut Self { + if self.apply.is_none() { + self.apply = Some(Box::new(not_configured)); + } + self.services.push((name.as_ref().to_string(), lst)); + self + } } pub(super) struct ConfiguredService { rt: Box, - names: HashMap, + names: HashMap, topics: HashMap, services: Vec, } @@ -91,7 +96,7 @@ impl ConfiguredService { } } - pub(super) fn stream(&mut self, token: Token, name: String, addr: net::SocketAddr) { + pub(super) fn stream(&mut self, token: Token, name: String, addr: StdSocketAddr) { self.names.insert(token, (name.clone(), addr)); self.topics.insert(name, token); self.services.push(token); @@ -121,7 +126,7 @@ impl InternalServiceFactory for ConfiguredService { let tokens = self.services.clone(); // construct services - async move { + Box::pin(async move { let mut services = rt.services; // TODO: Proper error handling here for f in rt.onstart.into_iter() { @@ -146,14 +151,13 @@ impl InternalServiceFactory for ConfiguredService { token, Box::new(StreamService::new(fn_service(move |_: TcpStream| { error!("Service {:?} is not configured", name); - ok::<_, ()>(()) + ready::>(Ok(())) }))), )); }; } Ok(res) - } - .boxed_local() + }) } } @@ -233,13 +237,13 @@ impl ServiceRuntime { where F: Future + 'static, { - self.onstart.push(fut.boxed_local()) + self.onstart.push(Box::pin(fut)) } } type BoxedNewService = Box< dyn BaseServiceFactory< - (Option, StdStream), + (Option, MioStream), Response = (), Error = (), InitError = (), @@ -253,7 +257,7 @@ struct ServiceFactory { inner: T, } -impl BaseServiceFactory<(Option, StdStream)> for ServiceFactory +impl BaseServiceFactory<(Option, MioStream)> for ServiceFactory where T: BaseServiceFactory, T::Future: 'static, @@ -270,7 +274,7 @@ where fn new_service(&self, _: ()) -> Self::Future { let fut = self.inner.new_service(()); - async move { + Box::pin(async move { match fut.await { Ok(s) => Ok(Box::new(StreamService::new(s)) as BoxedServerService), Err(e) => { @@ -278,7 +282,6 @@ where Err(()) } } - } - .boxed_local() + }) } } diff --git a/actix-server/src/lib.rs b/actix-server/src/lib.rs index d7a7c242..64aca7e4 100644 --- a/actix-server/src/lib.rs +++ b/actix-server/src/lib.rs @@ -11,6 +11,7 @@ mod server; mod service; mod signals; mod socket; +mod waker_queue; mod worker; pub use self::builder::ServerBuilder; @@ -21,11 +22,25 @@ pub use self::service::ServiceFactory; #[doc(hidden)] pub use self::socket::FromStream; +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; + /// Socket ID token #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash)] pub(crate) struct Token(usize); +impl Default for Token { + fn default() -> Self { + Self::new() + } +} + impl Token { + fn new() -> Self { + Self(0) + } + pub(crate) fn next(&mut self) -> Token { let token = Token(self.0); self.0 += 1; @@ -37,3 +52,90 @@ impl Token { pub fn new() -> ServerBuilder { ServerBuilder::default() } + +// temporary Ready type for std::future::{ready, Ready}; Can be removed when MSRV surpass 1.48 +#[doc(hidden)] +pub struct Ready(Option); + +pub(crate) fn ready(t: T) -> Ready { + Ready(Some(t)) +} + +impl Unpin for Ready {} + +impl Future for Ready { + type Output = T; + + fn poll(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll { + Poll::Ready(self.get_mut().0.take().unwrap()) + } +} + +// a poor man's join future. joined future is only used when starting/stopping the server. +// pin_project and pinned futures are overkill for this task. +pub(crate) struct JoinAll { + fut: Vec>, +} + +pub(crate) fn join_all(fut: Vec + 'static>) -> JoinAll { + let fut = fut + .into_iter() + .map(|f| JoinFuture::Future(Box::pin(f))) + .collect(); + + JoinAll { fut } +} + +enum JoinFuture { + Future(Pin>>), + Result(Option), +} + +impl Unpin for JoinAll {} + +impl Future for JoinAll { + type Output = Vec; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let mut ready = true; + + let this = self.get_mut(); + for fut in this.fut.iter_mut() { + if let JoinFuture::Future(f) = fut { + match f.as_mut().poll(cx) { + Poll::Ready(t) => { + *fut = JoinFuture::Result(Some(t)); + } + Poll::Pending => ready = false, + } + } + } + + if ready { + let mut res = Vec::new(); + for fut in this.fut.iter_mut() { + if let JoinFuture::Result(f) = fut { + res.push(f.take().unwrap()); + } + } + + Poll::Ready(res) + } else { + Poll::Pending + } + } +} + +#[cfg(test)] +mod test { + use super::*; + + #[actix_rt::test] + async fn test_join_all() { + let futs = vec![ready(Ok(1)), ready(Err(3)), ready(Ok(9))]; + let mut res = join_all(futs).await.into_iter(); + assert_eq!(Ok(1), res.next().unwrap()); + assert_eq!(Err(3), res.next().unwrap()); + assert_eq!(Ok(9), res.next().unwrap()); + } +} diff --git a/actix-server/src/server.rs b/actix-server/src/server.rs index b29a9e02..6b0d0aea 100644 --- a/actix-server/src/server.rs +++ b/actix-server/src/server.rs @@ -3,9 +3,8 @@ use std::io; use std::pin::Pin; use std::task::{Context, Poll}; -use futures_channel::mpsc::UnboundedSender; -use futures_channel::oneshot; -use futures_util::FutureExt; +use tokio::sync::mpsc::UnboundedSender; +use tokio::sync::oneshot; use crate::builder::ServerBuilder; use crate::signals::Signal; @@ -42,11 +41,11 @@ impl Server { } pub(crate) fn signal(&self, sig: Signal) { - let _ = self.0.unbounded_send(ServerCommand::Signal(sig)); + let _ = self.0.send(ServerCommand::Signal(sig)); } pub(crate) fn worker_faulted(&self, idx: usize) { - let _ = self.0.unbounded_send(ServerCommand::WorkerFaulted(idx)); + let _ = self.0.send(ServerCommand::WorkerFaulted(idx)); } /// Pause accepting incoming connections @@ -55,15 +54,19 @@ impl Server { /// All opened connection remains active. pub fn pause(&self) -> impl Future { let (tx, rx) = oneshot::channel(); - let _ = self.0.unbounded_send(ServerCommand::Pause(tx)); - rx.map(|_| ()) + let _ = self.0.send(ServerCommand::Pause(tx)); + async { + let _ = rx.await; + } } /// Resume accepting incoming connections pub fn resume(&self) -> impl Future { let (tx, rx) = oneshot::channel(); - let _ = self.0.unbounded_send(ServerCommand::Resume(tx)); - rx.map(|_| ()) + let _ = self.0.send(ServerCommand::Resume(tx)); + async { + let _ = rx.await; + } } /// Stop incoming connection processing, stop all workers and exit. @@ -71,11 +74,13 @@ impl Server { /// If server starts with `spawn()` method, then spawned thread get terminated. pub fn stop(&self, graceful: bool) -> impl Future { let (tx, rx) = oneshot::channel(); - let _ = self.0.unbounded_send(ServerCommand::Stop { + let _ = self.0.send(ServerCommand::Stop { graceful, completion: Some(tx), }); - rx.map(|_| ()) + async { + let _ = rx.await; + } } } @@ -93,7 +98,7 @@ impl Future for Server { if this.1.is_none() { let (tx, rx) = oneshot::channel(); - if this.0.unbounded_send(ServerCommand::Notify(tx)).is_err() { + if this.0.send(ServerCommand::Notify(tx)).is_err() { return Poll::Ready(Ok(())); } this.1 = Some(rx); @@ -101,8 +106,7 @@ impl Future for Server { match Pin::new(this.1.as_mut().unwrap()).poll(cx) { Poll::Pending => Poll::Pending, - Poll::Ready(Ok(_)) => Poll::Ready(Ok(())), - Poll::Ready(Err(_)) => Poll::Ready(Ok(())), + Poll::Ready(_) => Poll::Ready(Ok(())), } } } diff --git a/actix-server/src/service.rs b/actix-server/src/service.rs index 569ce048..04b7dce8 100644 --- a/actix-server/src/service.rs +++ b/actix-server/src/service.rs @@ -2,15 +2,13 @@ use std::marker::PhantomData; use std::net::SocketAddr; use std::task::{Context, Poll}; -use actix_rt::spawn; use actix_service::{Service, ServiceFactory as BaseServiceFactory}; use actix_utils::counter::CounterGuard; -use futures_util::future::{err, ok, LocalBoxFuture, Ready}; -use futures_util::{FutureExt, TryFutureExt}; +use futures_core::future::LocalBoxFuture; use log::error; -use super::Token; -use crate::socket::{FromStream, StdStream}; +use crate::socket::{FromStream, MioStream}; +use crate::{ready, Ready, Token}; pub trait ServiceFactory: Send + Clone + 'static { type Factory: BaseServiceFactory; @@ -28,7 +26,7 @@ pub(crate) trait InternalServiceFactory: Send { pub(crate) type BoxedServerService = Box< dyn Service< - (Option, StdStream), + (Option, MioStream), Response = (), Error = (), Future = Ready>, @@ -49,7 +47,7 @@ impl StreamService { } } -impl Service<(Option, StdStream)> for StreamService +impl Service<(Option, MioStream)> for StreamService where S: Service, S::Future: 'static, @@ -64,21 +62,21 @@ where self.service.poll_ready(ctx).map_err(|_| ()) } - fn call(&mut self, (guard, req): (Option, StdStream)) -> Self::Future { - match FromStream::from_stdstream(req) { + fn call(&mut self, (guard, req): (Option, MioStream)) -> Self::Future { + ready(match FromStream::from_mio(req) { Ok(stream) => { let f = self.service.call(stream); - spawn(async move { + actix_rt::spawn(async move { let _ = f.await; drop(guard); }); - ok(()) + Ok(()) } Err(e) => { error!("Can not convert to an async tcp stream: {}", e); - err(()) + Err(()) } - } + }) } } @@ -132,15 +130,16 @@ where fn create(&self) -> LocalBoxFuture<'static, Result, ()>> { let token = self.token; - self.inner - .create() - .new_service(()) - .map_err(|_| ()) - .map_ok(move |inner| { - let service: BoxedServerService = Box::new(StreamService::new(inner)); - vec![(token, service)] - }) - .boxed_local() + let fut = self.inner.create().new_service(()); + Box::pin(async move { + match fut.await { + Ok(inner) => { + let service = Box::new(StreamService::new(inner)) as _; + Ok(vec![(token, service)]) + } + Err(_) => Err(()), + } + }) } } diff --git a/actix-server/src/signals.rs b/actix-server/src/signals.rs index 4fc51fc1..ea1de47e 100644 --- a/actix-server/src/signals.rs +++ b/actix-server/src/signals.rs @@ -2,7 +2,7 @@ use std::future::Future; use std::pin::Pin; use std::task::{Context, Poll}; -use futures_util::future::lazy; +use futures_core::future::LocalBoxFuture; use crate::server::Server; @@ -23,48 +23,51 @@ pub(crate) enum Signal { pub(crate) struct Signals { srv: Server, #[cfg(not(unix))] - stream: Pin>>>, + signals: LocalBoxFuture<'static, std::io::Result<()>>, #[cfg(unix)] - streams: Vec<(Signal, actix_rt::signal::unix::Signal)>, + signals: Vec<(Signal, LocalBoxFuture<'static, ()>)>, } impl Signals { pub(crate) fn start(srv: Server) { - actix_rt::spawn(lazy(|_| { - #[cfg(not(unix))] - { - actix_rt::spawn(Signals { - srv, - stream: Box::pin(actix_rt::signal::ctrl_c()), - }); - } - #[cfg(unix)] - { - use actix_rt::signal::unix; + #[cfg(not(unix))] + { + actix_rt::spawn(Signals { + srv, + signals: Box::pin(actix_rt::signal::ctrl_c()), + }); + } + #[cfg(unix)] + { + use actix_rt::signal::unix; - let mut streams = Vec::new(); + let sig_map = [ + (unix::SignalKind::interrupt(), Signal::Int), + (unix::SignalKind::hangup(), Signal::Hup), + (unix::SignalKind::terminate(), Signal::Term), + (unix::SignalKind::quit(), Signal::Quit), + ]; - let sig_map = [ - (unix::SignalKind::interrupt(), Signal::Int), - (unix::SignalKind::hangup(), Signal::Hup), - (unix::SignalKind::terminate(), Signal::Term), - (unix::SignalKind::quit(), Signal::Quit), - ]; + let mut signals = Vec::new(); - for (kind, sig) in sig_map.iter() { - match unix::signal(*kind) { - Ok(stream) => streams.push((*sig, stream)), - Err(e) => log::error!( - "Can not initialize stream handler for {:?} err: {}", - sig, - e - ), + for (kind, sig) in sig_map.iter() { + match unix::signal(*kind) { + Ok(mut stream) => { + let fut = Box::pin(async move { + let _ = stream.recv().await; + }) as _; + signals.push((*sig, fut)); } + Err(e) => log::error!( + "Can not initialize stream handler for {:?} err: {}", + sig, + e + ), } - - actix_rt::spawn(Signals { srv, streams }) } - })); + + actix_rt::spawn(Signals { srv, signals }); + } } } @@ -73,25 +76,20 @@ impl Future for Signals { fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { #[cfg(not(unix))] - match Pin::new(&mut self.stream).poll(cx) { + match self.signals.as_mut().poll(cx) { Poll::Ready(_) => { self.srv.signal(Signal::Int); Poll::Ready(()) } - Poll::Pending => return Poll::Pending, + Poll::Pending => Poll::Pending, } #[cfg(unix)] { - for idx in 0..self.streams.len() { - loop { - match self.streams[idx].1.poll_recv(cx) { - Poll::Ready(None) => return Poll::Ready(()), - Poll::Pending => break, - Poll::Ready(Some(_)) => { - let sig = self.streams[idx].0; - self.srv.signal(sig); - } - } + for (sig, fut) in self.signals.iter_mut() { + if fut.as_mut().poll(cx).is_ready() { + let sig = *sig; + self.srv.signal(sig); + return Poll::Ready(()); } } Poll::Pending diff --git a/actix-server/src/socket.rs b/actix-server/src/socket.rs index 3025660a..416e253b 100644 --- a/actix-server/src/socket.rs +++ b/actix-server/src/socket.rs @@ -1,135 +1,91 @@ -use std::{fmt, io, net}; +pub(crate) use std::net::{ + SocketAddr as StdSocketAddr, TcpListener as StdTcpListener, ToSocketAddrs, +}; + +pub(crate) use mio::net::{TcpListener as MioTcpListener, TcpSocket as MioTcpSocket}; +#[cfg(unix)] +pub(crate) use { + mio::net::UnixListener as MioUnixListener, + std::os::unix::net::UnixListener as StdUnixListener, +}; + +use std::{fmt, io}; -use actix_codec::{AsyncRead, AsyncWrite}; use actix_rt::net::TcpStream; +use mio::event::Source; +use mio::net::TcpStream as MioTcpStream; +use mio::{Interest, Registry, Token}; -pub(crate) enum StdListener { - Tcp(net::TcpListener), - #[cfg(all(unix))] - Uds(std::os::unix::net::UnixListener), +#[cfg(windows)] +use std::os::windows::io::{FromRawSocket, IntoRawSocket}; +#[cfg(unix)] +use { + actix_rt::net::UnixStream, + mio::net::{SocketAddr as MioSocketAddr, UnixStream as MioUnixStream}, + std::os::unix::io::{FromRawFd, IntoRawFd}, +}; + +pub(crate) enum MioListener { + Tcp(MioTcpListener), + #[cfg(unix)] + Uds(MioUnixListener), } -pub(crate) enum SocketAddr { - Tcp(net::SocketAddr), - #[cfg(all(unix))] - Uds(std::os::unix::net::SocketAddr), -} - -impl fmt::Display for SocketAddr { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match *self { - SocketAddr::Tcp(ref addr) => write!(f, "{}", addr), - #[cfg(all(unix))] - SocketAddr::Uds(ref addr) => write!(f, "{:?}", addr), - } - } -} - -impl fmt::Debug for SocketAddr { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match *self { - SocketAddr::Tcp(ref addr) => write!(f, "{:?}", addr), - #[cfg(all(unix))] - SocketAddr::Uds(ref addr) => write!(f, "{:?}", addr), - } - } -} - -impl fmt::Display for StdListener { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match *self { - StdListener::Tcp(ref lst) => write!(f, "{}", lst.local_addr().ok().unwrap()), - #[cfg(all(unix))] - StdListener::Uds(ref lst) => write!(f, "{:?}", lst.local_addr().ok().unwrap()), - } - } -} - -impl StdListener { +impl MioListener { pub(crate) fn local_addr(&self) -> SocketAddr { - match self { - StdListener::Tcp(lst) => SocketAddr::Tcp(lst.local_addr().unwrap()), - #[cfg(all(unix))] - StdListener::Uds(lst) => SocketAddr::Uds(lst.local_addr().unwrap()), - } - } - - pub(crate) fn into_listener(self) -> SocketListener { - match self { - StdListener::Tcp(lst) => SocketListener::Tcp( - mio::net::TcpListener::from_std(lst) - .expect("Can not create mio::net::TcpListener"), - ), - #[cfg(all(unix))] - StdListener::Uds(lst) => SocketListener::Uds( - mio_uds::UnixListener::from_listener(lst) - .expect("Can not create mio_uds::UnixListener"), - ), - } - } -} - -#[derive(Debug)] -pub enum StdStream { - Tcp(std::net::TcpStream), - #[cfg(all(unix))] - Uds(std::os::unix::net::UnixStream), -} - -pub(crate) enum SocketListener { - Tcp(mio::net::TcpListener), - #[cfg(all(unix))] - Uds(mio_uds::UnixListener), -} - -impl SocketListener { - pub(crate) fn accept(&self) -> io::Result> { match *self { - SocketListener::Tcp(ref lst) => lst - .accept_std() - .map(|(stream, addr)| Some((StdStream::Tcp(stream), SocketAddr::Tcp(addr)))), - #[cfg(all(unix))] - SocketListener::Uds(ref lst) => lst.accept_std().map(|res| { - res.map(|(stream, addr)| (StdStream::Uds(stream), SocketAddr::Uds(addr))) - }), + MioListener::Tcp(ref lst) => SocketAddr::Tcp(lst.local_addr().unwrap()), + #[cfg(unix)] + MioListener::Uds(ref lst) => SocketAddr::Uds(lst.local_addr().unwrap()), + } + } + + pub(crate) fn accept(&self) -> io::Result> { + match *self { + MioListener::Tcp(ref lst) => lst + .accept() + .map(|(stream, addr)| Some((MioStream::Tcp(stream), SocketAddr::Tcp(addr)))), + #[cfg(unix)] + MioListener::Uds(ref lst) => lst + .accept() + .map(|(stream, addr)| Some((MioStream::Uds(stream), SocketAddr::Uds(addr)))), } } } -impl mio::Evented for SocketListener { +impl Source for MioListener { fn register( - &self, - poll: &mio::Poll, - token: mio::Token, - interest: mio::Ready, - opts: mio::PollOpt, + &mut self, + registry: &Registry, + token: Token, + interests: Interest, ) -> io::Result<()> { match *self { - SocketListener::Tcp(ref lst) => lst.register(poll, token, interest, opts), - #[cfg(all(unix))] - SocketListener::Uds(ref lst) => lst.register(poll, token, interest, opts), + MioListener::Tcp(ref mut lst) => lst.register(registry, token, interests), + #[cfg(unix)] + MioListener::Uds(ref mut lst) => lst.register(registry, token, interests), } } fn reregister( - &self, - poll: &mio::Poll, - token: mio::Token, - interest: mio::Ready, - opts: mio::PollOpt, + &mut self, + registry: &Registry, + token: Token, + interests: Interest, ) -> io::Result<()> { match *self { - SocketListener::Tcp(ref lst) => lst.reregister(poll, token, interest, opts), - #[cfg(all(unix))] - SocketListener::Uds(ref lst) => lst.reregister(poll, token, interest, opts), + MioListener::Tcp(ref mut lst) => lst.reregister(registry, token, interests), + #[cfg(unix)] + MioListener::Uds(ref mut lst) => lst.reregister(registry, token, interests), } } - fn deregister(&self, poll: &mio::Poll) -> io::Result<()> { + + fn deregister(&mut self, registry: &Registry) -> io::Result<()> { match *self { - SocketListener::Tcp(ref lst) => lst.deregister(poll), - #[cfg(all(unix))] - SocketListener::Uds(ref lst) => { - let res = lst.deregister(poll); + MioListener::Tcp(ref mut lst) => lst.deregister(registry), + #[cfg(unix)] + MioListener::Uds(ref mut lst) => { + let res = lst.deregister(registry); // cleanup file path if let Ok(addr) = lst.local_addr() { @@ -143,28 +99,156 @@ impl mio::Evented for SocketListener { } } -pub trait FromStream: AsyncRead + AsyncWrite + Sized { - fn from_stdstream(sock: StdStream) -> io::Result; +impl From for MioListener { + fn from(lst: StdTcpListener) -> Self { + MioListener::Tcp(MioTcpListener::from_std(lst)) + } } -impl FromStream for TcpStream { - fn from_stdstream(sock: StdStream) -> io::Result { - match sock { - StdStream::Tcp(stream) => TcpStream::from_std(stream), +#[cfg(unix)] +impl From for MioListener { + fn from(lst: StdUnixListener) -> Self { + MioListener::Uds(MioUnixListener::from_std(lst)) + } +} + +impl fmt::Debug for MioListener { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match *self { + MioListener::Tcp(ref lst) => write!(f, "{:?}", lst), #[cfg(all(unix))] - StdStream::Uds(_) => { + MioListener::Uds(ref lst) => write!(f, "{:?}", lst), + } + } +} + +impl fmt::Display for MioListener { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match *self { + MioListener::Tcp(ref lst) => write!(f, "{}", lst.local_addr().ok().unwrap()), + #[cfg(unix)] + MioListener::Uds(ref lst) => write!(f, "{:?}", lst.local_addr().ok().unwrap()), + } + } +} + +pub(crate) enum SocketAddr { + Tcp(StdSocketAddr), + #[cfg(unix)] + Uds(MioSocketAddr), +} + +impl fmt::Display for SocketAddr { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match *self { + SocketAddr::Tcp(ref addr) => write!(f, "{}", addr), + #[cfg(unix)] + SocketAddr::Uds(ref addr) => write!(f, "{:?}", addr), + } + } +} + +impl fmt::Debug for SocketAddr { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match *self { + SocketAddr::Tcp(ref addr) => write!(f, "{:?}", addr), + #[cfg(unix)] + SocketAddr::Uds(ref addr) => write!(f, "{:?}", addr), + } + } +} + +#[derive(Debug)] +pub enum MioStream { + Tcp(MioTcpStream), + #[cfg(unix)] + Uds(MioUnixStream), +} + +/// helper trait for converting mio stream to tokio stream. +pub trait FromStream: Sized { + fn from_mio(sock: MioStream) -> io::Result; +} + +// FIXME: This is a workaround and we need an efficient way to convert between mio and tokio stream +#[cfg(unix)] +impl FromStream for TcpStream { + fn from_mio(sock: MioStream) -> io::Result { + match sock { + MioStream::Tcp(mio) => { + let raw = IntoRawFd::into_raw_fd(mio); + // SAFETY: This is a in place conversion from mio stream to tokio stream. + TcpStream::from_std(unsafe { FromRawFd::from_raw_fd(raw) }) + } + MioStream::Uds(_) => { panic!("Should not happen, bug in server impl"); } } } } -#[cfg(all(unix))] -impl FromStream for actix_rt::net::UnixStream { - fn from_stdstream(sock: StdStream) -> io::Result { +// FIXME: This is a workaround and we need an efficient way to convert between mio and tokio stream +#[cfg(windows)] +impl FromStream for TcpStream { + fn from_mio(sock: MioStream) -> io::Result { match sock { - StdStream::Tcp(_) => panic!("Should not happen, bug in server impl"), - StdStream::Uds(stream) => actix_rt::net::UnixStream::from_std(stream), + MioStream::Tcp(mio) => { + let raw = IntoRawSocket::into_raw_socket(mio); + // SAFETY: This is a in place conversion from mio stream to tokio stream. + TcpStream::from_std(unsafe { FromRawSocket::from_raw_socket(raw) }) + } + } + } +} + +// FIXME: This is a workaround and we need an efficient way to convert between mio and tokio stream +#[cfg(unix)] +impl FromStream for UnixStream { + fn from_mio(sock: MioStream) -> io::Result { + match sock { + MioStream::Tcp(_) => panic!("Should not happen, bug in server impl"), + MioStream::Uds(mio) => { + let raw = IntoRawFd::into_raw_fd(mio); + // SAFETY: This is a in place conversion from mio stream to tokio stream. + UnixStream::from_std(unsafe { FromRawFd::from_raw_fd(raw) }) + } + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn socket_addr() { + let addr = SocketAddr::Tcp("127.0.0.1:8080".parse().unwrap()); + assert!(format!("{:?}", addr).contains("127.0.0.1:8080")); + assert_eq!(format!("{}", addr), "127.0.0.1:8080"); + + let addr: StdSocketAddr = "127.0.0.1:0".parse().unwrap(); + let socket = MioTcpSocket::new_v4().unwrap(); + socket.set_reuseaddr(true).unwrap(); + socket.bind(addr).unwrap(); + let tcp = socket.listen(128).unwrap(); + let lst = MioListener::Tcp(tcp); + assert!(format!("{:?}", lst).contains("TcpListener")); + assert!(format!("{}", lst).contains("127.0.0.1")); + } + + #[test] + #[cfg(unix)] + fn uds() { + let _ = std::fs::remove_file("/tmp/sock.xxxxx"); + if let Ok(socket) = MioUnixListener::bind("/tmp/sock.xxxxx") { + let addr = socket.local_addr().expect("Couldn't get local address"); + let a = SocketAddr::Uds(addr); + assert!(format!("{:?}", a).contains("/tmp/sock.xxxxx")); + assert!(format!("{}", a).contains("/tmp/sock.xxxxx")); + + let lst = MioListener::Uds(socket); + assert!(format!("{:?}", lst).contains("/tmp/sock.xxxxx")); + assert!(format!("{}", lst).contains("/tmp/sock.xxxxx")); } } } diff --git a/actix-server/src/waker_queue.rs b/actix-server/src/waker_queue.rs new file mode 100644 index 00000000..f92363b5 --- /dev/null +++ b/actix-server/src/waker_queue.rs @@ -0,0 +1,89 @@ +use std::{ + collections::VecDeque, + ops::Deref, + sync::{Arc, Mutex, MutexGuard}, +}; + +use mio::{Registry, Token as MioToken, Waker}; + +use crate::worker::WorkerHandle; + +/// waker token for `mio::Poll` instance +pub(crate) const WAKER_TOKEN: MioToken = MioToken(usize::MAX); + +/// `mio::Waker` with a queue for waking up the `Accept`'s `Poll` and contains the `WakerInterest` +/// the `Poll` would want to look into. +pub(crate) struct WakerQueue(Arc<(Waker, Mutex>)>); + +impl Clone for WakerQueue { + fn clone(&self) -> Self { + Self(self.0.clone()) + } +} + +impl Deref for WakerQueue { + type Target = (Waker, Mutex>); + + fn deref(&self) -> &Self::Target { + self.0.deref() + } +} + +impl WakerQueue { + /// construct a waker queue with given `Poll`'s `Registry` and capacity. + /// + /// A fixed `WAKER_TOKEN` is used to identify the wake interest and the `Poll` needs to match + /// event's token for it to properly handle `WakerInterest`. + pub(crate) fn new(registry: &Registry) -> std::io::Result { + let waker = Waker::new(registry, WAKER_TOKEN)?; + let queue = Mutex::new(VecDeque::with_capacity(16)); + + Ok(Self(Arc::new((waker, queue)))) + } + + /// push a new interest to the queue and wake up the accept poll afterwards. + pub(crate) fn wake(&self, interest: WakerInterest) { + let (waker, queue) = self.deref(); + + queue + .lock() + .expect("Failed to lock WakerQueue") + .push_back(interest); + + waker + .wake() + .unwrap_or_else(|e| panic!("can not wake up Accept Poll: {}", e)); + } + + /// get a MutexGuard of the waker queue. + pub(crate) fn guard(&self) -> MutexGuard<'_, VecDeque> { + self.deref().1.lock().expect("Failed to lock WakerQueue") + } + + /// reset the waker queue so it does not grow infinitely. + pub(crate) fn reset(queue: &mut VecDeque) { + std::mem::swap(&mut VecDeque::::with_capacity(16), queue); + } +} + +/// types of interests we would look into when `Accept`'s `Poll` is waked up by waker. +/// +/// *. These interests should not be confused with `mio::Interest` and mostly not I/O related +pub(crate) enum WakerInterest { + /// `WorkerAvailable` is an interest from `Worker` notifying `Accept` there is a worker + /// available and can accept new tasks. + WorkerAvailable, + /// `Pause`, `Resume`, `Stop` Interest are from `ServerBuilder` future. It listens to + /// `ServerCommand` and notify `Accept` to do exactly these tasks. + Pause, + Resume, + 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 + /// by if work can be sent to it successfully).`Accept` would be waked up and add the new + /// `WorkerHandle`. + Worker(WorkerHandle), +} diff --git a/actix-server/src/worker.rs b/actix-server/src/worker.rs index bfd11979..91e98fc2 100644 --- a/actix-server/src/worker.rs +++ b/actix-server/src/worker.rs @@ -1,22 +1,22 @@ +use std::future::Future; use std::pin::Pin; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use std::sync::Arc; use std::task::{Context, Poll}; -use std::time; +use std::time::Duration; -use actix_rt::time::{delay_until, Delay, Instant}; +use actix_rt::time::{sleep_until, Instant, Sleep}; use actix_rt::{spawn, Arbiter}; use actix_utils::counter::Counter; -use futures_channel::mpsc::{unbounded, UnboundedReceiver, UnboundedSender}; -use futures_channel::oneshot; -use futures_util::future::{join_all, LocalBoxFuture, MapOk}; -use futures_util::{future::Future, stream::Stream, FutureExt, TryFutureExt}; +use futures_core::future::LocalBoxFuture; use log::{error, info, trace}; +use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; +use tokio::sync::oneshot; -use crate::accept::AcceptNotify; use crate::service::{BoxedServerService, InternalServiceFactory}; -use crate::socket::{SocketAddr, StdStream}; -use crate::Token; +use crate::socket::{MioStream, SocketAddr}; +use crate::waker_queue::{WakerInterest, WakerQueue}; +use crate::{join_all, Token}; pub(crate) struct WorkerCommand(Conn); @@ -29,7 +29,7 @@ pub(crate) struct StopCommand { #[derive(Debug)] pub(crate) struct Conn { - pub io: StdStream, + pub io: MioStream, pub token: Token, pub peer: Option, } @@ -46,31 +46,33 @@ pub fn max_concurrent_connections(num: usize) { MAX_CONNS.store(num, Ordering::Relaxed); } -pub(crate) fn num_connections() -> usize { - MAX_CONNS_COUNTER.with(|conns| conns.total()) -} - thread_local! { static MAX_CONNS_COUNTER: Counter = Counter::new(MAX_CONNS.load(Ordering::Relaxed)); } +pub(crate) fn num_connections() -> usize { + MAX_CONNS_COUNTER.with(|conns| conns.total()) +} + +// a handle to worker that can send message to worker and share the availability of worker to other +// thread. #[derive(Clone)] -pub(crate) struct WorkerClient { +pub(crate) struct WorkerHandle { pub idx: usize, tx1: UnboundedSender, tx2: UnboundedSender, avail: WorkerAvailability, } -impl WorkerClient { +impl WorkerHandle { pub fn new( idx: usize, tx1: UnboundedSender, tx2: UnboundedSender, avail: WorkerAvailability, ) -> Self { - WorkerClient { + WorkerHandle { idx, tx1, tx2, @@ -79,9 +81,7 @@ impl WorkerClient { } pub fn send(&self, msg: Conn) -> Result<(), Conn> { - self.tx1 - .unbounded_send(WorkerCommand(msg)) - .map_err(|msg| msg.into_inner().0) + self.tx1.send(WorkerCommand(msg)).map_err(|msg| msg.0 .0) } pub fn available(&self) -> bool { @@ -90,21 +90,21 @@ impl WorkerClient { pub fn stop(&self, graceful: bool) -> oneshot::Receiver { let (result, rx) = oneshot::channel(); - let _ = self.tx2.unbounded_send(StopCommand { graceful, result }); + let _ = self.tx2.send(StopCommand { graceful, result }); rx } } #[derive(Clone)] pub(crate) struct WorkerAvailability { - notify: AcceptNotify, + waker: WakerQueue, available: Arc, } impl WorkerAvailability { - pub fn new(notify: AcceptNotify) -> Self { + pub fn new(waker: WakerQueue) -> Self { WorkerAvailability { - notify, + waker, available: Arc::new(AtomicBool::new(false)), } } @@ -115,8 +115,9 @@ impl WorkerAvailability { pub fn set(&self, val: bool) { let old = self.available.swap(val, Ordering::Release); + // notify the accept on switched to available. if !old && val { - self.notify.notify() + self.waker.wake(WakerInterest::WorkerAvailable); } } } @@ -133,7 +134,7 @@ pub(crate) struct Worker { conns: Counter, factories: Vec>, state: WorkerState, - shutdown_timeout: time::Duration, + shutdown_timeout: Duration, } struct WorkerService { @@ -164,63 +165,65 @@ impl Worker { idx: usize, factories: Vec>, availability: WorkerAvailability, - shutdown_timeout: time::Duration, - ) -> WorkerClient { - let (tx1, rx) = unbounded(); - let (tx2, rx2) = unbounded(); + shutdown_timeout: Duration, + ) -> WorkerHandle { + let (tx1, rx) = unbounded_channel(); + let (tx2, rx2) = unbounded_channel(); let avail = availability.clone(); - Arbiter::new().send( - async move { - availability.set(false); - let mut wrk = MAX_CONNS_COUNTER.with(move |conns| Worker { - rx, - rx2, - availability, - factories, - shutdown_timeout, - services: Vec::new(), - conns: conns.clone(), - state: WorkerState::Unavailable(Vec::new()), - }); + // every worker runs in it's own arbiter. + Arbiter::new().send(Box::pin(async move { + availability.set(false); + let mut wrk = MAX_CONNS_COUNTER.with(move |conns| Worker { + rx, + rx2, + availability, + factories, + shutdown_timeout, + services: Vec::new(), + conns: conns.clone(), + state: WorkerState::Unavailable, + }); - let mut fut: Vec, _>> = Vec::new(); - for (idx, factory) in wrk.factories.iter().enumerate() { - fut.push(factory.create().map_ok(move |r| { - r.into_iter() - .map(|(t, s): (Token, _)| (idx, t, s)) - .collect::>() - })); - } + let fut = wrk + .factories + .iter() + .enumerate() + .map(|(idx, factory)| { + let fut = factory.create(); + async move { + fut.await.map(|r| { + r.into_iter().map(|(t, s)| (idx, t, s)).collect::>() + }) + } + }) + .collect::>(); - spawn(async move { - let res = join_all(fut).await; - let res: Result, _> = res.into_iter().collect(); - match res { - Ok(services) => { - for item in services { - for (factory, token, service) in item { - assert_eq!(token.0, wrk.services.len()); - wrk.services.push(WorkerService { - factory, - service, - status: WorkerServiceStatus::Unavailable, - }); - } + spawn(async move { + let res: Result, _> = join_all(fut).await.into_iter().collect(); + match res { + Ok(services) => { + for item in services { + for (factory, token, service) in item { + assert_eq!(token.0, wrk.services.len()); + wrk.services.push(WorkerService { + factory, + service, + status: WorkerServiceStatus::Unavailable, + }); } } - Err(e) => { - error!("Can not start worker: {:?}", e); - Arbiter::current().stop(); - } } - wrk.await - }); - } - .boxed(), - ); + Err(e) => { + error!("Can not start worker: {:?}", e); + Arbiter::current().stop(); + } + } + wrk.await + }); + })); - WorkerClient::new(idx, tx1, tx2, avail) + WorkerHandle::new(idx, tx1, tx2, avail) } fn shutdown(&mut self, force: bool) { @@ -242,7 +245,7 @@ impl Worker { fn check_readiness(&mut self, cx: &mut Context<'_>) -> Result { let mut ready = self.conns.available(cx); let mut failed = None; - for (idx, srv) in &mut self.services.iter_mut().enumerate() { + for (idx, srv) in self.services.iter_mut().enumerate() { if srv.status == WorkerServiceStatus::Available || srv.status == WorkerServiceStatus::Unavailable { @@ -288,16 +291,15 @@ impl Worker { enum WorkerState { Available, - Unavailable(Vec), + Unavailable, Restarting( usize, Token, - #[allow(clippy::type_complexity)] - Pin, ()>>>>, + LocalBoxFuture<'static, Result, ()>>, ), Shutdown( - Pin>, - Pin>, + Pin>, + Pin>, Option>, ), } @@ -305,12 +307,10 @@ enum WorkerState { impl Future for Worker { type Output = (); - // FIXME: remove this attribute - #[allow(clippy::never_loop)] fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { // `StopWorker` message handler if let Poll::Ready(Some(StopCommand { graceful, result })) = - Pin::new(&mut self.rx2).poll_next(cx) + Pin::new(&mut self.rx2).poll_recv(cx) { self.availability.set(false); let num = num_connections(); @@ -324,8 +324,8 @@ impl Future for Worker { if num != 0 { info!("Graceful worker shutdown, {} connections", num); self.state = WorkerState::Shutdown( - Box::pin(delay_until(Instant::now() + time::Duration::from_secs(1))), - Box::pin(delay_until(Instant::now() + self.shutdown_timeout)), + Box::pin(sleep_until(Instant::now() + Duration::from_secs(1))), + Box::pin(sleep_until(Instant::now() + self.shutdown_timeout)), Some(result), ); } else { @@ -341,53 +341,35 @@ impl Future for Worker { } match self.state { - WorkerState::Unavailable(ref mut conns) => { - let conn = conns.pop(); - match self.check_readiness(cx) { - Ok(true) => { - // process requests from wait queue - if let Some(conn) = conn { - let guard = self.conns.get(); - let _ = self.services[conn.token.0] - .service - .call((Some(guard), conn.io)); - } else { - self.state = WorkerState::Available; - self.availability.set(true); - } - self.poll(cx) - } - Ok(false) => { - // push connection back to queue - if let Some(conn) = conn { - if let WorkerState::Unavailable(ref mut conns) = self.state { - conns.push(conn); - } - } - Poll::Pending - } - Err((token, idx)) => { - trace!( - "Service {:?} failed, restarting", - self.factories[idx].name(token) - ); - self.services[token.0].status = WorkerServiceStatus::Restarting; - self.state = - WorkerState::Restarting(idx, token, self.factories[idx].create()); - self.poll(cx) - } + WorkerState::Unavailable => match self.check_readiness(cx) { + Ok(true) => { + self.state = WorkerState::Available; + self.availability.set(true); + self.poll(cx) } - } + Ok(false) => Poll::Pending, + Err((token, idx)) => { + trace!( + "Service {:?} failed, restarting", + self.factories[idx].name(token) + ); + self.services[token.0].status = WorkerServiceStatus::Restarting; + self.state = + WorkerState::Restarting(idx, token, self.factories[idx].create()); + self.poll(cx) + } + }, WorkerState::Restarting(idx, token, ref mut fut) => { - match Pin::new(fut).poll(cx) { + match fut.as_mut().poll(cx) { Poll::Ready(Ok(item)) => { - for (token, service) in item { + // only interest in the first item? + if let Some((token, service)) = item.into_iter().next() { trace!( "Service {:?} has been restarted", self.factories[idx].name(token) ); self.services[token.0].created(service); - self.state = WorkerState::Unavailable(Vec::new()); + self.state = WorkerState::Unavailable; return self.poll(cx); } } @@ -397,9 +379,7 @@ impl Future for Worker { self.factories[idx].name(token) ); } - Poll::Pending => { - return Poll::Pending; - } + Poll::Pending => return Poll::Pending, } self.poll(cx) } @@ -412,71 +392,56 @@ impl Future for Worker { } // check graceful timeout - match t2.as_mut().poll(cx) { - Poll::Pending => (), - Poll::Ready(_) => { - let _ = tx.take().unwrap().send(false); - self.shutdown(true); - Arbiter::current().stop(); - return Poll::Ready(()); - } + if Pin::new(t2).poll(cx).is_ready() { + let _ = tx.take().unwrap().send(false); + self.shutdown(true); + Arbiter::current().stop(); + return Poll::Ready(()); } // sleep for 1 second and then check again - match t1.as_mut().poll(cx) { - Poll::Pending => (), - Poll::Ready(_) => { - *t1 = Box::pin(delay_until( - Instant::now() + time::Duration::from_secs(1), - )); - let _ = t1.as_mut().poll(cx); - } + if t1.as_mut().poll(cx).is_ready() { + *t1 = Box::pin(sleep_until(Instant::now() + Duration::from_secs(1))); + let _ = t1.as_mut().poll(cx); } + Poll::Pending } - WorkerState::Available => { - loop { - match Pin::new(&mut self.rx).poll_next(cx) { - // handle incoming io stream - Poll::Ready(Some(WorkerCommand(msg))) => { - match self.check_readiness(cx) { - Ok(true) => { - let guard = self.conns.get(); - let _ = self.services[msg.token.0] - .service - .call((Some(guard), msg.io)); - continue; - } - Ok(false) => { - trace!("Worker is unavailable"); - self.availability.set(false); - self.state = WorkerState::Unavailable(vec![msg]); - } - Err((token, idx)) => { - trace!( - "Service {:?} failed, restarting", - self.factories[idx].name(token) - ); - self.availability.set(false); - self.services[token.0].status = - WorkerServiceStatus::Restarting; - self.state = WorkerState::Restarting( - idx, - token, - self.factories[idx].create(), - ); - } - } - return self.poll(cx); - } - Poll::Pending => { - self.state = WorkerState::Available; - return Poll::Pending; - } - Poll::Ready(None) => return Poll::Ready(()), + // actively poll stream and handle worker command + WorkerState::Available => loop { + match self.check_readiness(cx) { + Ok(true) => (), + Ok(false) => { + trace!("Worker is unavailable"); + self.availability.set(false); + self.state = WorkerState::Unavailable; + return self.poll(cx); + } + Err((token, idx)) => { + trace!( + "Service {:?} failed, restarting", + self.factories[idx].name(token) + ); + self.availability.set(false); + self.services[token.0].status = WorkerServiceStatus::Restarting; + self.state = + WorkerState::Restarting(idx, token, self.factories[idx].create()); + return self.poll(cx); } } - } + + match Pin::new(&mut self.rx).poll_recv(cx) { + // handle incoming io stream + Poll::Ready(Some(WorkerCommand(msg))) => { + let guard = self.conns.get(); + let _ = self.services[msg.token.0] + .service + .call((Some(guard), msg.io)); + } + Poll::Pending => return Poll::Pending, + Poll::Ready(None) => return Poll::Ready(()), + }; + }, } } } diff --git a/actix-server/tests/test_server.rs b/actix-server/tests/test_server.rs index 838c3cf1..2604df74 100644 --- a/actix-server/tests/test_server.rs +++ b/actix-server/tests/test_server.rs @@ -5,14 +5,13 @@ use std::{net, thread, time}; use actix_server::Server; use actix_service::fn_service; use futures_util::future::{lazy, ok}; -use socket2::{Domain, Protocol, Socket, Type}; fn unused_addr() -> net::SocketAddr { let addr: net::SocketAddr = "127.0.0.1:0".parse().unwrap(); - let socket = Socket::new(Domain::ipv4(), Type::stream(), Some(Protocol::tcp())).unwrap(); - socket.bind(&addr.into()).unwrap(); - socket.set_reuse_address(true).unwrap(); - let tcp = socket.into_tcp_listener(); + let socket = mio::net::TcpSocket::new_v4().unwrap(); + socket.bind(addr).unwrap(); + socket.set_reuseaddr(true).unwrap(); + let tcp = socket.listen(32).unwrap(); tcp.local_addr().unwrap() } @@ -22,8 +21,7 @@ fn test_bind() { let (tx, rx) = mpsc::channel(); let h = thread::spawn(move || { - let mut sys = actix_rt::System::new("test"); - + let sys = actix_rt::System::new("test"); let srv = sys.block_on(lazy(|_| { Server::build() .workers(1) @@ -49,17 +47,17 @@ fn test_listen() { let (tx, rx) = mpsc::channel(); let h = thread::spawn(move || { - let mut sys = actix_rt::System::new("test"); + let sys = actix_rt::System::new("test"); let lst = net::TcpListener::bind(addr).unwrap(); - sys.block_on(lazy(|_| { + sys.block_on(async { Server::build() .disable_signals() .workers(1) .listen("test", lst, move || fn_service(|_| ok::<_, ()>(()))) .unwrap() - .start() - })); - let _ = tx.send(actix_rt::System::current()); + .start(); + let _ = tx.send(actix_rt::System::current()); + }); let _ = sys.run(); }); let sys = rx.recv().unwrap(); @@ -83,7 +81,7 @@ fn test_start() { let (tx, rx) = mpsc::channel(); let h = thread::spawn(move || { - let mut sys = actix_rt::System::new("test"); + let sys = actix_rt::System::new("test"); let srv = sys.block_on(lazy(|_| { Server::build() .backlog(100) @@ -102,6 +100,7 @@ fn test_start() { let _ = tx.send((srv, actix_rt::System::current())); let _ = sys.run(); }); + let (srv, sys) = rx.recv().unwrap(); let mut buf = [1u8; 4]; @@ -151,7 +150,7 @@ fn test_configure() { let h = thread::spawn(move || { let num = num2.clone(); - let mut sys = actix_rt::System::new("test"); + let sys = actix_rt::System::new("test"); let srv = sys.block_on(lazy(|_| { Server::build() .disable_signals() diff --git a/actix-testing/Cargo.toml b/actix-testing/Cargo.toml index 430a12b6..17855a24 100644 --- a/actix-testing/Cargo.toml +++ b/actix-testing/Cargo.toml @@ -18,10 +18,10 @@ name = "actix_testing" path = "src/lib.rs" [dependencies] -actix-rt = "1.0.0" +actix-rt = "2.0.0-beta.1" actix-macros = "0.1.0" actix-server = "1.0.0" -actix-service = "1.0.0" +actix-service = "2.0.0-beta.1" log = "0.4" socket2 = "0.3" diff --git a/actix-testing/src/lib.rs b/actix-testing/src/lib.rs index eadfe6c9..57e2c223 100644 --- a/actix-testing/src/lib.rs +++ b/actix-testing/src/lib.rs @@ -83,7 +83,7 @@ impl TestServer { // run server in separate thread thread::spawn(move || { - let mut sys = System::new("actix-test-server"); + let sys = System::new("actix-test-server"); let tcp = net::TcpListener::bind("127.0.0.1:0").unwrap(); let local_addr = tcp.local_addr().unwrap(); @@ -94,9 +94,8 @@ impl TestServer { .workers(1) .disable_signals() .start(); + tx.send((System::current(), local_addr)).unwrap(); }); - - tx.send((System::current(), local_addr)).unwrap(); sys.run() }); From 98a17081b89b26b1c9eb08b5c4daa041d7a2dbc7 Mon Sep 17 00:00:00 2001 From: Rob Ede Date: Mon, 28 Dec 2020 23:50:00 +0000 Subject: [PATCH 2/8] prepare server beta release --- actix-server/CHANGES.md | 17 +++++++++++------ actix-server/Cargo.toml | 9 ++++++--- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/actix-server/CHANGES.md b/actix-server/CHANGES.md index 15d7d596..302ea576 100644 --- a/actix-server/CHANGES.md +++ b/actix-server/CHANGES.md @@ -1,17 +1,22 @@ # Changes ## Unreleased - 2020-xx-xx + + +## 2.0.0-beta.1 - 2020-12-28 * Added explicit info log message on accept queue pause. [#215] * Prevent double registration of sockets when back-pressure is resolved. [#223] -* Update `mio` dependency to `0.7.3`. -* Remove `socket2` dependency. -* `ServerBuilder::backlog` would accept `u32` instead of `i32`. -* Remove `AcceptNotify` type and pass `WakerQueue` to `Worker` for wake up the `Accept`'s `Poll`. -* Convert `mio::net::TcpStream` to `actix_rt::net::TcpStream`(`UnixStream` for uds) using `FromRawFd` and `IntoRawFd`(`FromRawSocket` and `IntoRawSocket` on windows). -* Remove `AsyncRead` and `AsyncWrite` trait bound for `socket::FromStream` trait. +* Update `mio` dependency to `0.7.3`. [#239] +* Remove `socket2` dependency. [#239] +* `ServerBuilder::backlog` now accepts `u32` instead of `i32`. [#239] +* Remove `AcceptNotify` type and pass `WakerQueue` to `Worker` to wake up `Accept`'s `Poll`. [#239] +* Convert `mio::net::TcpStream` to `actix_rt::net::TcpStream`(`UnixStream` for uds) using + `FromRawFd` and `IntoRawFd`(`FromRawSocket` and `IntoRawSocket` on windows). [#239] +* Remove `AsyncRead` and `AsyncWrite` trait bound for `socket::FromStream` trait. [#239] [#215]: https://github.com/actix/actix-net/pull/215 [#223]: https://github.com/actix/actix-net/pull/223 +[#239]: https://github.com/actix/actix-net/pull/239 ## 1.0.4 - 2020-09-12 diff --git a/actix-server/Cargo.toml b/actix-server/Cargo.toml index 34fb3775..a222f9c2 100644 --- a/actix-server/Cargo.toml +++ b/actix-server/Cargo.toml @@ -1,7 +1,10 @@ [package] name = "actix-server" -version = "1.0.4" -authors = ["Nikolay Kim "] +version = "2.0.0-beta.1" +authors = [ + "Nikolay Kim ", + "fakeshadow <24548779@qq.com>", +] description = "General purpose TCP server built for the Actix ecosystem" keywords = ["network", "framework", "async", "futures"] homepage = "https://actix.rs" @@ -35,6 +38,6 @@ tokio = { version = "1", features = ["sync"] } [dev-dependencies] actix-testing = "1.0.0" bytes = "1" -env_logger = "0.7" +env_logger = "0.8" futures-util = { version = "0.3.7", default-features = false, features = ["sink"] } tokio = { version = "1", features = ["io-util"] } From ef83647ac9936d3b6e3520e42156dcb53dd42674 Mon Sep 17 00:00:00 2001 From: Rob Ede Date: Mon, 28 Dec 2020 23:54:21 +0000 Subject: [PATCH 3/8] prepare testing beta release --- actix-server/Cargo.toml | 2 +- actix-testing/CHANGES.md | 22 +++++++++++++--------- actix-testing/Cargo.toml | 7 +++---- actix-testing/README.md | 2 +- actix-tls/Cargo.toml | 4 ++-- 5 files changed, 20 insertions(+), 17 deletions(-) diff --git a/actix-server/Cargo.toml b/actix-server/Cargo.toml index a222f9c2..c57b8ac6 100644 --- a/actix-server/Cargo.toml +++ b/actix-server/Cargo.toml @@ -36,7 +36,7 @@ slab = "0.4" tokio = { version = "1", features = ["sync"] } [dev-dependencies] -actix-testing = "1.0.0" +actix-testing = "2.0.0-beta.1" bytes = "1" env_logger = "0.8" futures-util = { version = "0.3.7", default-features = false, features = ["sink"] } diff --git a/actix-testing/CHANGES.md b/actix-testing/CHANGES.md index 86a79ed7..f23c7521 100644 --- a/actix-testing/CHANGES.md +++ b/actix-testing/CHANGES.md @@ -1,33 +1,37 @@ # Changes -## [1.0.1] - 2020-05-19 +## Unreleased - 2021-xx-xx + +## 2.0.0-beta.1 - 2020-12-28 +* Update `actix-server` to v2.0.0-beta.1. + + +## 1.0.1 - 2020-05-19 * Replace deprecated `net2` crate with `socket2` - * Remove unused `futures` dependency -## [1.0.0] - 2019-12-11 +## 1.0.0 - 2019-12-11 * Update actix-server to 1.0.0 -## [1.0.0-alpha.3] - 2019-12-07 +## 1.0.0-alpha.3 - 2019-12-07 * Migrate to tokio 0.2 -## [1.0.0-alpha.2] - 2019-12-02 +## 1.0.0-alpha.2 - 2019-12-02 * Re-export `test` attribute macros -## [0.3.0-alpha.1] - 2019-11-22 +## 0.3.0-alpha.1 - 2019-11-22 * Migrate to std::future -## [0.2.0] - 2019-10-14 +## 0.2.0 - 2019-10-14 * Upgrade actix-server and actix-server-config deps -## [0.1.0] - 2019-09-25 - +## 0.1.0 - 2019-09-25 * Initial impl diff --git a/actix-testing/Cargo.toml b/actix-testing/Cargo.toml index 17855a24..3f29679e 100644 --- a/actix-testing/Cargo.toml +++ b/actix-testing/Cargo.toml @@ -1,8 +1,8 @@ [package] name = "actix-testing" -version = "1.0.1" +version = "2.0.0-beta.1" authors = ["Nikolay Kim "] -description = "Actix testing utils" +description = "Various helpers for Actix applications to use during testing" keywords = ["network", "framework", "async", "futures"] homepage = "https://actix.rs" repository = "https://github.com/actix/actix-net.git" @@ -10,7 +10,6 @@ documentation = "https://docs.rs/actix-testing/" categories = ["network-programming", "asynchronous"] license = "MIT OR Apache-2.0" edition = "2018" -workspace = ".." readme = "README.md" [lib] @@ -20,7 +19,7 @@ path = "src/lib.rs" [dependencies] actix-rt = "2.0.0-beta.1" actix-macros = "0.1.0" -actix-server = "1.0.0" +actix-server = "2.0.0-beta.1" actix-service = "2.0.0-beta.1" log = "0.4" diff --git a/actix-testing/README.md b/actix-testing/README.md index bd4eec2f..ae54b46d 100644 --- a/actix-testing/README.md +++ b/actix-testing/README.md @@ -6,4 +6,4 @@ * [API Documentation](https://docs.rs/actix-testing/) * [Chat on gitter](https://gitter.im/actix/actix) * Cargo package: [actix-http-test](https://crates.io/crates/actix-testing) -* Minimum supported Rust version: 1.37 or later +* Minimum supported Rust version: 1.46 or later diff --git a/actix-tls/Cargo.toml b/actix-tls/Cargo.toml index 801f6144..dcd5978a 100644 --- a/actix-tls/Cargo.toml +++ b/actix-tls/Cargo.toml @@ -59,6 +59,6 @@ tokio-tls = { version = "0.3", optional = true } bytes = "0.5" log = "0.4" env_logger = "0.7" -actix-testing = "1.0.0" -actix-server = "1" +actix-testing = "2.0.0-beta.1" +actix-server = "2.0.0-beta.1" actix-rt = "1" From 3c6de3a81ba3d3923f2b03e3299cba49f5b8b06a Mon Sep 17 00:00:00 2001 From: Rob Ede Date: Tue, 29 Dec 2020 00:08:59 +0000 Subject: [PATCH 4/8] use correct service version for tracing --- actix-tracing/Cargo.toml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/actix-tracing/Cargo.toml b/actix-tracing/Cargo.toml index d0b7bf29..720e7ad5 100644 --- a/actix-tracing/Cargo.toml +++ b/actix-tracing/Cargo.toml @@ -16,11 +16,12 @@ name = "actix_tracing" path = "src/lib.rs" [dependencies] -actix-service = "1.0.4" +actix-service = "2.0.0-beta.1" + futures-util = { version = "0.3.4", default-features = false } tracing = "0.1" tracing-futures = "0.2" [dev_dependencies] actix-rt = "1.0" -slab = "0.4" \ No newline at end of file +slab = "0.4" From 5759c9e14467c59f9dedf4b2d80bf745106dd40e Mon Sep 17 00:00:00 2001 From: Rob Ede Date: Tue, 29 Dec 2020 00:38:41 +0000 Subject: [PATCH 5/8] merge -connect and -tls and upgrade to rt v2 (#238) --- Cargo.toml | 10 +- actix-connect/CHANGES.md | 154 ------------------ actix-connect/Cargo.toml | 58 ------- actix-connect/LICENSE-APACHE | 1 - actix-connect/LICENSE-MIT | 1 - actix-connect/tests/test_connect.rs | 127 --------------- actix-tls/CHANGES.md | 5 + actix-tls/Cargo.toml | 68 +++++--- actix-tls/examples/basic.rs | 8 +- actix-tls/src/accept/mod.rs | 42 +++++ actix-tls/src/{ => accept}/nativetls.rs | 16 +- actix-tls/src/{ => accept}/openssl.rs | 48 +++--- actix-tls/src/{ => accept}/rustls.rs | 28 +++- .../src => actix-tls/src/connect}/connect.rs | 0 .../src/connect}/connector.rs | 1 + .../src => actix-tls/src/connect}/error.rs | 0 .../lib.rs => actix-tls/src/connect/mod.rs | 10 -- .../src => actix-tls/src/connect}/resolve.rs | 7 +- .../src => actix-tls/src/connect}/service.rs | 8 +- .../src => actix-tls/src/connect}/ssl/mod.rs | 0 .../src/connect}/ssl/openssl.rs | 67 ++++---- .../src/connect}/ssl/rustls.rs | 11 +- .../src => actix-tls/src/connect}/uri.rs | 2 +- actix-tls/src/lib.rs | 47 +----- 24 files changed, 212 insertions(+), 507 deletions(-) delete mode 100644 actix-connect/CHANGES.md delete mode 100644 actix-connect/Cargo.toml delete mode 120000 actix-connect/LICENSE-APACHE delete mode 120000 actix-connect/LICENSE-MIT delete mode 100644 actix-connect/tests/test_connect.rs create mode 100644 actix-tls/src/accept/mod.rs rename actix-tls/src/{ => accept}/nativetls.rs (87%) rename actix-tls/src/{ => accept}/openssl.rs (69%) rename actix-tls/src/{ => accept}/rustls.rs (85%) rename {actix-connect/src => actix-tls/src/connect}/connect.rs (100%) rename {actix-connect/src => actix-tls/src/connect}/connector.rs (99%) rename {actix-connect/src => actix-tls/src/connect}/error.rs (100%) rename actix-connect/src/lib.rs => actix-tls/src/connect/mod.rs (93%) rename {actix-connect/src => actix-tls/src/connect}/resolve.rs (97%) rename {actix-connect/src => actix-tls/src/connect}/service.rs (97%) rename {actix-connect/src => actix-tls/src/connect}/ssl/mod.rs (100%) rename {actix-connect/src => actix-tls/src/connect}/ssl/openssl.rs (80%) rename {actix-connect/src => actix-tls/src/connect}/ssl/rustls.rs (93%) rename {actix-connect/src => actix-tls/src/connect}/uri.rs (97%) diff --git a/Cargo.toml b/Cargo.toml index d46b6283..533fbc17 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,7 +1,6 @@ [workspace] members = [ "actix-codec", - "actix-connect", "actix-rt", "actix-macros", "actix-service", @@ -16,16 +15,15 @@ members = [ ] [patch.crates-io] -actix-codec = { git = "https://github.com/actix/actix-net.git", rev = "ba44ea7d0bafaf5fccb9a34003d503e1910943ee" } -actix-connect = { path = "actix-connect" } -actix-rt = { git = "https://github.com/actix/actix-net.git", rev = "ba44ea7d0bafaf5fccb9a34003d503e1910943ee" } +actix-codec = { path = "actix-codec" } +actix-rt = { path = "actix-rt" } actix-macros = { path = "actix-macros" } actix-server = { path = "actix-server" } -actix-service = { git = "https://github.com/actix/actix-net.git", rev = "ba44ea7d0bafaf5fccb9a34003d503e1910943ee" } +actix-service = { path = "actix-service" } actix-testing = { path = "actix-testing" } actix-threadpool = { path = "actix-threadpool" } actix-tls = { path = "actix-tls" } actix-tracing = { path = "actix-tracing" } -actix-utils = { git = "https://github.com/actix/actix-net.git", rev = "ba44ea7d0bafaf5fccb9a34003d503e1910943ee" } +actix-utils = { path = "actix-utils" } actix-router = { path = "router" } bytestring = { path = "string" } diff --git a/actix-connect/CHANGES.md b/actix-connect/CHANGES.md deleted file mode 100644 index 6f18b5b4..00000000 --- a/actix-connect/CHANGES.md +++ /dev/null @@ -1,154 +0,0 @@ -# Changes - -## Unreleased - 2020-xx-xx - - -## 2.0.0 - 2020-09-02 -- No significant changes from `2.0.0-alpha.4`. - -## 2.0.0-alpha.4 - 2020-08-17 - -### Changed - -* Update `rustls` dependency to 0.18 -* Update `tokio-rustls` dependency to 0.14 - - -## [2.0.0-alpha.3] - 2020-05-08 - -### Fixed - -* Corrected spelling of `ConnectError::Unresolverd` to `ConnectError::Unresolved` - -## [2.0.0-alpha.2] - 2020-03-08 - -### Changed - -* Update `trust-dns-proto` dependency to 0.19. [#116] -* Update `trust-dns-resolver` dependency to 0.19. [#116] -* `Address` trait is now required to have static lifetime. [#116] -* `start_resolver` and `start_default_resolver` are now `async` and may return a `ConnectError`. [#116] - -[#116]: https://github.com/actix/actix-net/pull/116 - -## [2.0.0-alpha.1] - 2020-03-03 - -### Changed - -* Update `rustls` dependency to 0.17 -* Update `tokio-rustls` dependency to 0.13 - -## [1.0.2] - 2020-01-15 - -* Fix actix-service 1.0.3 compatibility - -## [1.0.1] - 2019-12-15 - -* Fix trust-dns-resolver compilation - -## [1.0.0] - 2019-12-11 - -* Release - -## [1.0.0-alpha.3] - 2019-12-07 - -### Changed - -* Migrate to tokio 0.2 - - -## [1.0.0-alpha.2] - 2019-12-02 - -### Changed - -* Migrated to `std::future` - - -## [0.3.0] - 2019-10-03 - -### Changed - -* Update `rustls` to 0.16 -* Minimum required Rust version upped to 1.37.0 - -## [0.2.5] - 2019-09-05 - -* Add `TcpConnectService` - -## [0.2.4] - 2019-09-02 - -* Use arbiter's storage for default async resolver - -## [0.2.3] - 2019-08-05 - -* Add `ConnectService` and `OpensslConnectService` - -## [0.2.2] - 2019-07-24 - -* Add `rustls` support - -## [0.2.1] - 2019-07-17 - -### Added - -* Expose Connect addrs #30 - -### Changed - -* Update `derive_more` to 0.15 - - -## [0.2.0] - 2019-05-12 - -### Changed - -* Upgrade to actix-service 0.4 - - -## [0.1.5] - 2019-04-19 - -### Added - -* `Connect::set_addr()` - -### Changed - -* Use trust-dns-resolver 0.11.0 - - -## [0.1.4] - 2019-04-12 - -### Changed - -* Do not start default resolver immediately for default connector. - - -## [0.1.3] - 2019-04-11 - -### Changed - -* Start trust-dns default resolver on first use - -## [0.1.2] - 2019-04-04 - -### Added - -* Log error if dns system config could not be loaded. - -### Changed - -* Rename connect Connector to TcpConnector #10 - - -## [0.1.1] - 2019-03-15 - -### Fixed - -* Fix error handling for single address - - -## [0.1.0] - 2019-03-14 - -* Refactor resolver and connector services - -* Rename crate diff --git a/actix-connect/Cargo.toml b/actix-connect/Cargo.toml deleted file mode 100644 index 233195c5..00000000 --- a/actix-connect/Cargo.toml +++ /dev/null @@ -1,58 +0,0 @@ -[package] -name = "actix-connect" -version = "2.0.0" -authors = ["Nikolay Kim "] -description = "TCP connector service for Actix ecosystem." -keywords = ["network", "framework", "async", "futures"] -homepage = "https://actix.rs" -repository = "https://github.com/actix/actix-net.git" -documentation = "https://docs.rs/actix-connect/" -categories = ["network-programming", "asynchronous"] -license = "MIT OR Apache-2.0" -edition = "2018" - -[package.metadata.docs.rs] -features = ["openssl", "rustls", "uri"] - -[lib] -name = "actix_connect" -path = "src/lib.rs" - -[features] -default = ["uri"] - -# openssl -openssl = ["open-ssl", "tokio-openssl"] - -# rustls -rustls = ["rust-tls", "tokio-rustls", "webpki"] - -# support http::Uri as connect address -uri = ["http"] - -[dependencies] -actix-service = "1.0.6" -actix-codec = "0.3.0" -actix-utils = "2.0.0" -actix-rt = "1.1.1" - -derive_more = "0.99.2" -either = "1.5.3" -futures-util = { version = "0.3.4", default-features = false } -http = { version = "0.2.0", optional = true } -log = "0.4" -trust-dns-proto = { version = "0.19", default-features = false, features = ["tokio-runtime"] } -trust-dns-resolver = { version = "0.19", default-features = false, features = ["tokio-runtime", "system-config"] } - -# openssl -open-ssl = { package = "openssl", version = "0.10", optional = true } -tokio-openssl = { version = "0.4.0", optional = true } - -# rustls -rust-tls = { package = "rustls", version = "0.18.0", optional = true } -tokio-rustls = { version = "0.14.0", optional = true } -webpki = { version = "0.21", optional = true } - -[dev-dependencies] -bytes = "0.5.3" -actix-testing = "1.0.0" diff --git a/actix-connect/LICENSE-APACHE b/actix-connect/LICENSE-APACHE deleted file mode 120000 index 965b606f..00000000 --- a/actix-connect/LICENSE-APACHE +++ /dev/null @@ -1 +0,0 @@ -../LICENSE-APACHE \ No newline at end of file diff --git a/actix-connect/LICENSE-MIT b/actix-connect/LICENSE-MIT deleted file mode 120000 index 76219eb7..00000000 --- a/actix-connect/LICENSE-MIT +++ /dev/null @@ -1 +0,0 @@ -../LICENSE-MIT \ No newline at end of file diff --git a/actix-connect/tests/test_connect.rs b/actix-connect/tests/test_connect.rs deleted file mode 100644 index 21d78d2c..00000000 --- a/actix-connect/tests/test_connect.rs +++ /dev/null @@ -1,127 +0,0 @@ -use std::io; - -use actix_codec::{BytesCodec, Framed}; -use actix_rt::net::TcpStream; -use actix_service::{fn_service, Service, ServiceFactory}; -use actix_testing::TestServer; -use bytes::Bytes; -use futures_util::sink::SinkExt; - -use actix_connect::resolver::{ResolverConfig, ResolverOpts}; -use actix_connect::Connect; - -#[cfg(feature = "openssl")] -#[actix_rt::test] -async fn test_string() { - let srv = TestServer::with(|| { - fn_service(|io: TcpStream| async { - let mut framed = Framed::new(io, BytesCodec); - framed.send(Bytes::from_static(b"test")).await?; - Ok::<_, io::Error>(()) - }) - }); - - let mut conn = actix_connect::default_connector(); - let addr = format!("localhost:{}", srv.port()); - let con = conn.call(addr.into()).await.unwrap(); - assert_eq!(con.peer_addr().unwrap(), srv.addr()); -} - -#[cfg(feature = "rustls")] -#[actix_rt::test] -async fn test_rustls_string() { - let srv = TestServer::with(|| { - fn_service(|io: TcpStream| async { - let mut framed = Framed::new(io, BytesCodec); - framed.send(Bytes::from_static(b"test")).await?; - Ok::<_, io::Error>(()) - }) - }); - - let mut conn = actix_connect::default_connector(); - let addr = format!("localhost:{}", srv.port()); - let con = conn.call(addr.into()).await.unwrap(); - assert_eq!(con.peer_addr().unwrap(), srv.addr()); -} - -#[actix_rt::test] -async fn test_static_str() { - let srv = TestServer::with(|| { - fn_service(|io: TcpStream| async { - let mut framed = Framed::new(io, BytesCodec); - framed.send(Bytes::from_static(b"test")).await?; - Ok::<_, io::Error>(()) - }) - }); - - let resolver = actix_connect::start_default_resolver().await.unwrap(); - let mut conn = actix_connect::new_connector(resolver.clone()); - - let con = conn.call(Connect::with("10", srv.addr())).await.unwrap(); - assert_eq!(con.peer_addr().unwrap(), srv.addr()); - - let connect = Connect::new(srv.host().to_owned()); - let mut conn = actix_connect::new_connector(resolver); - let con = conn.call(connect).await; - assert!(con.is_err()); -} - -#[actix_rt::test] -async fn test_new_service() { - let srv = TestServer::with(|| { - fn_service(|io: TcpStream| async { - let mut framed = Framed::new(io, BytesCodec); - framed.send(Bytes::from_static(b"test")).await?; - Ok::<_, io::Error>(()) - }) - }); - - let resolver = - actix_connect::start_resolver(ResolverConfig::default(), ResolverOpts::default()) - .await - .unwrap(); - - let factory = actix_connect::new_connector_factory(resolver); - - let mut conn = factory.new_service(()).await.unwrap(); - let con = conn.call(Connect::with("10", srv.addr())).await.unwrap(); - assert_eq!(con.peer_addr().unwrap(), srv.addr()); -} - -#[cfg(all(feature = "openssl", feature = "uri"))] -#[actix_rt::test] -async fn test_openssl_uri() { - use std::convert::TryFrom; - - let srv = TestServer::with(|| { - fn_service(|io: TcpStream| async { - let mut framed = Framed::new(io, BytesCodec); - framed.send(Bytes::from_static(b"test")).await?; - Ok::<_, io::Error>(()) - }) - }); - - let mut conn = actix_connect::default_connector(); - let addr = http::Uri::try_from(format!("https://localhost:{}", srv.port())).unwrap(); - let con = conn.call(addr.into()).await.unwrap(); - assert_eq!(con.peer_addr().unwrap(), srv.addr()); -} - -#[cfg(all(feature = "rustls", feature = "uri"))] -#[actix_rt::test] -async fn test_rustls_uri() { - use std::convert::TryFrom; - - let srv = TestServer::with(|| { - fn_service(|io: TcpStream| async { - let mut framed = Framed::new(io, BytesCodec); - framed.send(Bytes::from_static(b"test")).await?; - Ok::<_, io::Error>(()) - }) - }); - - let mut conn = actix_connect::default_connector(); - let addr = http::Uri::try_from(format!("https://localhost:{}", srv.port())).unwrap(); - let con = conn.call(addr.into()).await.unwrap(); - assert_eq!(con.peer_addr().unwrap(), srv.addr()); -} diff --git a/actix-tls/CHANGES.md b/actix-tls/CHANGES.md index 592b96b0..9f03bfeb 100644 --- a/actix-tls/CHANGES.md +++ b/actix-tls/CHANGES.md @@ -1,6 +1,11 @@ # Changes ## Unreleased - 2020-xx-xx +* Move acceptors under `accept` module. [#238] +* Merge `actix-connect` crate under `connect` module. [#238] +* Add feature flags to enable acceptors and/or connectors individually. + +[#238]: https://github.com/actix/actix-net/pull/238 ## 2.0.0 - 2020-09-03 diff --git a/actix-tls/Cargo.toml b/actix-tls/Cargo.toml index dcd5978a..de2d859f 100644 --- a/actix-tls/Cargo.toml +++ b/actix-tls/Cargo.toml @@ -2,8 +2,8 @@ name = "actix-tls" version = "2.0.0" authors = ["Nikolay Kim "] -description = "TLS acceptor services for Actix ecosystem." -keywords = ["network", "framework", "async", "tls", "ssl"] +description = "TLS acceptor and connector services for Actix ecosystem" +keywords = ["network", "tls", "ssl", "async", "transport"] homepage = "https://actix.rs" repository = "https://github.com/actix/actix-net.git" documentation = "https://docs.rs/actix-tls/" @@ -12,7 +12,7 @@ license = "MIT OR Apache-2.0" edition = "2018" [package.metadata.docs.rs] -features = ["openssl", "rustls", "nativetls"] +features = ["openssl", "rustls", "native-tls", "accept", "connect", "http"] [lib] name = "actix_tls" @@ -20,45 +20,59 @@ path = "src/lib.rs" [[example]] name = "basic" -required-features = ["rustls"] +required-features = ["accept", "rustls"] [features] -default = [] +default = ["accept", "connect", "http"] -# openssl -openssl = ["open-ssl", "tokio-openssl"] +# enable acceptor services +accept = [] -# rustls -rustls = ["rust-tls", "webpki", "webpki-roots", "tokio-rustls"] +# enable connector services +connect = [] -# nativetls -nativetls = ["native-tls", "tokio-tls"] +# use openssl impls +openssl = ["tls-openssl", "tokio-openssl"] + +# use rustls impls +rustls = ["tls-rustls", "webpki", "webpki-roots", "tokio-rustls"] + +# use native-tls impls +native-tls = ["tls-native-tls", "tokio-native-tls"] [dependencies] -actix-service = "1.0.0" -actix-codec = "0.3.0" -actix-utils = "2.0.0" +actix-codec = "0.4.0-beta.1" +actix-rt = "2.0.0-beta.1" +actix-service = "2.0.0-beta.1" +actix-utils = "3.0.0-beta.1" -futures-util = { version = "0.3.4", default-features = false } +derive_more = "0.99.5" +either = "1.6" +futures-util = { version = "0.3.7", default-features = false } +http = { version = "0.2.0", optional = true } +log = "0.4" +trust-dns-proto = { version = "0.19", default-features = false, features = ["tokio-runtime"] } +trust-dns-resolver = { version = "0.19", default-features = false, features = ["tokio-runtime", "system-config"] } # openssl -open-ssl = { package = "openssl", version = "0.10", optional = true } -tokio-openssl = { version = "0.4.0", optional = true } +tls-openssl = { package = "openssl", version = "0.10", optional = true } +tokio-openssl = { version = "0.6", optional = true } + +# TODO: Reduce dependencies where tokio wrappers re-export base crate. # rustls -rust-tls = { package = "rustls", version = "0.18.0", optional = true } +tls-rustls = { package = "rustls", version = "0.19", optional = true } +tokio-rustls = { version = "0.22", optional = true } webpki = { version = "0.21", optional = true } -webpki-roots = { version = "0.20", optional = true } -tokio-rustls = { version = "0.14.0", optional = true } +webpki-roots = { version = "0.21", optional = true } # native-tls -native-tls = { version = "0.2", optional = true } -tokio-tls = { version = "0.3", optional = true } +tls-native-tls = { package = "native-tls", version = "0.2", optional = true } +tokio-native-tls = { version = "0.3", optional = true } [dev-dependencies] -bytes = "0.5" -log = "0.4" -env_logger = "0.7" -actix-testing = "2.0.0-beta.1" actix-server = "2.0.0-beta.1" -actix-rt = "1" +actix-testing = "2.0.0-beta.1" +bytes = "1" +log = "0.4" +env_logger = "0.8" diff --git a/actix-tls/examples/basic.rs b/actix-tls/examples/basic.rs index cd706958..3f4cdb24 100644 --- a/actix-tls/examples/basic.rs +++ b/actix-tls/examples/basic.rs @@ -15,6 +15,10 @@ //! http --verify=false https://127.0.0.1:8443 //! ``` +// this rename only exists because of how we have organised the crate's feature flags +// it is not necessary for your actual code +extern crate tls_rustls as rustls; + use std::{ env, fs::File, @@ -27,10 +31,10 @@ use std::{ use actix_server::Server; use actix_service::pipeline_factory; -use actix_tls::rustls::Acceptor as RustlsAcceptor; +use actix_tls::accept::rustls::Acceptor as RustlsAcceptor; use futures_util::future::ok; use log::info; -use rust_tls::{ +use rustls::{ internal::pemfile::certs, internal::pemfile::rsa_private_keys, NoClientAuth, ServerConfig, }; diff --git a/actix-tls/src/accept/mod.rs b/actix-tls/src/accept/mod.rs new file mode 100644 index 00000000..8b1fe47c --- /dev/null +++ b/actix-tls/src/accept/mod.rs @@ -0,0 +1,42 @@ +//! TLS acceptor services for Actix ecosystem. +//! +//! ## Crate Features +//! * `openssl` - TLS acceptor using the `openssl` crate. +//! * `rustls` - TLS acceptor using the `rustls` crate. +//! * `native-tls` - TLS acceptor using the `native-tls` crate. + +use std::sync::atomic::{AtomicUsize, Ordering}; + +use actix_utils::counter::Counter; + +#[cfg(feature = "openssl")] +pub mod openssl; + +#[cfg(feature = "rustls")] +pub mod rustls; + +#[cfg(feature = "native-tls")] +pub mod nativetls; + +pub(crate) static MAX_CONN: AtomicUsize = AtomicUsize::new(256); + +thread_local! { + static MAX_CONN_COUNTER: Counter = Counter::new(MAX_CONN.load(Ordering::Relaxed)); +} + +/// Sets the maximum per-worker concurrent TLS connection limit. +/// +/// All listeners will stop accepting connections when this limit is reached. +/// It can be used to regulate the global TLS CPU usage. +/// +/// By default, the connection limit is 256. +pub fn max_concurrent_tls_connect(num: usize) { + MAX_CONN.store(num, Ordering::Relaxed); +} + +/// TLS error combined with service error. +#[derive(Debug)] +pub enum TlsError { + Tls(E1), + Service(E2), +} diff --git a/actix-tls/src/nativetls.rs b/actix-tls/src/accept/nativetls.rs similarity index 87% rename from actix-tls/src/nativetls.rs rename to actix-tls/src/accept/nativetls.rs index fa66bff6..12d23855 100644 --- a/actix-tls/src/nativetls.rs +++ b/actix-tls/src/accept/nativetls.rs @@ -7,13 +7,13 @@ use actix_utils::counter::Counter; use futures_util::future::{self, FutureExt, LocalBoxFuture, TryFutureExt}; pub use native_tls::Error; -pub use tokio_tls::{TlsAcceptor, TlsStream}; +pub use tokio_native_tls::{TlsAcceptor, TlsStream}; -use crate::MAX_CONN_COUNTER; +use super::MAX_CONN_COUNTER; /// Accept TLS connections via `native-tls` package. /// -/// `nativetls` feature enables this `Acceptor` type. +/// `native-tls` feature enables this `Acceptor` type. pub struct Acceptor { acceptor: TlsAcceptor, io: PhantomData, @@ -43,11 +43,10 @@ impl Clone for Acceptor { } } -impl ServiceFactory for Acceptor +impl ServiceFactory for Acceptor where T: AsyncRead + AsyncWrite + Unpin + 'static, { - type Request = T; type Response = TlsStream; type Error = Error; type Service = NativeTlsAcceptorService; @@ -83,11 +82,10 @@ impl Clone for NativeTlsAcceptorService { } } -impl Service for NativeTlsAcceptorService +impl Service for NativeTlsAcceptorService where T: AsyncRead + AsyncWrite + Unpin + 'static, { - type Request = T; type Response = TlsStream; type Error = Error; type Future = LocalBoxFuture<'static, Result, Error>>; @@ -100,10 +98,10 @@ where } } - fn call(&mut self, req: Self::Request) -> Self::Future { + fn call(&mut self, io: T) -> Self::Future { let guard = self.conns.get(); let this = self.clone(); - async move { this.acceptor.accept(req).await } + async move { this.acceptor.accept(io).await } .map_ok(move |io| { // Required to preserve `CounterGuard` until `Self::Future` is completely resolved. let _ = guard; diff --git a/actix-tls/src/openssl.rs b/actix-tls/src/accept/openssl.rs similarity index 69% rename from actix-tls/src/openssl.rs rename to actix-tls/src/accept/openssl.rs index 46dc10de..44877b24 100644 --- a/actix-tls/src/openssl.rs +++ b/actix-tls/src/accept/openssl.rs @@ -6,12 +6,17 @@ use std::task::{Context, Poll}; use actix_codec::{AsyncRead, AsyncWrite}; use actix_service::{Service, ServiceFactory}; use actix_utils::counter::{Counter, CounterGuard}; -use futures_util::future::{ok, FutureExt, LocalBoxFuture, Ready}; +use futures_util::{ + future::{ok, Ready}, + ready, +}; -pub use open_ssl::ssl::{AlpnError, SslAcceptor, SslAcceptorBuilder}; -pub use tokio_openssl::{HandshakeError, SslStream}; +pub use openssl::ssl::{ + AlpnError, Error as SslError, HandshakeError, Ssl, SslAcceptor, SslAcceptorBuilder, +}; +pub use tokio_openssl::SslStream; -use crate::MAX_CONN_COUNTER; +use super::MAX_CONN_COUNTER; /// Accept TLS connections via `openssl` package. /// @@ -42,10 +47,12 @@ impl Clone for Acceptor { } } -impl ServiceFactory for Acceptor { - type Request = T; +impl ServiceFactory for Acceptor +where + T: AsyncRead + AsyncWrite + Unpin + 'static, +{ type Response = SslStream; - type Error = HandshakeError; + type Error = SslError; type Config = (); type Service = AcceptorService; type InitError = (); @@ -68,10 +75,12 @@ pub struct AcceptorService { io: PhantomData, } -impl Service for AcceptorService { - type Request = T; +impl Service for AcceptorService +where + T: AsyncRead + AsyncWrite + Unpin + 'static, +{ type Response = SslStream; - type Error = HandshakeError; + type Error = SslError; type Future = AcceptorServiceResponse; fn poll_ready(&mut self, ctx: &mut Context<'_>) -> Poll> { @@ -82,15 +91,14 @@ impl Service for AcceptorService } } - fn call(&mut self, req: Self::Request) -> Self::Future { + fn call(&mut self, io: T) -> Self::Future { let acc = self.acceptor.clone(); + let ssl_ctx = acc.into_context(); + let ssl = Ssl::new(&ssl_ctx).expect("Provided SSL acceptor was invalid."); + AcceptorServiceResponse { _guard: self.conns.get(), - fut: async move { - let acc = acc; - tokio_openssl::accept(&acc, req).await - } - .boxed_local(), + stream: Some(SslStream::new(ssl, io).unwrap()), } } } @@ -99,15 +107,15 @@ pub struct AcceptorServiceResponse where T: AsyncRead + AsyncWrite, { - fut: LocalBoxFuture<'static, Result, HandshakeError>>, + stream: Option>, _guard: CounterGuard, } impl Future for AcceptorServiceResponse { - type Output = Result, HandshakeError>; + type Output = Result, SslError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let io = futures_util::ready!(Pin::new(&mut self.fut).poll(cx))?; - Poll::Ready(Ok(io)) + ready!(Pin::new(self.stream.as_mut().unwrap()).poll_accept(cx))?; + Poll::Ready(Ok(self.stream.take().expect("SSL connect has resolved."))) } } diff --git a/actix-tls/src/rustls.rs b/actix-tls/src/accept/rustls.rs similarity index 85% rename from actix-tls/src/rustls.rs rename to actix-tls/src/accept/rustls.rs index 3c3c60bf..1c40757e 100644 --- a/actix-tls/src/rustls.rs +++ b/actix-tls/src/accept/rustls.rs @@ -11,11 +11,11 @@ use actix_utils::counter::{Counter, CounterGuard}; use futures_util::future::{ok, Ready}; use tokio_rustls::{Accept, TlsAcceptor}; -pub use rust_tls::{ServerConfig, Session}; +pub use rustls::{ServerConfig, Session}; pub use tokio_rustls::server::TlsStream; pub use webpki_roots::TLS_SERVER_ROOTS; -use crate::MAX_CONN_COUNTER; +use super::MAX_CONN_COUNTER; /// Accept TLS connections via `rustls` package. /// @@ -25,7 +25,10 @@ pub struct Acceptor { io: PhantomData, } -impl Acceptor { +impl Acceptor +where + T: AsyncRead + AsyncWrite, +{ /// Create Rustls based `Acceptor` service factory. #[inline] pub fn new(config: ServerConfig) -> Self { @@ -46,8 +49,10 @@ impl Clone for Acceptor { } } -impl ServiceFactory for Acceptor { - type Request = T; +impl ServiceFactory for Acceptor +where + T: AsyncRead + AsyncWrite + Unpin, +{ type Response = TlsStream; type Error = io::Error; type Service = AcceptorService; @@ -74,8 +79,10 @@ pub struct AcceptorService { conns: Counter, } -impl Service for AcceptorService { - type Request = T; +impl Service for AcceptorService +where + T: AsyncRead + AsyncWrite + Unpin, +{ type Response = TlsStream; type Error = io::Error; type Future = AcceptorServiceFut; @@ -88,7 +95,7 @@ impl Service for AcceptorService { } } - fn call(&mut self, req: Self::Request) -> Self::Future { + fn call(&mut self, req: T) -> Self::Future { AcceptorServiceFut { _guard: self.conns.get(), fut: self.acceptor.accept(req), @@ -104,7 +111,10 @@ where _guard: CounterGuard, } -impl Future for AcceptorServiceFut { +impl Future for AcceptorServiceFut +where + T: AsyncRead + AsyncWrite + Unpin, +{ type Output = Result, io::Error>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { diff --git a/actix-connect/src/connect.rs b/actix-tls/src/connect/connect.rs similarity index 100% rename from actix-connect/src/connect.rs rename to actix-tls/src/connect/connect.rs diff --git a/actix-connect/src/connector.rs b/actix-tls/src/connect/connector.rs similarity index 99% rename from actix-connect/src/connector.rs rename to actix-tls/src/connect/connector.rs index d3ef9813..c7a31e0a 100644 --- a/actix-connect/src/connector.rs +++ b/actix-tls/src/connect/connector.rs @@ -9,6 +9,7 @@ use std::task::{Context, Poll}; use actix_rt::net::TcpStream; use actix_service::{Service, ServiceFactory}; use futures_util::future::{err, ok, BoxFuture, Either, FutureExt, Ready}; +use log::{error, trace}; use super::connect::{Address, Connect, Connection}; use super::error::ConnectError; diff --git a/actix-connect/src/error.rs b/actix-tls/src/connect/error.rs similarity index 100% rename from actix-connect/src/error.rs rename to actix-tls/src/connect/error.rs diff --git a/actix-connect/src/lib.rs b/actix-tls/src/connect/mod.rs similarity index 93% rename from actix-connect/src/lib.rs rename to actix-tls/src/connect/mod.rs index d2111d41..753cfc33 100644 --- a/actix-connect/src/lib.rs +++ b/actix-tls/src/connect/mod.rs @@ -5,22 +5,12 @@ //! * `openssl` - enables TLS support via `openssl` crate //! * `rustls` - enables TLS support via `rustls` crate -#![deny(rust_2018_idioms, nonstandard_style)] -#![recursion_limit = "128"] -#![doc(html_logo_url = "https://actix.rs/img/logo.png")] -#![doc(html_favicon_url = "https://actix.rs/favicon.ico")] - -#[macro_use] -extern crate log; - mod connect; mod connector; mod error; mod resolve; mod service; pub mod ssl; - -#[cfg(feature = "uri")] mod uri; use actix_rt::{net::TcpStream, Arbiter}; diff --git a/actix-connect/src/resolve.rs b/actix-tls/src/connect/resolve.rs similarity index 97% rename from actix-connect/src/resolve.rs rename to actix-tls/src/connect/resolve.rs index 2c75cc0d..61535faa 100644 --- a/actix-connect/src/resolve.rs +++ b/actix-tls/src/connect/resolve.rs @@ -6,12 +6,13 @@ use std::task::{Context, Poll}; use actix_service::{Service, ServiceFactory}; use futures_util::future::{ok, Either, Ready}; +use log::trace; use trust_dns_resolver::TokioAsyncResolver as AsyncResolver; use trust_dns_resolver::{error::ResolveError, lookup_ip::LookupIp}; -use crate::connect::{Address, Connect}; -use crate::error::ConnectError; -use crate::get_default_resolver; +use super::connect::{Address, Connect}; +use super::error::ConnectError; +use super::get_default_resolver; /// DNS Resolver Service factory pub struct ResolverFactory { diff --git a/actix-connect/src/service.rs b/actix-tls/src/connect/service.rs similarity index 97% rename from actix-connect/src/service.rs rename to actix-tls/src/connect/service.rs index b942d230..59fe20cc 100644 --- a/actix-connect/src/service.rs +++ b/actix-tls/src/connect/service.rs @@ -8,10 +8,10 @@ use either::Either; use futures_util::future::{ok, Ready}; use trust_dns_resolver::TokioAsyncResolver as AsyncResolver; -use crate::connect::{Address, Connect, Connection}; -use crate::connector::{TcpConnector, TcpConnectorFactory}; -use crate::error::ConnectError; -use crate::resolve::{Resolver, ResolverFactory}; +use super::connect::{Address, Connect, Connection}; +use super::connector::{TcpConnector, TcpConnectorFactory}; +use super::error::ConnectError; +use super::resolve::{Resolver, ResolverFactory}; pub struct ConnectServiceFactory { tcp: TcpConnectorFactory, diff --git a/actix-connect/src/ssl/mod.rs b/actix-tls/src/connect/ssl/mod.rs similarity index 100% rename from actix-connect/src/ssl/mod.rs rename to actix-tls/src/connect/ssl/mod.rs diff --git a/actix-connect/src/ssl/openssl.rs b/actix-tls/src/connect/ssl/openssl.rs similarity index 80% rename from actix-connect/src/ssl/openssl.rs rename to actix-tls/src/connect/ssl/openssl.rs index a9bcc3c7..f249717d 100644 --- a/actix-connect/src/ssl/openssl.rs +++ b/actix-tls/src/connect/ssl/openssl.rs @@ -4,16 +4,19 @@ use std::pin::Pin; use std::task::{Context, Poll}; use std::{fmt, io}; -pub use open_ssl::ssl::{Error as SslError, SslConnector, SslMethod}; -pub use tokio_openssl::{HandshakeError, SslStream}; - use actix_codec::{AsyncRead, AsyncWrite}; use actix_rt::net::TcpStream; use actix_service::{Service, ServiceFactory}; -use futures_util::future::{err, ok, Either, FutureExt, LocalBoxFuture, Ready}; +use futures_util::{ + future::{err, ok, Either, Ready}, + ready, +}; +use log::trace; +pub use openssl::ssl::{Error as SslError, HandshakeError, SslConnector, SslMethod}; +pub use tokio_openssl::SslStream; use trust_dns_resolver::TokioAsyncResolver as AsyncResolver; -use crate::{ +use crate::connect::{ Address, Connect, ConnectError, ConnectService, ConnectServiceFactory, Connection, }; @@ -54,12 +57,11 @@ impl Clone for OpensslConnector { } } -impl ServiceFactory for OpensslConnector +impl ServiceFactory> for OpensslConnector where T: Address + 'static, U: AsyncRead + AsyncWrite + Unpin + fmt::Debug + 'static, { - type Request = Connection; type Response = Connection>; type Error = io::Error; type Config = (); @@ -89,12 +91,11 @@ impl Clone for OpensslConnectorService { } } -impl Service for OpensslConnectorService +impl Service> for OpensslConnectorService where T: Address + 'static, U: AsyncRead + AsyncWrite + Unpin + fmt::Debug + 'static, { - type Request = Connection; type Response = Connection>; type Error = io::Error; #[allow(clippy::type_complexity)] @@ -109,18 +110,23 @@ where match self.connector.configure() { Err(e) => Either::Right(err(io::Error::new(io::ErrorKind::Other, e))), - Ok(config) => Either::Left(ConnectAsyncExt { - fut: async move { tokio_openssl::connect(config, &host, io).await } - .boxed_local(), - stream: Some(stream), - _t: PhantomData, - }), + Ok(config) => { + let ssl = config + .into_ssl(&host) + .expect("SSL connect configuration was invalid."); + + Either::Left(ConnectAsyncExt { + io: Some(SslStream::new(ssl, io).unwrap()), + stream: Some(stream), + _t: PhantomData, + }) + } } } } pub struct ConnectAsyncExt { - fut: LocalBoxFuture<'static, Result, HandshakeError>>, + io: Option>, stream: Option>, _t: PhantomData, } @@ -134,17 +140,16 @@ where fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); - match Pin::new(&mut this.fut).poll(cx) { - Poll::Ready(Ok(stream)) => { - let s = this.stream.take().unwrap(); - trace!("SSL Handshake success: {:?}", s.host()); - Poll::Ready(Ok(s.replace(stream).1)) + match ready!(Pin::new(this.io.as_mut().unwrap()).poll_connect(cx)) { + Ok(_) => { + let stream = this.stream.take().unwrap(); + trace!("SSL Handshake success: {:?}", stream.host()); + Poll::Ready(Ok(stream.replace(this.io.take().unwrap()).1)) } - Poll::Ready(Err(e)) => { + Err(e) => { trace!("SSL Handshake error: {:?}", e); Poll::Ready(Err(io::Error::new(io::ErrorKind::Other, format!("{}", e)))) } - Poll::Pending => Poll::Pending, } } } @@ -192,8 +197,7 @@ impl Clone for OpensslConnectServiceFactory { } } -impl ServiceFactory for OpensslConnectServiceFactory { - type Request = Connect; +impl ServiceFactory> for OpensslConnectServiceFactory { type Response = SslStream; type Error = ConnectError; type Config = (); @@ -212,8 +216,7 @@ pub struct OpensslConnectService { openssl: OpensslConnectorService, } -impl Service for OpensslConnectService { - type Request = Connect; +impl Service> for OpensslConnectService { type Response = SslStream; type Error = ConnectError; type Future = OpensslConnectServiceResponse; @@ -230,8 +233,10 @@ impl Service for OpensslConnectService { } pub struct OpensslConnectServiceResponse { - fut1: Option< as Service>::Future>, - fut2: Option< as Service>::Future>, + fut1: Option< as Service>>::Future>, + fut2: Option< + as Service>>::Future, + >, openssl: OpensslConnectorService, } @@ -240,7 +245,7 @@ impl Future for OpensslConnectServiceResponse { fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { if let Some(ref mut fut) = self.fut1 { - match futures_util::ready!(Pin::new(fut).poll(cx)) { + match ready!(Pin::new(fut).poll(cx)) { Ok(res) => { let _ = self.fut1.take(); self.fut2 = Some(self.openssl.call(res)); @@ -250,7 +255,7 @@ impl Future for OpensslConnectServiceResponse { } if let Some(ref mut fut) = self.fut2 { - match futures_util::ready!(Pin::new(fut).poll(cx)) { + match ready!(Pin::new(fut).poll(cx)) { Ok(connect) => Poll::Ready(Ok(connect.into_parts().0)), Err(e) => Poll::Ready(Err(ConnectError::Io(io::Error::new( io::ErrorKind::Other, diff --git a/actix-connect/src/ssl/rustls.rs b/actix-tls/src/connect/ssl/rustls.rs similarity index 93% rename from actix-connect/src/ssl/rustls.rs rename to actix-tls/src/connect/ssl/rustls.rs index 984fbe49..f5259280 100644 --- a/actix-connect/src/ssl/rustls.rs +++ b/actix-tls/src/connect/ssl/rustls.rs @@ -5,16 +5,17 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -pub use rust_tls::Session; +pub use rustls::Session; pub use tokio_rustls::{client::TlsStream, rustls::ClientConfig}; use actix_codec::{AsyncRead, AsyncWrite}; use actix_service::{Service, ServiceFactory}; use futures_util::future::{ok, Ready}; +use log::trace; use tokio_rustls::{Connect, TlsConnector}; use webpki::DNSNameRef; -use crate::{Address, Connection}; +use crate::connect::{Address, Connection}; /// Rustls connector factory pub struct RustlsConnector { @@ -53,11 +54,10 @@ impl Clone for RustlsConnector { } } -impl ServiceFactory for RustlsConnector +impl ServiceFactory> for RustlsConnector where U: AsyncRead + AsyncWrite + Unpin + fmt::Debug, { - type Request = Connection; type Response = Connection>; type Error = std::io::Error; type Config = (); @@ -87,11 +87,10 @@ impl Clone for RustlsConnectorService { } } -impl Service for RustlsConnectorService +impl Service> for RustlsConnectorService where U: AsyncRead + AsyncWrite + Unpin + fmt::Debug, { - type Request = Connection; type Response = Connection>; type Error = std::io::Error; type Future = ConnectAsyncExt; diff --git a/actix-connect/src/uri.rs b/actix-tls/src/connect/uri.rs similarity index 97% rename from actix-connect/src/uri.rs rename to actix-tls/src/connect/uri.rs index 5f5f15de..b208a8b3 100644 --- a/actix-connect/src/uri.rs +++ b/actix-tls/src/connect/uri.rs @@ -1,6 +1,6 @@ use http::Uri; -use crate::Address; +use super::Address; impl Address for Uri { fn host(&self) -> &str { diff --git a/actix-tls/src/lib.rs b/actix-tls/src/lib.rs index 8cc18046..1fa08b6b 100644 --- a/actix-tls/src/lib.rs +++ b/actix-tls/src/lib.rs @@ -1,46 +1,17 @@ -//! TLS acceptor services for Actix ecosystem. -//! -//! ## Crate Features -//! * `openssl` - TLS acceptor using the `openssl` crate. -//! * `rustls` - TLS acceptor using the `rustls` crate. -//! * `nativetls` - TLS acceptor using the `native-tls` crate. +//! TLS acceptor and connector services for Actix ecosystem #![deny(rust_2018_idioms, nonstandard_style)] #![doc(html_logo_url = "https://actix.rs/img/logo.png")] #![doc(html_favicon_url = "https://actix.rs/favicon.ico")] -use std::sync::atomic::{AtomicUsize, Ordering}; - -use actix_utils::counter::Counter; - +#[cfg(feature = "native-tls")] +extern crate tls_native_tls as native_tls; #[cfg(feature = "openssl")] -pub mod openssl; - +extern crate tls_openssl as openssl; #[cfg(feature = "rustls")] -pub mod rustls; +extern crate tls_rustls as rustls; -#[cfg(feature = "nativetls")] -pub mod nativetls; - -pub(crate) static MAX_CONN: AtomicUsize = AtomicUsize::new(256); - -thread_local! { - static MAX_CONN_COUNTER: Counter = Counter::new(MAX_CONN.load(Ordering::Relaxed)); -} - -/// Sets the maximum per-worker concurrent TLS connection limit. -/// -/// All listeners will stop accepting connections when this limit is reached. -/// It can be used to regulate the global TLS CPU usage. -/// -/// By default, the connection limit is 256. -pub fn max_concurrent_tls_connect(num: usize) { - MAX_CONN.store(num, Ordering::Relaxed); -} - -/// TLS error combined with service error. -#[derive(Debug)] -pub enum TlsError { - Tls(E1), - Service(E2), -} +#[cfg(feature = "accept")] +pub mod accept; +#[cfg(feature = "connect")] +pub mod connect; From 0934078947e294f6a70b56ecbf50cd41a963e47b Mon Sep 17 00:00:00 2001 From: Rob Ede Date: Tue, 29 Dec 2020 01:04:21 +0000 Subject: [PATCH 6/8] prepare tls beta release --- actix-tls/CHANGES.md | 7 +++++-- actix-tls/Cargo.toml | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/actix-tls/CHANGES.md b/actix-tls/CHANGES.md index 9f03bfeb..dcb477d8 100644 --- a/actix-tls/CHANGES.md +++ b/actix-tls/CHANGES.md @@ -1,9 +1,12 @@ # Changes -## Unreleased - 2020-xx-xx +## Unreleased - 2021-xx-xx + + +## 3.0.0-beta.1 - 2020-12-29 * Move acceptors under `accept` module. [#238] * Merge `actix-connect` crate under `connect` module. [#238] -* Add feature flags to enable acceptors and/or connectors individually. +* Add feature flags to enable acceptors and/or connectors individually. [#238] [#238]: https://github.com/actix/actix-net/pull/238 diff --git a/actix-tls/Cargo.toml b/actix-tls/Cargo.toml index de2d859f..af0b4238 100644 --- a/actix-tls/Cargo.toml +++ b/actix-tls/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "actix-tls" -version = "2.0.0" +version = "3.0.0-beta.1" authors = ["Nikolay Kim "] description = "TLS acceptor and connector services for Actix ecosystem" keywords = ["network", "tls", "ssl", "async", "transport"] From 03eb96d6d4007e9c9492ca0b07b8f39238a8afaa Mon Sep 17 00:00:00 2001 From: fakeshadow <24548779@qq.com> Date: Tue, 29 Dec 2020 19:36:17 +0800 Subject: [PATCH 7/8] fix actix-tls tests (#241) --- Cargo.toml | 4 + actix-threadpool/Cargo.toml | 2 +- actix-tls/Cargo.toml | 20 +++-- actix-tls/src/accept/nativetls.rs | 52 ++++------- actix-tls/src/accept/openssl.rs | 30 +++---- actix-tls/src/accept/rustls.rs | 38 +++----- actix-tls/src/connect/connector.rs | 91 ++++++++++--------- actix-tls/src/connect/mod.rs | 5 +- actix-tls/src/connect/ssl/openssl.rs | 60 +++++-------- actix-tls/src/connect/ssl/rustls.rs | 64 ++++++------- actix-tls/tests/test_connect.rs | 130 +++++++++++++++++++++++++++ string/Cargo.toml | 2 +- 12 files changed, 291 insertions(+), 207 deletions(-) create mode 100644 actix-tls/tests/test_connect.rs diff --git a/Cargo.toml b/Cargo.toml index 533fbc17..342f866a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -27,3 +27,7 @@ actix-tracing = { path = "actix-tracing" } actix-utils = { path = "actix-utils" } actix-router = { path = "router" } bytestring = { path = "string" } + +# FIXME: remove override +trust-dns-proto = { git = "https://github.com/bluejekyll/trust-dns.git", branch = "main" } +trust-dns-resolver = { git = "https://github.com/bluejekyll/trust-dns.git", branch = "main" } \ No newline at end of file diff --git a/actix-threadpool/Cargo.toml b/actix-threadpool/Cargo.toml index 5998bae7..3a633d62 100644 --- a/actix-threadpool/Cargo.toml +++ b/actix-threadpool/Cargo.toml @@ -19,7 +19,7 @@ path = "src/lib.rs" [dependencies] derive_more = "0.99.2" -futures-channel = "0.3.1" +futures-channel = "0.3.7" parking_lot = "0.11" lazy_static = "1.3" log = "0.4" diff --git a/actix-tls/Cargo.toml b/actix-tls/Cargo.toml index af0b4238..1c24e0eb 100644 --- a/actix-tls/Cargo.toml +++ b/actix-tls/Cargo.toml @@ -12,7 +12,7 @@ license = "MIT OR Apache-2.0" edition = "2018" [package.metadata.docs.rs] -features = ["openssl", "rustls", "native-tls", "accept", "connect", "http"] +features = ["openssl", "rustls", "native-tls", "accept", "connect", "uri"] [lib] name = "actix_tls" @@ -23,13 +23,13 @@ name = "basic" required-features = ["accept", "rustls"] [features] -default = ["accept", "connect", "http"] +default = ["accept", "connect", "uri"] # enable acceptor services accept = [] # enable connector services -connect = [] +connect = ["trust-dns-proto/tokio-runtime", "trust-dns-resolver/tokio-runtime", "trust-dns-resolver/system-config"] # use openssl impls openssl = ["tls-openssl", "tokio-openssl"] @@ -40,6 +40,9 @@ rustls = ["tls-rustls", "webpki", "webpki-roots", "tokio-rustls"] # use native-tls impls native-tls = ["tls-native-tls", "tokio-native-tls"] +# support http::Uri as connect address +uri = ["http"] + [dependencies] actix-codec = "0.4.0-beta.1" actix-rt = "2.0.0-beta.1" @@ -49,10 +52,12 @@ actix-utils = "3.0.0-beta.1" derive_more = "0.99.5" either = "1.6" futures-util = { version = "0.3.7", default-features = false } -http = { version = "0.2.0", optional = true } +http = { version = "0.2.2", optional = true } log = "0.4" -trust-dns-proto = { version = "0.19", default-features = false, features = ["tokio-runtime"] } -trust-dns-resolver = { version = "0.19", default-features = false, features = ["tokio-runtime", "system-config"] } + +# resolver +trust-dns-proto = { version = "0.20.0-alpha.3", default-features = false, optional = true } +trust-dns-resolver = { version = "0.20.0-alpha.3", default-features = false, optional = true } # openssl tls-openssl = { package = "openssl", version = "0.10", optional = true } @@ -74,5 +79,6 @@ tokio-native-tls = { version = "0.3", optional = true } actix-server = "2.0.0-beta.1" actix-testing = "2.0.0-beta.1" bytes = "1" -log = "0.4" env_logger = "0.8" +futures-util = { version = "0.3.7", default-features = false, features = ["sink"] } +log = "0.4" diff --git a/actix-tls/src/accept/nativetls.rs b/actix-tls/src/accept/nativetls.rs index 12d23855..5d80ce8b 100644 --- a/actix-tls/src/accept/nativetls.rs +++ b/actix-tls/src/accept/nativetls.rs @@ -1,10 +1,9 @@ -use std::marker::PhantomData; use std::task::{Context, Poll}; use actix_codec::{AsyncRead, AsyncWrite}; use actix_service::{Service, ServiceFactory}; use actix_utils::counter::Counter; -use futures_util::future::{self, FutureExt, LocalBoxFuture, TryFutureExt}; +use futures_util::future::{ready, LocalBoxFuture, Ready}; pub use native_tls::Error; pub use tokio_native_tls::{TlsAcceptor, TlsStream}; @@ -14,75 +13,64 @@ use super::MAX_CONN_COUNTER; /// Accept TLS connections via `native-tls` package. /// /// `native-tls` feature enables this `Acceptor` type. -pub struct Acceptor { +pub struct Acceptor { acceptor: TlsAcceptor, - io: PhantomData, } -impl Acceptor -where - T: AsyncRead + AsyncWrite + Unpin, -{ +impl Acceptor { /// Create `native-tls` based `Acceptor` service factory. #[inline] pub fn new(acceptor: TlsAcceptor) -> Self { - Acceptor { - acceptor, - io: PhantomData, - } + Acceptor { acceptor } } } -impl Clone for Acceptor { +impl Clone for Acceptor { #[inline] fn clone(&self) -> Self { Self { acceptor: self.acceptor.clone(), - io: PhantomData, } } } -impl ServiceFactory for Acceptor +impl ServiceFactory for Acceptor where T: AsyncRead + AsyncWrite + Unpin + 'static, { type Response = TlsStream; type Error = Error; - type Service = NativeTlsAcceptorService; - type Config = (); + + type Service = NativeTlsAcceptorService; type InitError = (); - type Future = future::Ready>; + type Future = Ready>; fn new_service(&self, _: ()) -> Self::Future { MAX_CONN_COUNTER.with(|conns| { - future::ok(NativeTlsAcceptorService { + ready(Ok(NativeTlsAcceptorService { acceptor: self.acceptor.clone(), conns: conns.clone(), - io: PhantomData, - }) + })) }) } } -pub struct NativeTlsAcceptorService { +pub struct NativeTlsAcceptorService { acceptor: TlsAcceptor, - io: PhantomData, conns: Counter, } -impl Clone for NativeTlsAcceptorService { +impl Clone for NativeTlsAcceptorService { fn clone(&self) -> Self { Self { acceptor: self.acceptor.clone(), - io: PhantomData, conns: self.conns.clone(), } } } -impl Service for NativeTlsAcceptorService +impl Service for NativeTlsAcceptorService where T: AsyncRead + AsyncWrite + Unpin + 'static, { @@ -101,12 +89,10 @@ where fn call(&mut self, io: T) -> Self::Future { let guard = self.conns.get(); let this = self.clone(); - async move { this.acceptor.accept(io).await } - .map_ok(move |io| { - // Required to preserve `CounterGuard` until `Self::Future` is completely resolved. - let _ = guard; - io - }) - .boxed_local() + Box::pin(async move { + let io = this.acceptor.accept(io).await; + drop(guard); + io + }) } } diff --git a/actix-tls/src/accept/openssl.rs b/actix-tls/src/accept/openssl.rs index 44877b24..e51808a3 100644 --- a/actix-tls/src/accept/openssl.rs +++ b/actix-tls/src/accept/openssl.rs @@ -1,5 +1,4 @@ use std::future::Future; -use std::marker::PhantomData; use std::pin::Pin; use std::task::{Context, Poll}; @@ -7,7 +6,7 @@ use actix_codec::{AsyncRead, AsyncWrite}; use actix_service::{Service, ServiceFactory}; use actix_utils::counter::{Counter, CounterGuard}; use futures_util::{ - future::{ok, Ready}, + future::{ready, Ready}, ready, }; @@ -21,61 +20,54 @@ use super::MAX_CONN_COUNTER; /// Accept TLS connections via `openssl` package. /// /// `openssl` feature enables this `Acceptor` type. -pub struct Acceptor { +pub struct Acceptor { acceptor: SslAcceptor, - io: PhantomData, } -impl Acceptor { +impl Acceptor { /// Create OpenSSL based `Acceptor` service factory. #[inline] pub fn new(acceptor: SslAcceptor) -> Self { - Acceptor { - acceptor, - io: PhantomData, - } + Acceptor { acceptor } } } -impl Clone for Acceptor { +impl Clone for Acceptor { #[inline] fn clone(&self) -> Self { Self { acceptor: self.acceptor.clone(), - io: PhantomData, } } } -impl ServiceFactory for Acceptor +impl ServiceFactory for Acceptor where T: AsyncRead + AsyncWrite + Unpin + 'static, { type Response = SslStream; type Error = SslError; type Config = (); - type Service = AcceptorService; + type Service = AcceptorService; type InitError = (); type Future = Ready>; fn new_service(&self, _: ()) -> Self::Future { MAX_CONN_COUNTER.with(|conns| { - ok(AcceptorService { + ready(Ok(AcceptorService { acceptor: self.acceptor.clone(), conns: conns.clone(), - io: PhantomData, - }) + })) }) } } -pub struct AcceptorService { +pub struct AcceptorService { acceptor: SslAcceptor, conns: Counter, - io: PhantomData, } -impl Service for AcceptorService +impl Service for AcceptorService where T: AsyncRead + AsyncWrite + Unpin + 'static, { diff --git a/actix-tls/src/accept/rustls.rs b/actix-tls/src/accept/rustls.rs index 1c40757e..a6686f44 100644 --- a/actix-tls/src/accept/rustls.rs +++ b/actix-tls/src/accept/rustls.rs @@ -1,6 +1,5 @@ use std::future::Future; use std::io; -use std::marker::PhantomData; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -8,7 +7,7 @@ use std::task::{Context, Poll}; use actix_codec::{AsyncRead, AsyncWrite}; use actix_service::{Service, ServiceFactory}; use actix_utils::counter::{Counter, CounterGuard}; -use futures_util::future::{ok, Ready}; +use futures_util::future::{ready, Ready}; use tokio_rustls::{Accept, TlsAcceptor}; pub use rustls::{ServerConfig, Session}; @@ -20,66 +19,58 @@ use super::MAX_CONN_COUNTER; /// Accept TLS connections via `rustls` package. /// /// `rustls` feature enables this `Acceptor` type. -pub struct Acceptor { +pub struct Acceptor { config: Arc, - io: PhantomData, } -impl Acceptor -where - T: AsyncRead + AsyncWrite, -{ +impl Acceptor { /// Create Rustls based `Acceptor` service factory. #[inline] pub fn new(config: ServerConfig) -> Self { Acceptor { config: Arc::new(config), - io: PhantomData, } } } -impl Clone for Acceptor { +impl Clone for Acceptor { #[inline] fn clone(&self) -> Self { Self { config: self.config.clone(), - io: PhantomData, } } } -impl ServiceFactory for Acceptor +impl ServiceFactory for Acceptor where T: AsyncRead + AsyncWrite + Unpin, { type Response = TlsStream; type Error = io::Error; - type Service = AcceptorService; - type Config = (); + + type Service = AcceptorService; type InitError = (); type Future = Ready>; fn new_service(&self, _: ()) -> Self::Future { MAX_CONN_COUNTER.with(|conns| { - ok(AcceptorService { + ready(Ok(AcceptorService { acceptor: self.config.clone().into(), conns: conns.clone(), - io: PhantomData, - }) + })) }) } } /// Rustls based `Acceptor` service -pub struct AcceptorService { +pub struct AcceptorService { acceptor: TlsAcceptor, - io: PhantomData, conns: Counter, } -impl Service for AcceptorService +impl Service for AcceptorService where T: AsyncRead + AsyncWrite + Unpin, { @@ -119,11 +110,6 @@ where fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); - - let res = futures_util::ready!(Pin::new(&mut this.fut).poll(cx)); - match res { - Ok(io) => Poll::Ready(Ok(io)), - Err(e) => Poll::Ready(Err(e)), - } + Pin::new(&mut this.fut).poll(cx) } } diff --git a/actix-tls/src/connect/connector.rs b/actix-tls/src/connect/connector.rs index c7a31e0a..a0a6b8b5 100644 --- a/actix-tls/src/connect/connector.rs +++ b/actix-tls/src/connect/connector.rs @@ -8,7 +8,7 @@ use std::task::{Context, Poll}; use actix_rt::net::TcpStream; use actix_service::{Service, ServiceFactory}; -use futures_util::future::{err, ok, BoxFuture, Either, FutureExt, Ready}; +use futures_util::future::{ready, Ready}; use log::{error, trace}; use super::connect::{Address, Connect, Connection}; @@ -50,7 +50,7 @@ impl ServiceFactory> for TcpConnectorFactory { type Future = Ready>; fn new_service(&self, _: ()) -> Self::Future { - ok(self.service()) + ready(Ok(self.service())) } } @@ -73,8 +73,7 @@ impl Clone for TcpConnector { impl Service> for TcpConnector { type Response = Connection; type Error = ConnectError; - #[allow(clippy::type_complexity)] - type Future = Either, Ready>>; + type Future = TcpConnectorResponse; actix_service::always_ready!(); @@ -83,21 +82,26 @@ impl Service> for TcpConnector { let Connect { req, addr, .. } = req; if let Some(addr) = addr { - Either::Left(TcpConnectorResponse::new(req, port, addr)) + TcpConnectorResponse::new(req, port, addr) } else { error!("TCP connector: got unresolved address"); - Either::Right(err(ConnectError::Unresolved)) + TcpConnectorResponse::Error(Some(ConnectError::Unresolved)) } } } +type LocalBoxFuture<'a, T> = Pin + 'a>>; + #[doc(hidden)] /// TCP stream connector response future -pub struct TcpConnectorResponse { - req: Option, - port: u16, - addrs: Option>, - stream: Option>>, +pub enum TcpConnectorResponse { + Response { + req: Option, + port: u16, + addrs: Option>, + stream: Option>>, + }, + Error(Option), } impl TcpConnectorResponse { @@ -113,13 +117,13 @@ impl TcpConnectorResponse { ); match addr { - either::Either::Left(addr) => TcpConnectorResponse { + either::Either::Left(addr) => TcpConnectorResponse::Response { req: Some(req), port, addrs: None, - stream: Some(TcpStream::connect(addr).boxed()), + stream: Some(Box::pin(TcpStream::connect(addr))), }, - either::Either::Right(addrs) => TcpConnectorResponse { + either::Either::Right(addrs) => TcpConnectorResponse::Response { req: Some(req), port, addrs: Some(addrs), @@ -134,36 +138,43 @@ impl Future for TcpConnectorResponse { fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); - - // connect - loop { - if let Some(new) = this.stream.as_mut() { - match new.as_mut().poll(cx) { - Poll::Ready(Ok(sock)) => { - let req = this.req.take().unwrap(); - trace!( - "TCP connector - successfully connected to connecting to {:?} - {:?}", - req.host(), sock.peer_addr() - ); - return Poll::Ready(Ok(Connection::new(sock, req))); - } - Poll::Pending => return Poll::Pending, - Poll::Ready(Err(err)) => { - trace!( - "TCP connector - failed to connect to connecting to {:?} port: {}", - this.req.as_ref().unwrap().host(), - this.port, - ); - if this.addrs.is_none() || this.addrs.as_ref().unwrap().is_empty() { - return Poll::Ready(Err(err.into())); + match this { + TcpConnectorResponse::Error(e) => Poll::Ready(Err(e.take().unwrap())), + // connect + TcpConnectorResponse::Response { + req, + port, + addrs, + stream, + } => loop { + if let Some(new) = stream.as_mut() { + match new.as_mut().poll(cx) { + Poll::Ready(Ok(sock)) => { + let req = req.take().unwrap(); + trace!( + "TCP connector - successfully connected to connecting to {:?} - {:?}", + req.host(), sock.peer_addr() + ); + return Poll::Ready(Ok(Connection::new(sock, req))); + } + Poll::Pending => return Poll::Pending, + Poll::Ready(Err(err)) => { + trace!( + "TCP connector - failed to connect to connecting to {:?} port: {}", + req.as_ref().unwrap().host(), + port, + ); + if addrs.is_none() || addrs.as_ref().unwrap().is_empty() { + return Poll::Ready(Err(err.into())); + } } } } - } - // try to connect - let addr = this.addrs.as_mut().unwrap().pop_front().unwrap(); - this.stream = Some(TcpStream::connect(addr).boxed()); + // try to connect + let addr = addrs.as_mut().unwrap().pop_front().unwrap(); + *stream = Some(Box::pin(TcpStream::connect(addr))); + }, } } } diff --git a/actix-tls/src/connect/mod.rs b/actix-tls/src/connect/mod.rs index 753cfc33..75312c59 100644 --- a/actix-tls/src/connect/mod.rs +++ b/actix-tls/src/connect/mod.rs @@ -11,6 +11,7 @@ mod error; mod resolve; mod service; pub mod ssl; +#[cfg(feature = "uri")] mod uri; use actix_rt::{net::TcpStream, Arbiter}; @@ -35,7 +36,7 @@ pub async fn start_resolver( cfg: ResolverConfig, opts: ResolverOpts, ) -> Result { - Ok(AsyncResolver::tokio(cfg, opts).await?) + Ok(AsyncResolver::tokio(cfg, opts)?) } struct DefaultResolver(AsyncResolver); @@ -52,7 +53,7 @@ pub(crate) async fn get_default_resolver() -> Result { +pub struct OpensslConnector { connector: SslConnector, - _t: PhantomData<(T, U)>, } -impl OpensslConnector { +impl OpensslConnector { pub fn new(connector: SslConnector) -> Self { - OpensslConnector { - connector, - _t: PhantomData, - } + OpensslConnector { connector } } } -impl OpensslConnector -where - T: Address + 'static, - U: AsyncRead + AsyncWrite + Unpin + fmt::Debug + 'static, -{ - pub fn service(connector: SslConnector) -> OpensslConnectorService { - OpensslConnectorService { - connector, - _t: PhantomData, - } +impl OpensslConnector { + pub fn service(connector: SslConnector) -> OpensslConnectorService { + OpensslConnectorService { connector } } } -impl Clone for OpensslConnector { +impl Clone for OpensslConnector { fn clone(&self) -> Self { Self { connector: self.connector.clone(), - _t: PhantomData, } } } -impl ServiceFactory> for OpensslConnector +impl ServiceFactory> for OpensslConnector where T: Address + 'static, U: AsyncRead + AsyncWrite + Unpin + fmt::Debug + 'static, @@ -65,33 +53,30 @@ where type Response = Connection>; type Error = io::Error; type Config = (); - type Service = OpensslConnectorService; + type Service = OpensslConnectorService; type InitError = (); type Future = Ready>; fn new_service(&self, _: ()) -> Self::Future { - ok(OpensslConnectorService { + ready(Ok(OpensslConnectorService { connector: self.connector.clone(), - _t: PhantomData, - }) + })) } } -pub struct OpensslConnectorService { +pub struct OpensslConnectorService { connector: SslConnector, - _t: PhantomData<(T, U)>, } -impl Clone for OpensslConnectorService { +impl Clone for OpensslConnectorService { fn clone(&self) -> Self { Self { connector: self.connector.clone(), - _t: PhantomData, } } } -impl Service> for OpensslConnectorService +impl Service> for OpensslConnectorService where T: Address + 'static, U: AsyncRead + AsyncWrite + Unpin + fmt::Debug + 'static, @@ -109,7 +94,7 @@ where let host = stream.host().to_string(); match self.connector.configure() { - Err(e) => Either::Right(err(io::Error::new(io::ErrorKind::Other, e))), + Err(e) => Either::Right(ready(Err(io::Error::new(io::ErrorKind::Other, e)))), Ok(config) => { let ssl = config .into_ssl(&host) @@ -156,7 +141,7 @@ where pub struct OpensslConnectServiceFactory { tcp: ConnectServiceFactory, - openssl: OpensslConnector, + openssl: OpensslConnector, } impl OpensslConnectServiceFactory { @@ -182,7 +167,6 @@ impl OpensslConnectServiceFactory { tcp: self.tcp.service(), openssl: OpensslConnectorService { connector: self.openssl.connector.clone(), - _t: PhantomData, }, } } @@ -206,14 +190,14 @@ impl ServiceFactory> for OpensslConnectServiceF type Future = Ready>; fn new_service(&self, _: ()) -> Self::Future { - ok(self.service()) + ready(Ok(self.service())) } } #[derive(Clone)] pub struct OpensslConnectService { tcp: ConnectService, - openssl: OpensslConnectorService, + openssl: OpensslConnectorService, } impl Service> for OpensslConnectService { @@ -234,10 +218,8 @@ impl Service> for OpensslConnectService { pub struct OpensslConnectServiceResponse { fut1: Option< as Service>>::Future>, - fut2: Option< - as Service>>::Future, - >, - openssl: OpensslConnectorService, + fut2: Option<>>::Future>, + openssl: OpensslConnectorService, } impl Future for OpensslConnectServiceResponse { diff --git a/actix-tls/src/connect/ssl/rustls.rs b/actix-tls/src/connect/ssl/rustls.rs index f5259280..390ba413 100644 --- a/actix-tls/src/connect/ssl/rustls.rs +++ b/actix-tls/src/connect/ssl/rustls.rs @@ -1,6 +1,5 @@ use std::fmt; use std::future::Future; -use std::marker::PhantomData; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -10,7 +9,10 @@ pub use tokio_rustls::{client::TlsStream, rustls::ClientConfig}; use actix_codec::{AsyncRead, AsyncWrite}; use actix_service::{Service, ServiceFactory}; -use futures_util::future::{ok, Ready}; +use futures_util::{ + future::{ready, Ready}, + ready, +}; use log::trace; use tokio_rustls::{Connect, TlsConnector}; use webpki::DNSNameRef; @@ -18,77 +20,63 @@ use webpki::DNSNameRef; use crate::connect::{Address, Connection}; /// Rustls connector factory -pub struct RustlsConnector { +pub struct RustlsConnector { connector: Arc, - _t: PhantomData<(T, U)>, } -impl RustlsConnector { +impl RustlsConnector { pub fn new(connector: Arc) -> Self { - RustlsConnector { - connector, - _t: PhantomData, - } + RustlsConnector { connector } } } -impl RustlsConnector -where - T: Address, - U: AsyncRead + AsyncWrite + Unpin + fmt::Debug, -{ - pub fn service(connector: Arc) -> RustlsConnectorService { - RustlsConnectorService { - connector, - _t: PhantomData, - } +impl RustlsConnector { + pub fn service(connector: Arc) -> RustlsConnectorService { + RustlsConnectorService { connector } } } -impl Clone for RustlsConnector { +impl Clone for RustlsConnector { fn clone(&self) -> Self { Self { connector: self.connector.clone(), - _t: PhantomData, } } } -impl ServiceFactory> for RustlsConnector +impl ServiceFactory> for RustlsConnector where U: AsyncRead + AsyncWrite + Unpin + fmt::Debug, { type Response = Connection>; type Error = std::io::Error; type Config = (); - type Service = RustlsConnectorService; + type Service = RustlsConnectorService; type InitError = (); type Future = Ready>; fn new_service(&self, _: ()) -> Self::Future { - ok(RustlsConnectorService { + ready(Ok(RustlsConnectorService { connector: self.connector.clone(), - _t: PhantomData, - }) + })) } } -pub struct RustlsConnectorService { +pub struct RustlsConnectorService { connector: Arc, - _t: PhantomData<(T, U)>, } -impl Clone for RustlsConnectorService { +impl Clone for RustlsConnectorService { fn clone(&self) -> Self { Self { connector: self.connector.clone(), - _t: PhantomData, } } } -impl Service> for RustlsConnectorService +impl Service> for RustlsConnectorService where + T: Address, U: AsyncRead + AsyncWrite + Unpin + fmt::Debug, { type Response = Connection>; @@ -114,20 +102,18 @@ pub struct ConnectAsyncExt { stream: Option>, } -impl Future for ConnectAsyncExt +impl Future for ConnectAsyncExt where + T: Address, U: AsyncRead + AsyncWrite + Unpin + fmt::Debug, { type Output = Result>, std::io::Error>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.get_mut(); - Poll::Ready( - futures_util::ready!(Pin::new(&mut this.fut).poll(cx)).map(|stream| { - let s = this.stream.take().unwrap(); - trace!("SSL Handshake success: {:?}", s.host()); - s.replace(stream).1 - }), - ) + let stream = ready!(Pin::new(&mut this.fut).poll(cx))?; + let s = this.stream.take().unwrap(); + trace!("SSL Handshake success: {:?}", s.host()); + Poll::Ready(Ok(s.replace(stream).1)) } } diff --git a/actix-tls/tests/test_connect.rs b/actix-tls/tests/test_connect.rs new file mode 100644 index 00000000..fd083893 --- /dev/null +++ b/actix-tls/tests/test_connect.rs @@ -0,0 +1,130 @@ +use std::io; + +use actix_codec::{BytesCodec, Framed}; +use actix_rt::net::TcpStream; +use actix_service::{fn_service, Service, ServiceFactory}; +use actix_testing::TestServer; +use bytes::Bytes; +use futures_util::sink::SinkExt; + +use actix_tls::connect::{ + self as actix_connect, + resolver::{ResolverConfig, ResolverOpts}, + Connect, +}; + +#[cfg(all(feature = "connect", feature = "openssl"))] +#[actix_rt::test] +async fn test_string() { + let srv = TestServer::with(|| { + fn_service(|io: TcpStream| async { + let mut framed = Framed::new(io, BytesCodec); + framed.send(Bytes::from_static(b"test")).await?; + Ok::<_, io::Error>(()) + }) + }); + + let mut conn = actix_connect::default_connector(); + let addr = format!("localhost:{}", srv.port()); + let con = conn.call(addr.into()).await.unwrap(); + assert_eq!(con.peer_addr().unwrap(), srv.addr()); +} + +#[cfg(feature = "rustls")] +#[actix_rt::test] +async fn test_rustls_string() { + let srv = TestServer::with(|| { + fn_service(|io: TcpStream| async { + let mut framed = Framed::new(io, BytesCodec); + framed.send(Bytes::from_static(b"test")).await?; + Ok::<_, io::Error>(()) + }) + }); + + let mut conn = actix_connect::default_connector(); + let addr = format!("localhost:{}", srv.port()); + let con = conn.call(addr.into()).await.unwrap(); + assert_eq!(con.peer_addr().unwrap(), srv.addr()); +} + +#[actix_rt::test] +async fn test_static_str() { + let srv = TestServer::with(|| { + fn_service(|io: TcpStream| async { + let mut framed = Framed::new(io, BytesCodec); + framed.send(Bytes::from_static(b"test")).await?; + Ok::<_, io::Error>(()) + }) + }); + + let resolver = actix_connect::start_default_resolver().await.unwrap(); + let mut conn = actix_connect::new_connector(resolver.clone()); + + let con = conn.call(Connect::with("10", srv.addr())).await.unwrap(); + assert_eq!(con.peer_addr().unwrap(), srv.addr()); + + let connect = Connect::new(srv.host().to_owned()); + let mut conn = actix_connect::new_connector(resolver); + let con = conn.call(connect).await; + assert!(con.is_err()); +} + +#[actix_rt::test] +async fn test_new_service() { + let srv = TestServer::with(|| { + fn_service(|io: TcpStream| async { + let mut framed = Framed::new(io, BytesCodec); + framed.send(Bytes::from_static(b"test")).await?; + Ok::<_, io::Error>(()) + }) + }); + + let resolver = + actix_connect::start_resolver(ResolverConfig::default(), ResolverOpts::default()) + .await + .unwrap(); + + let factory = actix_connect::new_connector_factory(resolver); + + let mut conn = factory.new_service(()).await.unwrap(); + let con = conn.call(Connect::with("10", srv.addr())).await.unwrap(); + assert_eq!(con.peer_addr().unwrap(), srv.addr()); +} + +#[cfg(all(feature = "openssl", feature = "uri"))] +#[actix_rt::test] +async fn test_openssl_uri() { + use std::convert::TryFrom; + + let srv = TestServer::with(|| { + fn_service(|io: TcpStream| async { + let mut framed = Framed::new(io, BytesCodec); + framed.send(Bytes::from_static(b"test")).await?; + Ok::<_, io::Error>(()) + }) + }); + + let mut conn = actix_connect::default_connector(); + let addr = http::Uri::try_from(format!("https://localhost:{}", srv.port())).unwrap(); + let con = conn.call(addr.into()).await.unwrap(); + assert_eq!(con.peer_addr().unwrap(), srv.addr()); +} + +#[cfg(all(feature = "rustls", feature = "uri"))] +#[actix_rt::test] +async fn test_rustls_uri() { + use std::convert::TryFrom; + + let srv = TestServer::with(|| { + fn_service(|io: TcpStream| async { + let mut framed = Framed::new(io, BytesCodec); + framed.send(Bytes::from_static(b"test")).await?; + Ok::<_, io::Error>(()) + }) + }); + + let mut conn = actix_connect::default_connector(); + let addr = http::Uri::try_from(format!("https://localhost:{}", srv.port())).unwrap(); + let con = conn.call(addr.into()).await.unwrap(); + assert_eq!(con.peer_addr().unwrap(), srv.addr()); +} diff --git a/string/Cargo.toml b/string/Cargo.toml index 3653fe58..d3c290c8 100644 --- a/string/Cargo.toml +++ b/string/Cargo.toml @@ -15,7 +15,7 @@ name = "bytestring" path = "src/lib.rs" [dependencies] -bytes = "0.5.3" +bytes = "1" serde = { version = "1.0", optional = true } [dev-dependencies] From b5eefb4d42bbe96e6ba07a1a8b12026fd7a4041c Mon Sep 17 00:00:00 2001 From: fakeshadow <24548779@qq.com> Date: Wed, 30 Dec 2020 05:20:24 +0800 Subject: [PATCH 8/8] merge actix-testing into actix-server (#242) --- Cargo.toml | 2 - actix-server/CHANGES.md | 2 +- actix-server/Cargo.toml | 1 - actix-server/src/lib.rs | 2 + .../lib.rs => actix-server/src/test_server.rs | 23 +++--------- actix-service/src/map_config.rs | 2 +- actix-testing/CHANGES.md | 37 ------------------- actix-testing/Cargo.toml | 26 ------------- actix-testing/LICENSE-APACHE | 1 - actix-testing/LICENSE-MIT | 1 - actix-testing/README.md | 9 ----- actix-tls/Cargo.toml | 1 - actix-tls/tests/test_connect.rs | 2 +- 13 files changed, 11 insertions(+), 98 deletions(-) rename actix-testing/src/lib.rs => actix-server/src/test_server.rs (81%) delete mode 100644 actix-testing/CHANGES.md delete mode 100644 actix-testing/Cargo.toml delete mode 120000 actix-testing/LICENSE-APACHE delete mode 120000 actix-testing/LICENSE-MIT delete mode 100644 actix-testing/README.md diff --git a/Cargo.toml b/Cargo.toml index 342f866a..17d1c857 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -5,7 +5,6 @@ members = [ "actix-macros", "actix-service", "actix-server", - "actix-testing", "actix-threadpool", "actix-tls", "actix-tracing", @@ -20,7 +19,6 @@ actix-rt = { path = "actix-rt" } actix-macros = { path = "actix-macros" } actix-server = { path = "actix-server" } actix-service = { path = "actix-service" } -actix-testing = { path = "actix-testing" } actix-threadpool = { path = "actix-threadpool" } actix-tls = { path = "actix-tls" } actix-tracing = { path = "actix-tracing" } diff --git a/actix-server/CHANGES.md b/actix-server/CHANGES.md index 302ea576..f3b7c93c 100644 --- a/actix-server/CHANGES.md +++ b/actix-server/CHANGES.md @@ -1,7 +1,7 @@ # Changes ## Unreleased - 2020-xx-xx - +* Merge `actix-testing` to `actix-server` as `test_server` mod. ## 2.0.0-beta.1 - 2020-12-28 * Added explicit info log message on accept queue pause. [#215] diff --git a/actix-server/Cargo.toml b/actix-server/Cargo.toml index c57b8ac6..0bdd5d23 100644 --- a/actix-server/Cargo.toml +++ b/actix-server/Cargo.toml @@ -36,7 +36,6 @@ slab = "0.4" tokio = { version = "1", features = ["sync"] } [dev-dependencies] -actix-testing = "2.0.0-beta.1" bytes = "1" env_logger = "0.8" futures-util = { version = "0.3.7", default-features = false, features = ["sink"] } diff --git a/actix-server/src/lib.rs b/actix-server/src/lib.rs index 64aca7e4..24129b5a 100644 --- a/actix-server/src/lib.rs +++ b/actix-server/src/lib.rs @@ -11,6 +11,7 @@ mod server; mod service; mod signals; mod socket; +mod test_server; mod waker_queue; mod worker; @@ -18,6 +19,7 @@ pub use self::builder::ServerBuilder; pub use self::config::{ServiceConfig, ServiceRuntime}; pub use self::server::Server; pub use self::service::ServiceFactory; +pub use self::test_server::TestServer; #[doc(hidden)] pub use self::socket::FromStream; diff --git a/actix-testing/src/lib.rs b/actix-server/src/test_server.rs similarity index 81% rename from actix-testing/src/lib.rs rename to actix-server/src/test_server.rs index 57e2c223..627cc675 100644 --- a/actix-testing/src/lib.rs +++ b/actix-server/src/test_server.rs @@ -1,19 +1,9 @@ -//! Various helpers for Actix applications to use during testing. - -#![deny(rust_2018_idioms, nonstandard_style)] -#![allow(clippy::type_complexity, clippy::needless_doctest_main)] -#![doc(html_logo_url = "https://actix.rs/img/logo.png")] -#![doc(html_favicon_url = "https://actix.rs/favicon.ico")] - use std::sync::mpsc; use std::{net, thread}; use actix_rt::{net::TcpStream, System}; -use actix_server::{Server, ServerBuilder, ServiceFactory}; -use socket2::{Domain, Protocol, Socket, Type}; -#[cfg(not(test))] // Work around for rust-lang/rust#62127 -pub use actix_macros::test; +use crate::{Server, ServerBuilder, ServiceFactory}; /// The `TestServer` type. /// @@ -24,7 +14,7 @@ pub use actix_macros::test; /// /// ```rust /// use actix_service::fn_service; -/// use actix_testing::TestServer; +/// use actix_server::TestServer; /// /// #[actix_rt::main] /// async fn main() { @@ -115,11 +105,10 @@ impl TestServer { /// Get first available unused local address pub fn unused_addr() -> net::SocketAddr { let addr: net::SocketAddr = "127.0.0.1:0".parse().unwrap(); - let socket = - Socket::new(Domain::ipv4(), Type::stream(), Some(Protocol::tcp())).unwrap(); - socket.bind(&addr.into()).unwrap(); - socket.set_reuse_address(true).unwrap(); - let tcp = socket.into_tcp_listener(); + let socket = mio::net::TcpSocket::new_v4().unwrap(); + socket.bind(addr).unwrap(); + socket.set_reuseaddr(true).unwrap(); + let tcp = socket.listen(1024).unwrap(); tcp.local_addr().unwrap() } } diff --git a/actix-service/src/map_config.rs b/actix-service/src/map_config.rs index d6d6f6b2..1297f7a0 100644 --- a/actix-service/src/map_config.rs +++ b/actix-service/src/map_config.rs @@ -6,7 +6,7 @@ use super::{IntoServiceFactory, ServiceFactory}; /// /// Note that this function consumes the receiving service factory and returns /// a wrapped version of it. -pub fn map_config(factory: I, f: F) -> MapConfig +pub fn map_config(factory: I, f: F) -> MapConfig where I: IntoServiceFactory, SF: ServiceFactory, diff --git a/actix-testing/CHANGES.md b/actix-testing/CHANGES.md deleted file mode 100644 index f23c7521..00000000 --- a/actix-testing/CHANGES.md +++ /dev/null @@ -1,37 +0,0 @@ -# Changes - -## Unreleased - 2021-xx-xx - - -## 2.0.0-beta.1 - 2020-12-28 -* Update `actix-server` to v2.0.0-beta.1. - - -## 1.0.1 - 2020-05-19 -* Replace deprecated `net2` crate with `socket2` -* Remove unused `futures` dependency - - -## 1.0.0 - 2019-12-11 -* Update actix-server to 1.0.0 - - -## 1.0.0-alpha.3 - 2019-12-07 -* Migrate to tokio 0.2 - - -## 1.0.0-alpha.2 - 2019-12-02 -* Re-export `test` attribute macros - - - -## 0.3.0-alpha.1 - 2019-11-22 -* Migrate to std::future - - -## 0.2.0 - 2019-10-14 -* Upgrade actix-server and actix-server-config deps - - -## 0.1.0 - 2019-09-25 -* Initial impl diff --git a/actix-testing/Cargo.toml b/actix-testing/Cargo.toml deleted file mode 100644 index 3f29679e..00000000 --- a/actix-testing/Cargo.toml +++ /dev/null @@ -1,26 +0,0 @@ -[package] -name = "actix-testing" -version = "2.0.0-beta.1" -authors = ["Nikolay Kim "] -description = "Various helpers for Actix applications to use during testing" -keywords = ["network", "framework", "async", "futures"] -homepage = "https://actix.rs" -repository = "https://github.com/actix/actix-net.git" -documentation = "https://docs.rs/actix-testing/" -categories = ["network-programming", "asynchronous"] -license = "MIT OR Apache-2.0" -edition = "2018" -readme = "README.md" - -[lib] -name = "actix_testing" -path = "src/lib.rs" - -[dependencies] -actix-rt = "2.0.0-beta.1" -actix-macros = "0.1.0" -actix-server = "2.0.0-beta.1" -actix-service = "2.0.0-beta.1" - -log = "0.4" -socket2 = "0.3" diff --git a/actix-testing/LICENSE-APACHE b/actix-testing/LICENSE-APACHE deleted file mode 120000 index 965b606f..00000000 --- a/actix-testing/LICENSE-APACHE +++ /dev/null @@ -1 +0,0 @@ -../LICENSE-APACHE \ No newline at end of file diff --git a/actix-testing/LICENSE-MIT b/actix-testing/LICENSE-MIT deleted file mode 120000 index 76219eb7..00000000 --- a/actix-testing/LICENSE-MIT +++ /dev/null @@ -1 +0,0 @@ -../LICENSE-MIT \ No newline at end of file diff --git a/actix-testing/README.md b/actix-testing/README.md deleted file mode 100644 index ae54b46d..00000000 --- a/actix-testing/README.md +++ /dev/null @@ -1,9 +0,0 @@ -# Actix test utilities [![crates.io](https://meritbadge.herokuapp.com/actix-testing)](https://crates.io/crates/actix-testint) [![Join the chat at https://gitter.im/actix/actix](https://badges.gitter.im/actix/actix.svg)](https://gitter.im/actix/actix?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge) - -## Documentation & community resources - -* [User Guide](https://actix.rs/docs/) -* [API Documentation](https://docs.rs/actix-testing/) -* [Chat on gitter](https://gitter.im/actix/actix) -* Cargo package: [actix-http-test](https://crates.io/crates/actix-testing) -* Minimum supported Rust version: 1.46 or later diff --git a/actix-tls/Cargo.toml b/actix-tls/Cargo.toml index 1c24e0eb..2905fd5a 100644 --- a/actix-tls/Cargo.toml +++ b/actix-tls/Cargo.toml @@ -77,7 +77,6 @@ tokio-native-tls = { version = "0.3", optional = true } [dev-dependencies] actix-server = "2.0.0-beta.1" -actix-testing = "2.0.0-beta.1" bytes = "1" env_logger = "0.8" futures-util = { version = "0.3.7", default-features = false, features = ["sink"] } diff --git a/actix-tls/tests/test_connect.rs b/actix-tls/tests/test_connect.rs index fd083893..aa773c7f 100644 --- a/actix-tls/tests/test_connect.rs +++ b/actix-tls/tests/test_connect.rs @@ -2,8 +2,8 @@ use std::io; use actix_codec::{BytesCodec, Framed}; use actix_rt::net::TcpStream; +use actix_server::TestServer; use actix_service::{fn_service, Service, ServiceFactory}; -use actix_testing::TestServer; use bytes::Bytes; use futures_util::sink::SinkExt;