Rework swarm and allow interrupting a dial (#366)

* Rework swarm and allow interrupting a dial
* Improve the UniqueConnec situation
* Remove UniqueConnec::get
* Rename `get_or_dial()` to `dial()` and add `dial_if_empty()`
* Clean the UniqueConnec is the future is dropped
* Rename `set_until` to `tie_or_stop` and add `tie_or_passthrough`
* Add some tests, docs
* Fix memory leak with tasks registration
* Interrupt dialing when a UniqueConnec is dropped or cleared
This commit is contained in:
Pierre Krieger
2018-08-08 12:00:38 +02:00
committed by Benjamin Kampmann
parent e2618dc1b3
commit ea881e3dfa
7 changed files with 907 additions and 254 deletions

View File

@ -18,11 +18,14 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use futures::stream::{FuturesUnordered, StreamFuture};
use futures::sync::{mpsc, oneshot};
use futures::{future, Async, Future, IntoFuture, Poll, Stream};
use futures::stream::StreamFuture;
use futures::sync::oneshot;
use futures::{Async, Future, IntoFuture, Poll, Stream};
use futures::task;
use parking_lot::Mutex;
use std::fmt;
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::sync::Arc;
use {Multiaddr, MuxedTransport, Transport};
/// Creates a swarm.
@ -32,38 +35,33 @@ use {Multiaddr, MuxedTransport, Transport};
///
/// Produces a `SwarmController` and an implementation of `Future`. The controller can be used to
/// control, and the `Future` must be driven to completion in order for things to work.
///
pub fn swarm<T, H, F>(
transport: T,
handler: H,
) -> (SwarmController<T>, SwarmFuture<T, H, F::Future>)
) -> (SwarmController<T>, SwarmFuture<T, H>)
where
T: MuxedTransport + Clone + 'static, // TODO: 'static :-/
H: FnMut(T::Output, Box<Future<Item = Multiaddr, Error = IoError>>) -> F,
F: IntoFuture<Item = (), Error = IoError>,
{
let (new_dialers_tx, new_dialers_rx) = mpsc::unbounded();
let (new_listeners_tx, new_listeners_rx) = mpsc::unbounded();
let (new_toprocess_tx, new_toprocess_rx) = mpsc::unbounded();
let shared = Arc::new(Mutex::new(Shared {
next_incoming: transport.clone().next_incoming(),
listeners: Vec::new(),
listeners_upgrade: Vec::new(),
dialers: Vec::new(),
to_process: Vec::new(),
task_to_notify: None,
}));
let future = SwarmFuture {
transport: transport.clone(),
shared: shared.clone(),
handler: handler,
new_listeners: new_listeners_rx,
next_incoming: transport.clone().next_incoming(),
listeners: FuturesUnordered::new(),
listeners_upgrade: FuturesUnordered::new(),
dialers: FuturesUnordered::new(),
new_dialers: new_dialers_rx,
to_process: FuturesUnordered::new(),
new_toprocess: new_toprocess_rx,
};
let controller = SwarmController {
transport: transport,
new_listeners: new_listeners_tx,
new_dialers: new_dialers_tx,
new_toprocess: new_toprocess_tx,
transport,
shared,
};
(controller, future)
@ -74,10 +72,11 @@ pub struct SwarmController<T>
where
T: MuxedTransport + 'static, // TODO: 'static :-/
{
/// Shared between the swarm infrastructure.
shared: Arc<Mutex<Shared<T>>>,
/// Transport used to dial or listen.
transport: T,
new_listeners: mpsc::UnboundedSender<T::Listener>,
new_dialers: mpsc::UnboundedSender<Box<Future<Item = (T::Output, oneshot::Sender<Result<(), IoError>>, Box<Future<Item = Multiaddr, Error = IoError>>), Error = ()>>>,
new_toprocess: mpsc::UnboundedSender<Box<Future<Item = (), Error = IoError>>>,
}
impl<T> fmt::Debug for SwarmController<T>
@ -98,9 +97,7 @@ where
fn clone(&self) -> SwarmController<T> {
SwarmController {
transport: self.transport.clone(),
new_listeners: self.new_listeners.clone(),
new_dialers: self.new_dialers.clone(),
new_toprocess: self.new_toprocess.clone(),
shared: self.shared.clone(),
}
}
}
@ -116,33 +113,62 @@ where
/// Returns a future that is signalled once the closure in the `swarm` has returned its future.
/// Therefore if the closure in the swarm has some side effect (eg. write something in a
/// variable), this side effect will be observable when this future succeeds.
#[inline]
pub fn dial<Du>(&self, multiaddr: Multiaddr, transport: Du)
-> Result<impl Future<Item = (), Error = IoError>, Multiaddr>
where
Du: Transport + 'static, // TODO: 'static :-/
Du::Output: Into<T::Output>,
{
self.dial_then(multiaddr, transport, |v| v)
}
/// Internal version of `dial` that allows adding a closure that is called after either the
/// dialing fails or the handler has been called with the resulting future.
///
/// The returned future is filled with the output of `then`.
pub(crate) fn dial_then<Du, F>(&self, multiaddr: Multiaddr, transport: Du, then: F)
-> Result<impl Future<Item = (), Error = IoError>, Multiaddr>
where
Du: Transport + 'static, // TODO: 'static :-/
Du::Output: Into<T::Output>,
F: FnOnce(Result<(), IoError>) -> Result<(), IoError> + 'static,
{
trace!("Swarm dialing {}", multiaddr);
match transport.dial(multiaddr.clone()) {
Ok(dial) => {
let (tx, rx) = oneshot::channel();
let mut then = Some(move |val| {
let _ = tx.send(then(val));
});
// Unfortunately the `Box<FnOnce(_)>` type is still unusable in Rust right now,
// so we use a `Box<FnMut(_)>` instead and panic if it is called multiple times.
let mut then = Box::new(move |val: Result<(), IoError>| {
let then = then.take().expect("The Boxed FnMut should only be called once");
then(val);
}) as Box<FnMut(_)>;
let dial = dial.then(|result| {
match result {
Ok((output, client_addr)) => {
let client_addr = Box::new(client_addr) as Box<Future<Item = _, Error = _>>;
Ok((output.into(), tx, client_addr))
Ok((output.into(), then, client_addr))
}
Err(err) => {
debug!("Error in dialer upgrade: {:?}", err);
let _ = tx.send(Err(err));
then(Err(err));
Err(())
}
}
});
// Ignoring errors if the receiver has been closed, because in that situation
// nothing is going to be processed anyway.
let _ = self.new_dialers.unbounded_send(Box::new(dial) as Box<_>);
let mut shared = self.shared.lock();
shared.dialers.push((multiaddr, Box::new(dial) as Box<_>));
if let Some(task) = shared.task_to_notify.take() {
task.notify();
}
Ok(rx.then(|result| {
match result {
Ok(Ok(())) => Ok(()),
@ -156,15 +182,38 @@ where
}
}
/// Interrupts all dialing attempts to a specific multiaddress.
///
/// Has no effect if the dialing attempt has already succeeded, in which case it will be
/// dispatched to the handler.
pub fn interrupt_dial(&self, multiaddr: &Multiaddr) {
let mut shared = self.shared.lock();
shared.dialers.retain(|dialer| {
&dialer.0 != multiaddr
});
}
/// Adds a multiaddr to listen on. All the incoming connections will use the `upgrade` that
/// was passed to `swarm`.
// TODO: add a way to cancel a listener
pub fn listen_on(&self, multiaddr: Multiaddr) -> Result<Multiaddr, Multiaddr> {
match self.transport.clone().listen_on(multiaddr) {
Ok((listener, new_addr)) => {
trace!("Swarm listening on {}", new_addr);
// Ignoring errors if the receiver has been closed, because in that situation
// nothing is going to be processed anyway.
let _ = self.new_listeners.unbounded_send(listener);
let mut shared = self.shared.lock();
let listener = Box::new(
listener.map(|f| {
let f = f.map(|(out, maf)| {
(out, Box::new(maf) as Box<Future<Item = Multiaddr, Error = IoError>>)
});
Box::new(f) as Box<Future<Item = _, Error = _>>
}),
) as Box<Stream<Item = _, Error = _>>;
shared.listeners.push(listener.into_future());
if let Some(task) = shared.task_to_notify.take() {
task.notify();
}
Ok(new_addr)
}
Err((_, multiaddr)) => Err(multiaddr),
@ -173,15 +222,152 @@ where
}
/// Future that must be driven to completion in order for the swarm to work.
pub struct SwarmFuture<T, H, F>
pub struct SwarmFuture<T, H>
where
T: MuxedTransport + 'static, // TODO: 'static :-/
{
/// Shared between the swarm infrastructure.
shared: Arc<Mutex<Shared<T>>>,
/// The transport used to dial.
transport: T,
/// Swarm handler.
handler: H,
new_listeners: mpsc::UnboundedReceiver<T::Listener>,
}
impl<T, H, If, F> Future for SwarmFuture<T, H>
where
T: MuxedTransport + Clone + 'static, // TODO: 'static :-/,
H: FnMut(T::Output, Box<Future<Item = Multiaddr, Error = IoError>>) -> If,
If: IntoFuture<Future = F, Item = (), Error = IoError>,
F: Future<Item = (), Error = IoError> + 'static, // TODO: 'static :-/
{
type Item = ();
type Error = IoError;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let mut shared = self.shared.lock();
let handler = &mut self.handler;
loop {
match shared.next_incoming.poll() {
Ok(Async::Ready(connec)) => {
debug!("Swarm received new multiplexed incoming connection");
shared.next_incoming = self.transport.clone().next_incoming();
let connec = connec.map(|(out, maf)| {
(out, Box::new(maf) as Box<Future<Item = Multiaddr, Error = IoError>>)
});
shared.listeners_upgrade.push(Box::new(connec) as Box<_>);
}
Ok(Async::NotReady) => break,
Err(err) => {
// TODO: should that stop everything?
debug!("Error in multiplexed incoming connection: {:?}", err);
shared.next_incoming = self.transport.clone().next_incoming();
break;
}
}
}
// We remove each element from `shared.listeners` one by one and add them back only
// if relevant.
for n in (0 .. shared.listeners.len()).rev() {
let mut listener = shared.listeners.swap_remove(n);
loop {
match listener.poll() {
Ok(Async::Ready((Some(upgrade), remaining))) => {
trace!("Swarm received new connection on listener socket");
shared.listeners_upgrade.push(upgrade);
listener = remaining.into_future();
}
Ok(Async::Ready((None, _))) => {
debug!("Listener closed gracefully");
break;
},
Err((err, _)) => {
debug!("Error in listener: {:?}", err);
break;
}
Ok(Async::NotReady) => {
shared.listeners.push(listener);
break;
}
}
}
}
// We remove each element from `shared.listeners_upgrade` one by one and add them back
// only if relevant.
for n in (0 .. shared.listeners_upgrade.len()).rev() {
let mut listener_upgrade = shared.listeners_upgrade.swap_remove(n);
match listener_upgrade.poll() {
Ok(Async::Ready((output, client_addr))) => {
debug!("Successfully upgraded incoming connection");
// TODO: unlock mutex before calling handler, in order to avoid deadlocks if
// the user does something stupid
shared.to_process.push(Box::new(handler(output, client_addr).into_future()));
}
Err(err) => {
debug!("Error in listener upgrade: {:?}", err);
}
Ok(Async::NotReady) => {
shared.listeners_upgrade.push(listener_upgrade);
},
}
}
// We remove each element from `shared.dialers` one by one and add them back only
// if relevant.
for n in (0 .. shared.dialers.len()).rev() {
let (client_addr, mut dialer) = shared.dialers.swap_remove(n);
match dialer.poll() {
Ok(Async::Ready((output, mut notifier, addr))) => {
trace!("Successfully upgraded dialed connection");
// TODO: unlock mutex before calling handler, in order to avoid deadlocks if
// the user does something stupid
shared.to_process.push(Box::new(handler(output, addr).into_future()));
notifier(Ok(()));
}
Err(()) => {},
Ok(Async::NotReady) => {
shared.dialers.push((client_addr, dialer));
},
}
}
// We remove each element from `shared.to_process` one by one and add them back only
// if relevant.
for n in (0 .. shared.to_process.len()).rev() {
let mut to_process = shared.to_process.swap_remove(n);
match to_process.poll() {
Ok(Async::Ready(())) => {
trace!("Future returned by swarm handler driven to completion");
}
Err(err) => {
debug!("Error in processing: {:?}", err);
}
Ok(Async::NotReady) => {
shared.to_process.push(to_process);
}
}
}
// TODO: we never return `Ok(Ready)` because there's no way to know whether
// `next_incoming()` can produce anything more in the future ; also we would need to
// know when the controller has been dropped
shared.task_to_notify = Some(task::current());
Ok(Async::NotReady)
}
}
// TODO: stronger typing
struct Shared<T> where T: MuxedTransport + 'static {
/// Next incoming substream on the transport.
next_incoming: T::Incoming,
listeners: FuturesUnordered<
/// All the active listeners.
listeners: Vec<
StreamFuture<
Box<
Stream<
@ -191,162 +377,33 @@ where
>,
>,
>,
/// Futures that upgrade an incoming listening connection to a full connection.
listeners_upgrade:
FuturesUnordered<Box<Future<Item = (T::Output, Box<Future<Item = Multiaddr, Error = IoError>>), Error = IoError>>>,
dialers: FuturesUnordered<Box<Future<Item = (T::Output, oneshot::Sender<Result<(), IoError>>, Box<Future<Item = Multiaddr, Error = IoError>>), Error = ()>>>,
new_dialers:
mpsc::UnboundedReceiver<Box<Future<Item = (T::Output, oneshot::Sender<Result<(), IoError>>, Box<Future<Item = Multiaddr, Error = IoError>>), Error = ()>>>,
to_process: FuturesUnordered<future::Either<F, Box<Future<Item = (), Error = IoError>>>>,
new_toprocess: mpsc::UnboundedReceiver<Box<Future<Item = (), Error = IoError>>>,
}
Vec<Box<Future<Item = (T::Output, Box<Future<Item = Multiaddr, Error = IoError>>), Error = IoError>>>,
impl<T, H, If, F> Future for SwarmFuture<T, H, F>
where
T: MuxedTransport + Clone + 'static, // TODO: 'static :-/,
H: FnMut(T::Output, Box<Future<Item = Multiaddr, Error = IoError>>) -> If,
If: IntoFuture<Future = F, Item = (), Error = IoError>,
F: Future<Item = (), Error = IoError>,
{
type Item = ();
type Error = IoError;
/// Futures that dial a remote address.
///
/// Contains the address we dial, so that we can cancel it if necessary.
dialers: Vec<(Multiaddr, Box<Future<Item = (T::Output, Box<FnMut(Result<(), IoError>)>, Box<Future<Item = Multiaddr, Error = IoError>>), Error = ()>>)>,
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let handler = &mut self.handler;
/// List of futures produced by the swarm closure. Must be processed to the end.
to_process: Vec<Box<Future<Item = (), Error = IoError>>>,
loop {
match self.next_incoming.poll() {
Ok(Async::Ready(connec)) => {
debug!("Swarm received new multiplexed incoming connection");
self.next_incoming = self.transport.clone().next_incoming();
let connec = connec.map(|(out, maf)| {
(out, Box::new(maf) as Box<Future<Item = Multiaddr, Error = IoError>>)
});
self.listeners_upgrade.push(Box::new(connec) as Box<_>);
}
Ok(Async::NotReady) => break,
Err(err) => {
debug!("Error in multiplexed incoming connection: {:?}", err);
self.next_incoming = self.transport.clone().next_incoming();
break;
}
}
}
loop {
match self.new_listeners.poll() {
Ok(Async::Ready(Some(new_listener))) => {
let new_listener = Box::new(
new_listener.map(|f| {
let f = f.map(|(out, maf)| {
(out, Box::new(maf) as Box<Future<Item = Multiaddr, Error = IoError>>)
});
Box::new(f) as Box<Future<Item = _, Error = _>>
}),
) as Box<Stream<Item = _, Error = _>>;
self.listeners.push(new_listener.into_future());
}
Ok(Async::Ready(None)) | Err(_) => {
// New listener sender has been closed.
break;
}
Ok(Async::NotReady) => break,
}
}
loop {
match self.new_dialers.poll() {
Ok(Async::Ready(Some(new_dialer))) => {
self.dialers.push(new_dialer);
}
Ok(Async::Ready(None)) | Err(_) => {
// New dialers sender has been closed.
break
}
Ok(Async::NotReady) => break,
}
}
loop {
match self.new_toprocess.poll() {
Ok(Async::Ready(Some(new_toprocess))) => {
self.to_process.push(future::Either::B(new_toprocess));
}
Ok(Async::Ready(None)) | Err(_) => {
// New to-process sender has been closed.
break
}
Ok(Async::NotReady) => break,
}
}
loop {
match self.listeners.poll() {
Ok(Async::Ready(Some((Some(upgrade), remaining)))) => {
trace!("Swarm received new connection on listener socket");
self.listeners_upgrade.push(upgrade);
self.listeners.push(remaining.into_future());
}
Err((err, _)) => {
debug!("Error in listener: {:?}", err);
break
}
_ => break
}
}
loop {
match self.listeners_upgrade.poll() {
Ok(Async::Ready(Some((output, client_addr)))) => {
debug!("Successfully upgraded incoming connection");
self.to_process.push(future::Either::A(
handler(output, client_addr).into_future(),
));
}
Err(err) => {
debug!("Error in listener upgrade: {:?}", err);
break;
}
_ => break
}
}
loop {
match self.dialers.poll() {
Ok(Async::Ready(Some((output, notifier, addr)))) => {
trace!("Successfully upgraded dialed connection");
self.to_process
.push(future::Either::A(handler(output, addr).into_future()));
let _ = notifier.send(Ok(()));
}
Err(()) => break,
_ => break,
}
}
loop {
match self.to_process.poll() {
Ok(Async::Ready(Some(()))) => {
trace!("Future returned by swarm handler driven to completion");
}
Err(err) => {
debug!("Error in processing: {:?}", err);
}
_ => break,
}
}
// TODO: we never return `Ok(Ready)` because there's no way to know whether
// `next_incoming()` can produce anything more in the future
Ok(Async::NotReady)
}
/// The task to notify whenever we add a new element in one of the lists.
/// Necessary so that the task wakes up and the element gets polled.
task_to_notify: Option<task::Task>,
}
#[cfg(test)]
mod tests {
use futures::future;
use transport::DeniedTransport;
use futures::{Future, future};
use rand;
use transport::{self, DeniedTransport, Transport};
use std::io::Error as IoError;
use std::sync::{atomic, Arc};
use swarm;
use tokio::runtime::current_thread;
#[test]
fn transport_error_propagation_listen() {
@ -360,4 +417,74 @@ mod tests {
let addr = "/ip4/127.0.0.1/tcp/10000".parse().unwrap();
assert!(swarm_ctrl.dial(addr, DeniedTransport).is_err());
}
#[test]
fn basic_dial() {
let (tx, rx) = transport::connector();
let reached_tx = Arc::new(atomic::AtomicBool::new(false));
let reached_tx2 = reached_tx.clone();
let reached_rx = Arc::new(atomic::AtomicBool::new(false));
let reached_rx2 = reached_rx.clone();
let (swarm_ctrl1, swarm_future1) = swarm(rx.with_dummy_muxing(), |_, _| {
reached_rx2.store(true, atomic::Ordering::SeqCst);
future::empty()
});
swarm_ctrl1.listen_on("/memory".parse().unwrap()).unwrap();
let (swarm_ctrl2, swarm_future2) = swarm(tx.clone().with_dummy_muxing(), |_, _| {
reached_tx2.store(true, atomic::Ordering::SeqCst);
future::empty()
});
let dial_success = swarm_ctrl2.dial("/memory".parse().unwrap(), tx).unwrap();
let future = swarm_future2
.select(swarm_future1).map(|_| ()).map_err(|(err, _)| err)
.select(dial_success).map(|_| ()).map_err(|(err, _)| err);
current_thread::Runtime::new().unwrap().block_on(future).unwrap();
assert!(reached_tx.load(atomic::Ordering::SeqCst));
assert!(reached_rx.load(atomic::Ordering::SeqCst));
}
#[test]
fn dial_multiple_times() {
let (tx, rx) = transport::connector();
let reached = Arc::new(atomic::AtomicUsize::new(0));
let reached2 = reached.clone();
let (swarm_ctrl, swarm_future) = swarm(rx.with_dummy_muxing(), |_, _| {
reached2.fetch_add(1, atomic::Ordering::SeqCst);
future::empty()
});
swarm_ctrl.listen_on("/memory".parse().unwrap()).unwrap();
let num_dials = 20000 + rand::random::<usize>() % 20000;
let mut dials = Vec::new();
for _ in 0 .. num_dials {
let f = swarm_ctrl.dial("/memory".parse().unwrap(), tx.clone()).unwrap();
dials.push(f);
}
let future = future::join_all(dials)
.map(|_| ())
.select(swarm_future)
.map_err(|(err, _)| err);
current_thread::Runtime::new().unwrap().block_on(future).unwrap();
assert_eq!(reached.load(atomic::Ordering::SeqCst), num_dials);
}
#[test]
fn future_isnt_dropped() {
// Tests that the future in the closure isn't being dropped.
let (tx, rx) = transport::connector();
let (swarm_ctrl, swarm_future) = swarm(rx.with_dummy_muxing(), |_, _| {
future::empty()
.then(|_: Result<(), ()>| -> Result<(), IoError> { panic!() }) // <-- the test
});
swarm_ctrl.listen_on("/memory".parse().unwrap()).unwrap();
let dial_success = swarm_ctrl.dial("/memory".parse().unwrap(), tx).unwrap();
let future = dial_success.select(swarm_future)
.map_err(|(err, _)| err);
current_thread::Runtime::new().unwrap().block_on(future).unwrap();
}
}