mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-23 23:01:33 +00:00
Switch to stable futures (#1196)
* Switch to stable futures * Remove from_fn * Fix secio * Fix core --lib tests
This commit is contained in:
@ -20,8 +20,6 @@
|
||||
|
||||
//! Implementation of the libp2p `Transport` trait for TCP/IP.
|
||||
//!
|
||||
//! Uses [the *tokio* library](https://tokio.rs).
|
||||
//!
|
||||
//! # Usage
|
||||
//!
|
||||
//! Example:
|
||||
@ -38,11 +36,13 @@
|
||||
//! The `TcpConfig` structs implements the `Transport` trait of the `swarm` library. See the
|
||||
//! documentation of `swarm` and of libp2p in general to learn how to use the `Transport` trait.
|
||||
|
||||
use async_std::net::TcpStream;
|
||||
use futures::{
|
||||
future::{self, Either, FutureResult},
|
||||
future::{self, Ready},
|
||||
io::Initializer,
|
||||
prelude::*,
|
||||
stream::{self, Chain, IterOk, Once}
|
||||
};
|
||||
use futures_timer::Delay;
|
||||
use get_if_addrs::{IfAddr, get_if_addrs};
|
||||
use ipnet::{IpNet, Ipv4Net, Ipv6Net};
|
||||
use libp2p_core::{
|
||||
@ -53,15 +53,13 @@ use libp2p_core::{
|
||||
use log::{debug, trace};
|
||||
use std::{
|
||||
collections::VecDeque,
|
||||
io::{self, Read, Write},
|
||||
io,
|
||||
iter::{self, FromIterator},
|
||||
net::{IpAddr, SocketAddr},
|
||||
time::{Duration, Instant},
|
||||
vec::IntoIter
|
||||
pin::Pin,
|
||||
task::{Context, Poll},
|
||||
time::Duration
|
||||
};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use tokio_timer::Delay;
|
||||
use tokio_tcp::{ConnectFuture, Incoming, TcpStream};
|
||||
|
||||
/// Represents the configuration for a TCP/IP transport capability for libp2p.
|
||||
///
|
||||
@ -130,9 +128,9 @@ impl TcpConfig {
|
||||
impl Transport for TcpConfig {
|
||||
type Output = TcpTransStream;
|
||||
type Error = io::Error;
|
||||
type Listener = TcpListener;
|
||||
type ListenerUpgrade = FutureResult<Self::Output, Self::Error>;
|
||||
type Dial = TcpDialFut;
|
||||
type Listener = Pin<Box<dyn Stream<Item = Result<ListenerEvent<Self::ListenerUpgrade>, io::Error>> + Send>>;
|
||||
type ListenerUpgrade = Ready<Result<Self::Output, Self::Error>>;
|
||||
type Dial = Pin<Box<dyn Future<Output = Result<TcpTransStream, io::Error>> + Send>>;
|
||||
|
||||
fn listen_on(self, addr: Multiaddr) -> Result<Self::Listener, TransportError<Self::Error>> {
|
||||
let socket_addr =
|
||||
@ -142,54 +140,59 @@ impl Transport for TcpConfig {
|
||||
return Err(TransportError::MultiaddrNotSupported(addr))
|
||||
};
|
||||
|
||||
let listener = tokio_tcp::TcpListener::bind(&socket_addr).map_err(TransportError::Other)?;
|
||||
let local_addr = listener.local_addr().map_err(TransportError::Other)?;
|
||||
let port = local_addr.port();
|
||||
async fn do_listen(cfg: TcpConfig, socket_addr: SocketAddr)
|
||||
-> Result<impl Stream<Item = Result<ListenerEvent<Ready<Result<TcpTransStream, io::Error>>>, io::Error>>, io::Error>
|
||||
{
|
||||
let listener = async_std::net::TcpListener::bind(&socket_addr).await?;
|
||||
let local_addr = listener.local_addr()?;
|
||||
let port = local_addr.port();
|
||||
|
||||
// Determine all our listen addresses which is either a single local IP address
|
||||
// or (if a wildcard IP address was used) the addresses of all our interfaces,
|
||||
// as reported by `get_if_addrs`.
|
||||
let addrs =
|
||||
if socket_addr.ip().is_unspecified() {
|
||||
let addrs = host_addresses(port).map_err(TransportError::Other)?;
|
||||
debug!("Listening on {:?}", addrs.iter().map(|(_, _, ma)| ma).collect::<Vec<_>>());
|
||||
Addresses::Many(addrs)
|
||||
} else {
|
||||
let ma = ip_to_multiaddr(local_addr.ip(), port);
|
||||
debug!("Listening on {:?}", ma);
|
||||
Addresses::One(ma)
|
||||
// Determine all our listen addresses which is either a single local IP address
|
||||
// or (if a wildcard IP address was used) the addresses of all our interfaces,
|
||||
// as reported by `get_if_addrs`.
|
||||
let addrs =
|
||||
if socket_addr.ip().is_unspecified() {
|
||||
let addrs = host_addresses(port)?;
|
||||
debug!("Listening on {:?}", addrs.iter().map(|(_, _, ma)| ma).collect::<Vec<_>>());
|
||||
Addresses::Many(addrs)
|
||||
} else {
|
||||
let ma = ip_to_multiaddr(local_addr.ip(), port);
|
||||
debug!("Listening on {:?}", ma);
|
||||
Addresses::One(ma)
|
||||
};
|
||||
|
||||
// Generate `NewAddress` events for each new `Multiaddr`.
|
||||
let pending = match addrs {
|
||||
Addresses::One(ref ma) => {
|
||||
let event = ListenerEvent::NewAddress(ma.clone());
|
||||
let mut list = VecDeque::new();
|
||||
list.push_back(Ok(event));
|
||||
list
|
||||
}
|
||||
Addresses::Many(ref aa) => {
|
||||
aa.iter()
|
||||
.map(|(_, _, ma)| ma)
|
||||
.cloned()
|
||||
.map(ListenerEvent::NewAddress)
|
||||
.map(Result::Ok)
|
||||
.collect::<VecDeque<_>>()
|
||||
}
|
||||
};
|
||||
|
||||
// Generate `NewAddress` events for each new `Multiaddr`.
|
||||
let events = match addrs {
|
||||
Addresses::One(ref ma) => {
|
||||
let event = ListenerEvent::NewAddress(ma.clone());
|
||||
Either::A(stream::once(Ok(event)))
|
||||
}
|
||||
Addresses::Many(ref aa) => {
|
||||
let events = aa.iter()
|
||||
.map(|(_, _, ma)| ma)
|
||||
.cloned()
|
||||
.map(ListenerEvent::NewAddress)
|
||||
.collect::<Vec<_>>();
|
||||
Either::B(stream::iter_ok(events))
|
||||
}
|
||||
};
|
||||
let listen_stream = TcpListenStream {
|
||||
stream: listener,
|
||||
pause: None,
|
||||
pause_duration: cfg.sleep_on_error,
|
||||
port,
|
||||
addrs,
|
||||
pending,
|
||||
config: cfg
|
||||
};
|
||||
|
||||
let stream = TcpListenStream {
|
||||
inner: Listener::new(listener.incoming(), self.sleep_on_error),
|
||||
port,
|
||||
addrs,
|
||||
pending: VecDeque::new(),
|
||||
config: self
|
||||
};
|
||||
Ok(stream::unfold(listen_stream, |s| s.next().map(Some)))
|
||||
}
|
||||
|
||||
Ok(TcpListener {
|
||||
inner: match events {
|
||||
Either::A(e) => Either::A(e.chain(stream)),
|
||||
Either::B(e) => Either::B(e.chain(stream))
|
||||
}
|
||||
})
|
||||
Ok(Box::pin(do_listen(self, socket_addr).try_flatten_stream()))
|
||||
}
|
||||
|
||||
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, TransportError<Self::Error>> {
|
||||
@ -206,12 +209,13 @@ impl Transport for TcpConfig {
|
||||
|
||||
debug!("Dialing {}", addr);
|
||||
|
||||
let future = TcpDialFut {
|
||||
inner: TcpStream::connect(&socket_addr),
|
||||
config: self
|
||||
};
|
||||
async fn do_dial(cfg: TcpConfig, socket_addr: SocketAddr) -> Result<TcpTransStream, io::Error> {
|
||||
let stream = TcpStream::connect(&socket_addr).await?;
|
||||
apply_config(&cfg, &stream)?;
|
||||
Ok(TcpTransStream { inner: stream })
|
||||
}
|
||||
|
||||
Ok(future)
|
||||
Ok(Box::pin(do_dial(self, socket_addr)))
|
||||
}
|
||||
}
|
||||
|
||||
@ -270,11 +274,11 @@ fn host_addresses(port: u16) -> io::Result<Vec<(IpAddr, IpNet, Multiaddr)>> {
|
||||
/// Applies the socket configuration parameters to a socket.
|
||||
fn apply_config(config: &TcpConfig, socket: &TcpStream) -> Result<(), io::Error> {
|
||||
if let Some(recv_buffer_size) = config.recv_buffer_size {
|
||||
socket.set_recv_buffer_size(recv_buffer_size)?;
|
||||
// TODO: socket.set_recv_buffer_size(recv_buffer_size)?;
|
||||
}
|
||||
|
||||
if let Some(send_buffer_size) = config.send_buffer_size {
|
||||
socket.set_send_buffer_size(send_buffer_size)?;
|
||||
// TODO: socket.set_send_buffer_size(send_buffer_size)?;
|
||||
}
|
||||
|
||||
if let Some(ttl) = config.ttl {
|
||||
@ -282,7 +286,7 @@ fn apply_config(config: &TcpConfig, socket: &TcpStream) -> Result<(), io::Error>
|
||||
}
|
||||
|
||||
if let Some(keepalive) = config.keepalive {
|
||||
socket.set_keepalive(keepalive)?;
|
||||
// TODO: socket.set_keepalive(keepalive)?;
|
||||
}
|
||||
|
||||
if let Some(nodelay) = config.nodelay {
|
||||
@ -292,55 +296,6 @@ fn apply_config(config: &TcpConfig, socket: &TcpStream) -> Result<(), io::Error>
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Future that dials a TCP/IP address.
|
||||
#[derive(Debug)]
|
||||
#[must_use = "futures do nothing unless polled"]
|
||||
pub struct TcpDialFut {
|
||||
inner: ConnectFuture,
|
||||
/// Original configuration.
|
||||
config: TcpConfig,
|
||||
}
|
||||
|
||||
impl Future for TcpDialFut {
|
||||
type Item = TcpTransStream;
|
||||
type Error = io::Error;
|
||||
|
||||
fn poll(&mut self) -> Poll<TcpTransStream, io::Error> {
|
||||
match self.inner.poll() {
|
||||
Ok(Async::Ready(stream)) => {
|
||||
apply_config(&self.config, &stream)?;
|
||||
Ok(Async::Ready(TcpTransStream { inner: stream }))
|
||||
}
|
||||
Ok(Async::NotReady) => Ok(Async::NotReady),
|
||||
Err(err) => {
|
||||
debug!("Error while dialing => {:?}", err);
|
||||
Err(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Stream of `ListenerEvent`s.
|
||||
#[derive(Debug)]
|
||||
pub struct TcpListener {
|
||||
inner: Either<
|
||||
Chain<Once<ListenerEvent<FutureResult<TcpTransStream, io::Error>>, io::Error>, TcpListenStream>,
|
||||
Chain<IterOk<IntoIter<ListenerEvent<FutureResult<TcpTransStream, io::Error>>>, io::Error>, TcpListenStream>
|
||||
>
|
||||
}
|
||||
|
||||
impl Stream for TcpListener {
|
||||
type Item = ListenerEvent<FutureResult<TcpTransStream, io::Error>>;
|
||||
type Error = io::Error;
|
||||
|
||||
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
|
||||
match self.inner {
|
||||
Either::A(ref mut it) => it.poll(),
|
||||
Either::B(ref mut it) => it.poll()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Listen address information.
|
||||
#[derive(Debug)]
|
||||
enum Addresses {
|
||||
@ -350,61 +305,16 @@ enum Addresses {
|
||||
Many(Vec<(IpAddr, IpNet, Multiaddr)>)
|
||||
}
|
||||
|
||||
type Buffer = VecDeque<ListenerEvent<FutureResult<TcpTransStream, io::Error>>>;
|
||||
type Buffer = VecDeque<Result<ListenerEvent<Ready<Result<TcpTransStream, io::Error>>>, io::Error>>;
|
||||
|
||||
/// Incoming connection stream which pauses after errors.
|
||||
#[derive(Debug)]
|
||||
struct Listener<S> {
|
||||
/// Stream that listens on an TCP/IP address.
|
||||
pub struct TcpListenStream {
|
||||
/// The incoming connections.
|
||||
stream: S,
|
||||
stream: async_std::net::TcpListener,
|
||||
/// The current pause if any.
|
||||
pause: Option<Delay>,
|
||||
/// How long to pause after an error.
|
||||
pause_duration: Duration
|
||||
}
|
||||
|
||||
impl<S> Listener<S>
|
||||
where
|
||||
S: Stream,
|
||||
S::Error: std::fmt::Display
|
||||
{
|
||||
fn new(stream: S, duration: Duration) -> Self {
|
||||
Listener { stream, pause: None, pause_duration: duration }
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Stream for Listener<S>
|
||||
where
|
||||
S: Stream,
|
||||
S::Error: std::fmt::Display
|
||||
{
|
||||
type Item = S::Item;
|
||||
type Error = S::Error;
|
||||
|
||||
/// Polls for incoming connections, pausing if an error is encountered.
|
||||
fn poll(&mut self) -> Poll<Option<S::Item>, S::Error> {
|
||||
match self.pause.as_mut().map(|p| p.poll()) {
|
||||
Some(Ok(Async::NotReady)) => return Ok(Async::NotReady),
|
||||
Some(Ok(Async::Ready(()))) | Some(Err(_)) => { self.pause.take(); }
|
||||
None => ()
|
||||
}
|
||||
|
||||
match self.stream.poll() {
|
||||
Ok(x) => Ok(x),
|
||||
Err(e) => {
|
||||
debug!("error accepting incoming connection: {}", e);
|
||||
self.pause = Some(Delay::new(Instant::now() + self.pause_duration));
|
||||
Err(e)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Stream that listens on an TCP/IP address.
|
||||
#[derive(Debug)]
|
||||
pub struct TcpListenStream {
|
||||
/// Stream of incoming sockets.
|
||||
inner: Listener<Incoming>,
|
||||
pause_duration: Duration,
|
||||
/// The port which we use as our listen port in listener event addresses.
|
||||
port: u16,
|
||||
/// The set of known addresses.
|
||||
@ -445,7 +355,7 @@ fn check_for_interface_changes(
|
||||
for (ip, _, ma) in old_listen_addrs.iter() {
|
||||
if listen_addrs.iter().find(|(i, ..)| i == ip).is_none() {
|
||||
debug!("Expired listen address: {}", ma);
|
||||
pending.push_back(ListenerEvent::AddressExpired(ma.clone()));
|
||||
pending.push_back(Ok(ListenerEvent::AddressExpired(ma.clone())));
|
||||
}
|
||||
}
|
||||
|
||||
@ -453,7 +363,7 @@ fn check_for_interface_changes(
|
||||
for (ip, _, ma) in listen_addrs.iter() {
|
||||
if old_listen_addrs.iter().find(|(i, ..)| i == ip).is_none() {
|
||||
debug!("New listen address: {}", ma);
|
||||
pending.push_back(ListenerEvent::NewAddress(ma.clone()));
|
||||
pending.push_back(Ok(ListenerEvent::NewAddress(ma.clone())));
|
||||
}
|
||||
}
|
||||
|
||||
@ -470,21 +380,26 @@ fn check_for_interface_changes(
|
||||
Ok(())
|
||||
}
|
||||
|
||||
impl Stream for TcpListenStream {
|
||||
type Item = ListenerEvent<FutureResult<TcpTransStream, io::Error>>;
|
||||
type Error = io::Error;
|
||||
|
||||
fn poll(&mut self) -> Poll<Option<Self::Item>, io::Error> {
|
||||
impl TcpListenStream {
|
||||
/// Takes ownership of the listener, and returns the next incoming event and the listener.
|
||||
async fn next(mut self) -> (Result<ListenerEvent<Ready<Result<TcpTransStream, io::Error>>>, io::Error>, Self) {
|
||||
loop {
|
||||
if let Some(event) = self.pending.pop_front() {
|
||||
return Ok(Async::Ready(Some(event)))
|
||||
return (event, self);
|
||||
}
|
||||
|
||||
let sock = match self.inner.poll() {
|
||||
Ok(Async::Ready(Some(sock))) => sock,
|
||||
Ok(Async::Ready(None)) => return Ok(Async::Ready(None)),
|
||||
Ok(Async::NotReady) => return Ok(Async::NotReady),
|
||||
Err(e) => return Err(e)
|
||||
if let Some(pause) = self.pause.take() {
|
||||
let _ = pause.await;
|
||||
}
|
||||
|
||||
// TODO: do we get the peer_addr at the same time?
|
||||
let (sock, _) = match self.stream.accept().await {
|
||||
Ok(s) => s,
|
||||
Err(e) => {
|
||||
debug!("error accepting incoming connection: {}", e);
|
||||
self.pause = Some(Delay::new(self.pause_duration));
|
||||
return (Err(e), self);
|
||||
}
|
||||
};
|
||||
|
||||
let sock_addr = match sock.peer_addr() {
|
||||
@ -498,7 +413,9 @@ impl Stream for TcpListenStream {
|
||||
let local_addr = match sock.local_addr() {
|
||||
Ok(sock_addr) => {
|
||||
if let Addresses::Many(ref mut addrs) = self.addrs {
|
||||
check_for_interface_changes(&sock_addr, self.port, addrs, &mut self.pending)?
|
||||
if let Err(err) = check_for_interface_changes(&sock_addr, self.port, addrs, &mut self.pending) {
|
||||
return (Err(err), self);
|
||||
}
|
||||
}
|
||||
ip_to_multiaddr(sock_addr.ip(), sock_addr.port())
|
||||
}
|
||||
@ -513,19 +430,19 @@ impl Stream for TcpListenStream {
|
||||
match apply_config(&self.config, &sock) {
|
||||
Ok(()) => {
|
||||
trace!("Incoming connection from {} at {}", remote_addr, local_addr);
|
||||
self.pending.push_back(ListenerEvent::Upgrade {
|
||||
self.pending.push_back(Ok(ListenerEvent::Upgrade {
|
||||
upgrade: future::ok(TcpTransStream { inner: sock }),
|
||||
local_addr,
|
||||
remote_addr
|
||||
})
|
||||
}))
|
||||
}
|
||||
Err(err) => {
|
||||
debug!("Error upgrading incoming connection from {}: {:?}", remote_addr, err);
|
||||
self.pending.push_back(ListenerEvent::Upgrade {
|
||||
self.pending.push_back(Ok(ListenerEvent::Upgrade {
|
||||
upgrade: future::err(err),
|
||||
local_addr,
|
||||
remote_addr
|
||||
})
|
||||
}))
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -538,35 +455,27 @@ pub struct TcpTransStream {
|
||||
inner: TcpStream,
|
||||
}
|
||||
|
||||
impl Read for TcpTransStream {
|
||||
fn read(&mut self, buf: &mut [u8]) -> Result<usize, io::Error> {
|
||||
self.inner.read(buf)
|
||||
}
|
||||
}
|
||||
|
||||
impl AsyncRead for TcpTransStream {
|
||||
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool {
|
||||
self.inner.prepare_uninitialized_buffer(buf)
|
||||
fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll<Result<usize, io::Error>> {
|
||||
AsyncRead::poll_read(Pin::new(&mut self.inner), cx, buf)
|
||||
}
|
||||
|
||||
fn read_buf<B: bytes::BufMut>(&mut self, buf: &mut B) -> Poll<usize, io::Error> {
|
||||
self.inner.read_buf(buf)
|
||||
}
|
||||
}
|
||||
|
||||
impl Write for TcpTransStream {
|
||||
fn write(&mut self, buf: &[u8]) -> Result<usize, io::Error> {
|
||||
self.inner.write(buf)
|
||||
}
|
||||
|
||||
fn flush(&mut self) -> Result<(), io::Error> {
|
||||
self.inner.flush()
|
||||
unsafe fn initializer(&self) -> Initializer {
|
||||
self.inner.initializer()
|
||||
}
|
||||
}
|
||||
|
||||
impl AsyncWrite for TcpTransStream {
|
||||
fn shutdown(&mut self) -> Poll<(), io::Error> {
|
||||
AsyncWrite::shutdown(&mut self.inner)
|
||||
fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8]) -> Poll<Result<usize, io::Error>> {
|
||||
AsyncWrite::poll_write(Pin::new(&mut self.inner), cx, buf)
|
||||
}
|
||||
|
||||
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), io::Error>> {
|
||||
AsyncWrite::poll_flush(Pin::new(&mut self.inner), cx)
|
||||
}
|
||||
|
||||
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), io::Error>> {
|
||||
AsyncWrite::poll_close(Pin::new(&mut self.inner), cx)
|
||||
}
|
||||
}
|
||||
|
||||
@ -615,8 +524,7 @@ mod tests {
|
||||
.expect("listener");
|
||||
|
||||
// Get the first address.
|
||||
let addr = listener.by_ref()
|
||||
.wait()
|
||||
let addr = futures::executor::block_on_stream(listener.by_ref())
|
||||
.next()
|
||||
.expect("some event")
|
||||
.expect("no error")
|
||||
@ -626,7 +534,7 @@ mod tests {
|
||||
// Process all initial `NewAddress` events and make sure they
|
||||
// do not contain wildcard address or port.
|
||||
let server = listener
|
||||
.take_while(|event| match event {
|
||||
.take_while(|event| match event.as_ref().unwrap() {
|
||||
ListenerEvent::NewAddress(a) => {
|
||||
let mut iter = a.iter();
|
||||
match iter.next().expect("ip address") {
|
||||
@ -639,14 +547,14 @@ mod tests {
|
||||
} else {
|
||||
panic!("No TCP port in address: {}", a)
|
||||
}
|
||||
Ok(true)
|
||||
futures::future::ready(true)
|
||||
}
|
||||
_ => Ok(false)
|
||||
_ => futures::future::ready(false)
|
||||
})
|
||||
.for_each(|_| Ok(()));
|
||||
.for_each(|_| futures::future::ready(()));
|
||||
|
||||
let client = TcpConfig::new().dial(addr).expect("dialer");
|
||||
tokio::run(server.join(client).map(|_| ()).map_err(|e| panic!("error: {}", e)))
|
||||
futures::executor::block_on(futures::future::join(server, client)).1.unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
@ -705,8 +613,6 @@ mod tests {
|
||||
std::thread::spawn(move || {
|
||||
let addr = "/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap();
|
||||
let tcp = TcpConfig::new();
|
||||
let mut rt = Runtime::new().unwrap();
|
||||
let handle = rt.handle();
|
||||
let listener = tcp.listen_on(addr).unwrap()
|
||||
.filter_map(ListenerEvent::into_upgrade)
|
||||
.for_each(|(sock, _)| {
|
||||
@ -720,12 +626,11 @@ mod tests {
|
||||
// Spawn the future as a concurrent task
|
||||
handle.spawn(handle_conn).unwrap();
|
||||
|
||||
Ok(())
|
||||
futures::future::ready(())
|
||||
})
|
||||
});
|
||||
|
||||
rt.block_on(listener).unwrap();
|
||||
rt.run().unwrap();
|
||||
futures::executor::block_on(listener);
|
||||
});
|
||||
std::thread::sleep(std::time::Duration::from_millis(100));
|
||||
let addr = "/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap();
|
||||
@ -733,13 +638,12 @@ mod tests {
|
||||
// Obtain a future socket through dialing
|
||||
let socket = tcp.dial(addr.clone()).unwrap();
|
||||
// Define what to do with the socket once it's obtained
|
||||
let action = socket.then(|sock| -> Result<(), ()> {
|
||||
let action = socket.then(|sock| {
|
||||
sock.unwrap().write(&[0x1, 0x2, 0x3]).unwrap();
|
||||
Ok(())
|
||||
futures::future::ready(())
|
||||
});
|
||||
// Execute the future in our event loop
|
||||
let mut rt = Runtime::new().unwrap();
|
||||
let _ = rt.block_on(action).unwrap();
|
||||
futures::executor::block_on(action);
|
||||
}
|
||||
|
||||
#[test]
|
||||
@ -749,7 +653,7 @@ mod tests {
|
||||
let addr = "/ip4/127.0.0.1/tcp/0".parse::<Multiaddr>().unwrap();
|
||||
assert!(addr.to_string().contains("tcp/0"));
|
||||
|
||||
let new_addr = tcp.listen_on(addr).unwrap().wait()
|
||||
let new_addr = futures::executor::block_on_stream(tcp.listen_on(addr).unwrap())
|
||||
.next()
|
||||
.expect("some event")
|
||||
.expect("no error")
|
||||
@ -766,7 +670,7 @@ mod tests {
|
||||
let addr: Multiaddr = "/ip6/::1/tcp/0".parse().unwrap();
|
||||
assert!(addr.to_string().contains("tcp/0"));
|
||||
|
||||
let new_addr = tcp.listen_on(addr).unwrap().wait()
|
||||
let new_addr = futures::executor::block_on_stream(tcp.listen_on(addr).unwrap())
|
||||
.next()
|
||||
.expect("some event")
|
||||
.expect("no error")
|
||||
|
Reference in New Issue
Block a user