Swarm rework (#182)

* Rename Transport::RawConn to Output

* Remove AsyncRead + AsyncWrite bound on Transport::Output

* UpgradedNode now always implements Transport

* Add and tweak modifiers for Transport and ConnectionUpgrade

* Secio upgrade now returns the pubkey in its output

* Add upgrade::apply

* Add Transport::and_then

* Rework the swarm

* Rustfmt

* Fix concerns
This commit is contained in:
Pierre Krieger 2018-05-14 15:55:16 +02:00 committed by GitHub
parent 4382adcbde
commit f787f3d8b8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
58 changed files with 833 additions and 526 deletions

View File

@ -29,8 +29,8 @@
extern crate smallvec; extern crate smallvec;
use std::ops::Drop;
use std::mem::ManuallyDrop; use std::mem::ManuallyDrop;
use std::ops::Drop;
pub use smallvec::Array; pub use smallvec::Array;
@ -41,8 +41,8 @@ use owned_slice::OwnedSlice;
/// elements of these slices would be leaked after the slice goes out of scope. `OwnedSlice` simply /// elements of these slices would be leaked after the slice goes out of scope. `OwnedSlice` simply
/// manually drops all its elements when it goes out of scope. /// manually drops all its elements when it goes out of scope.
pub mod owned_slice { pub mod owned_slice {
use std::ops::{Deref, DerefMut, Drop};
use std::mem::ManuallyDrop; use std::mem::ManuallyDrop;
use std::ops::{Deref, DerefMut, Drop};
/// A slice that owns its elements, but not their storage. This is useful for things like /// A slice that owns its elements, but not their storage. This is useful for things like
/// `Vec::retain` and `CircularBuffer::pop_slice`, since these functions can return a slice but /// `Vec::retain` and `CircularBuffer::pop_slice`, since these functions can return a slice but

View File

@ -27,8 +27,8 @@ use futures::stream::{iter_ok, Stream};
use query::{naive_apply_query, Query}; use query::{naive_apply_query, Query};
use serde::Serialize; use serde::Serialize;
use serde::de::DeserializeOwned; use serde::de::DeserializeOwned;
use serde_json::{from_reader, from_value, to_value, to_writer, Map};
use serde_json::value::Value; use serde_json::value::Value;
use serde_json::{from_reader, from_value, to_value, to_writer, Map};
use std::borrow::Cow; use std::borrow::Cow;
use std::fs; use std::fs;
use std::io::Cursor; use std::io::Cursor;
@ -264,11 +264,11 @@ where
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use {Filter, FilterOp, FilterTy, Order, Query};
use Datastore; use Datastore;
use JsonFileDatastore; use JsonFileDatastore;
use futures::{Future, Stream}; use futures::{Future, Stream};
use tempfile::NamedTempFile; use tempfile::NamedTempFile;
use {Filter, FilterOp, FilterTy, Order, Query};
#[test] #[test]
fn open_and_flush() { fn open_and_flush() {

View File

@ -113,8 +113,8 @@ use std::borrow::Cow;
use std::io::Error as IoError; use std::io::Error as IoError;
use std::ops::DerefMut; use std::ops::DerefMut;
mod query;
mod json_file; mod json_file;
mod query;
pub use self::json_file::{JsonFileDatastore, JsonFileDatastoreEntry}; pub use self::json_file::{JsonFileDatastore, JsonFileDatastoreEntry};
pub use self::query::{Filter, FilterOp, FilterTy, Order, Query}; pub use self::query::{Filter, FilterOp, FilterTy, Order, Query};

View File

@ -18,8 +18,8 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use futures::{Async, Future, Poll, Stream};
use futures::stream::{iter_ok, Skip as StreamSkip, Take as StreamTake}; use futures::stream::{iter_ok, Skip as StreamSkip, Take as StreamTake};
use futures::{Async, Future, Poll, Stream};
use std::borrow::Cow; use std::borrow::Cow;
use std::cmp::Ordering; use std::cmp::Ordering;
use std::io::Error as IoError; use std::io::Error as IoError;

View File

@ -99,10 +99,10 @@ impl<T> Transport for DnsConfig<T>
where where
T: Transport + 'static, // TODO: 'static :-/ T: Transport + 'static, // TODO: 'static :-/
{ {
type RawConn = T::RawConn; type Output = T::Output;
type Listener = T::Listener; type Listener = T::Listener;
type ListenerUpgrade = T::ListenerUpgrade; type ListenerUpgrade = T::ListenerUpgrade;
type Dial = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
@ -238,10 +238,10 @@ mod tests {
#[derive(Clone)] #[derive(Clone)]
struct CustomTransport; struct CustomTransport;
impl Transport for CustomTransport { impl Transport for CustomTransport {
type RawConn = <TcpConfig as Transport>::RawConn; type Output = <TcpConfig as Transport>::Output;
type Listener = <TcpConfig as Transport>::Listener; type Listener = <TcpConfig as Transport>::Listener;
type ListenerUpgrade = <TcpConfig as Transport>::ListenerUpgrade; type ListenerUpgrade = <TcpConfig as Transport>::ListenerUpgrade;
type Dial = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
#[inline] #[inline]
fn listen_on( fn listen_on(

View File

@ -29,11 +29,11 @@ extern crate libp2p_websocket as websocket;
extern crate tokio_core; extern crate tokio_core;
extern crate tokio_io; extern crate tokio_io;
use futures::{Future, Sink, Stream};
use futures::sync::oneshot; use futures::sync::oneshot;
use futures::{Future, Sink, Stream};
use std::env; use std::env;
use swarm::Transport; use swarm::Transport;
use swarm::upgrade::{self, DeniedConnectionUpgrade, SimpleProtocol, UpgradeExt}; use swarm::upgrade::{self, DeniedConnectionUpgrade, SimpleProtocol};
use tcp::TcpConfig; use tcp::TcpConfig;
use tokio_core::reactor::Core; use tokio_core::reactor::Core;
use tokio_io::AsyncRead; use tokio_io::AsyncRead;
@ -73,7 +73,7 @@ fn main() {
} }
}; };
plain_text.or_upgrade(secio) upgrade::or(plain_text, upgrade::map(secio, |(socket, _)| socket))
}) })
// On top of plaintext or secio, we will use the multiplex protocol. // On top of plaintext or secio, we will use the multiplex protocol.
@ -89,8 +89,7 @@ fn main() {
// by the listening part. We don't want to accept anything, so we pass a dummy object that // by the listening part. We don't want to accept anything, so we pass a dummy object that
// represents a connection that is always denied. // represents a connection that is always denied.
let (swarm_controller, swarm_future) = swarm::swarm( let (swarm_controller, swarm_future) = swarm::swarm(
transport, transport.clone().with_upgrade(DeniedConnectionUpgrade),
DeniedConnectionUpgrade,
|_socket, _client_addr| -> Result<(), _> { |_socket, _client_addr| -> Result<(), _> {
unreachable!("All incoming connections should have been denied") unreachable!("All incoming connections should have been denied")
}, },
@ -108,7 +107,7 @@ fn main() {
// We now use the controller to dial to the address. // We now use the controller to dial to the address.
let (finished_tx, finished_rx) = oneshot::channel(); let (finished_tx, finished_rx) = oneshot::channel();
swarm_controller swarm_controller
.dial_custom_handler(target_addr.parse().expect("invalid multiaddr"), proto, |echo, _| { .dial_custom_handler(target_addr.parse().expect("invalid multiaddr"), transport.with_upgrade(proto), |echo, _| {
// `echo` is what the closure used when initializing `proto` returns. // `echo` is what the closure used when initializing `proto` returns.
// Consequently, please note that the `send` method is available only because the type // Consequently, please note that the `send` method is available only because the type
// `length_delimited::Framed` has a `send` method. // `length_delimited::Framed` has a `send` method.

View File

@ -33,7 +33,7 @@ use futures::future::{loop_fn, Future, IntoFuture, Loop};
use futures::{Sink, Stream}; use futures::{Sink, Stream};
use std::env; use std::env;
use swarm::Transport; use swarm::Transport;
use swarm::upgrade::{self, SimpleProtocol, UpgradeExt}; use swarm::upgrade::{self, SimpleProtocol};
use tcp::TcpConfig; use tcp::TcpConfig;
use tokio_core::reactor::Core; use tokio_core::reactor::Core;
use tokio_io::AsyncRead; use tokio_io::AsyncRead;
@ -72,7 +72,7 @@ fn main() {
} }
}; };
plain_text.or_upgrade(secio) upgrade::or(plain_text, upgrade::map(secio, |(socket, _)| socket))
}) })
// On top of plaintext or secio, we will use the multiplex protocol. // On top of plaintext or secio, we will use the multiplex protocol.
@ -99,36 +99,39 @@ fn main() {
// Let's put this `transport` into a *swarm*. The swarm will handle all the incoming and // Let's put this `transport` into a *swarm*. The swarm will handle all the incoming and
// outgoing connections for us. // outgoing connections for us.
let (swarm_controller, swarm_future) = swarm::swarm(transport, proto, |socket, client_addr| { let (swarm_controller, swarm_future) = swarm::swarm(
println!("Successfully negotiated protocol with {}", client_addr); transport.clone().with_upgrade(proto),
|socket, client_addr| {
println!("Successfully negotiated protocol with {}", client_addr);
// The type of `socket` is exactly what the closure of `SimpleProtocol` returns. // The type of `socket` is exactly what the closure of `SimpleProtocol` returns.
// We loop forever in order to handle all the messages sent by the client. // We loop forever in order to handle all the messages sent by the client.
loop_fn(socket, move |socket| { loop_fn(socket, move |socket| {
let client_addr = client_addr.clone(); let client_addr = client_addr.clone();
socket socket
.into_future() .into_future()
.map_err(|(e, _)| e) .map_err(|(e, _)| e)
.and_then(move |(msg, rest)| { .and_then(move |(msg, rest)| {
if let Some(msg) = msg { if let Some(msg) = msg {
// One message has been received. We send it back to the client. // One message has been received. We send it back to the client.
println!( println!(
"Received a message from {}: {:?}\n => Sending back \ "Received a message from {}: {:?}\n => Sending back \
identical message to remote", identical message to remote",
client_addr, msg client_addr, msg
); );
Box::new(rest.send(msg.freeze()).map(|m| Loop::Continue(m))) Box::new(rest.send(msg.freeze()).map(|m| Loop::Continue(m)))
as Box<Future<Item = _, Error = _>> as Box<Future<Item = _, Error = _>>
} else { } else {
// End of stream. Connection closed. Breaking the loop. // End of stream. Connection closed. Breaking the loop.
println!("Received EOF from {}\n => Dropping connection", client_addr); println!("Received EOF from {}\n => Dropping connection", client_addr);
Box::new(Ok(Loop::Break(())).into_future()) Box::new(Ok(Loop::Break(())).into_future())
as Box<Future<Item = _, Error = _>> as Box<Future<Item = _, Error = _>>
} }
}) })
}) })
}); },
);
// We now use the controller to listen on the address. // We now use the controller to listen on the address.
let address = swarm_controller let address = swarm_controller

View File

@ -33,12 +33,12 @@ extern crate tokio_core;
extern crate tokio_io; extern crate tokio_io;
extern crate tokio_stdin; extern crate tokio_stdin;
use futures::future::Future;
use futures::Stream; use futures::Stream;
use futures::future::Future;
use peerstore::PeerId; use peerstore::PeerId;
use std::{env, mem}; use std::{env, mem};
use swarm::upgrade;
use swarm::{Multiaddr, Transport}; use swarm::{Multiaddr, Transport};
use swarm::upgrade::{self, UpgradeExt};
use tcp::TcpConfig; use tcp::TcpConfig;
use tokio_core::reactor::Core; use tokio_core::reactor::Core;
use websocket::WsConfig; use websocket::WsConfig;
@ -75,7 +75,7 @@ fn main() {
} }
}; };
plain_text.or_upgrade(secio) upgrade::or(plain_text, upgrade::map(secio, |(socket, _)| socket))
}) })
// On top of plaintext or secio, we will use the multiplex protocol. // On top of plaintext or secio, we will use the multiplex protocol.
@ -102,8 +102,7 @@ fn main() {
// Let's put this `transport` into a *swarm*. The swarm will handle all the incoming and // Let's put this `transport` into a *swarm*. The swarm will handle all the incoming and
// outgoing connections for us. // outgoing connections for us.
let (swarm_controller, swarm_future) = swarm::swarm( let (swarm_controller, swarm_future) = swarm::swarm(
transport, transport.clone().with_upgrade(floodsub_upgrade.clone()),
floodsub_upgrade.clone(),
|socket, client_addr| { |socket, client_addr| {
println!("Successfully negotiated protocol with {}", client_addr); println!("Successfully negotiated protocol with {}", client_addr);
socket socket
@ -142,7 +141,10 @@ fn main() {
let target: Multiaddr = msg[6..].parse().unwrap(); let target: Multiaddr = msg[6..].parse().unwrap();
println!("*Dialing {}*", target); println!("*Dialing {}*", target);
swarm_controller swarm_controller
.dial_to_handler(target, floodsub_upgrade.clone()) .dial_to_handler(
target,
transport.clone().with_upgrade(floodsub_upgrade.clone()),
)
.unwrap(); .unwrap();
} else { } else {
floodsub_ctl.publish(&topic, msg.into_bytes()); floodsub_ctl.publish(&topic, msg.into_bytes());

View File

@ -40,7 +40,7 @@ use std::env;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use swarm::Transport; use swarm::Transport;
use swarm::upgrade::{self, UpgradeExt}; use swarm::upgrade;
use tcp::TcpConfig; use tcp::TcpConfig;
use tokio_core::reactor::Core; use tokio_core::reactor::Core;
@ -80,7 +80,7 @@ fn main() {
} }
}; };
plain_text.or_upgrade(secio) upgrade::or(plain_text, upgrade::map(secio, |(socket, _)| socket))
}) })
// On top of plaintext or secio, we will use the multiplex protocol. // On top of plaintext or secio, we will use the multiplex protocol.
@ -116,9 +116,13 @@ fn main() {
// Let's put this `transport` into a *swarm*. The swarm will handle all the incoming and // Let's put this `transport` into a *swarm*. The swarm will handle all the incoming and
// outgoing connections for us. // outgoing connections for us.
let (swarm_controller, swarm_future) = swarm::swarm(transport, proto, |upgrade, _| upgrade); let (swarm_controller, swarm_future) = swarm::swarm(
transport.clone().with_upgrade(proto.clone()),
|upgrade, _| upgrade,
);
let (kad_controller, _kad_init) = kad_ctl_proto.start(swarm_controller.clone()); let (kad_controller, _kad_init) =
kad_ctl_proto.start(swarm_controller.clone(), transport.with_upgrade(proto));
for listen_addr in listen_addrs { for listen_addr in listen_addrs {
let addr = swarm_controller let addr = swarm_controller

View File

@ -33,7 +33,7 @@ use futures::Future;
use futures::sync::oneshot; use futures::sync::oneshot;
use std::env; use std::env;
use swarm::Transport; use swarm::Transport;
use swarm::upgrade::{self, DeniedConnectionUpgrade, UpgradeExt}; use swarm::upgrade::{self, DeniedConnectionUpgrade};
use tcp::TcpConfig; use tcp::TcpConfig;
use tokio_core::reactor::Core; use tokio_core::reactor::Core;
@ -65,7 +65,7 @@ fn main() {
} }
}; };
plain_text.or_upgrade(secio) upgrade::or(plain_text, upgrade::map(secio, |(socket, _)| socket))
}) })
// On top of plaintext or secio, we will use the multiplex protocol. // On top of plaintext or secio, we will use the multiplex protocol.
@ -81,8 +81,7 @@ fn main() {
// by the listening part. We don't want to accept anything, so we pass a dummy object that // by the listening part. We don't want to accept anything, so we pass a dummy object that
// represents a connection that is always denied. // represents a connection that is always denied.
let (swarm_controller, swarm_future) = swarm::swarm( let (swarm_controller, swarm_future) = swarm::swarm(
transport, transport.clone().with_upgrade(DeniedConnectionUpgrade),
DeniedConnectionUpgrade,
|_socket, _client_addr| -> Result<(), _> { |_socket, _client_addr| -> Result<(), _> {
unreachable!("All incoming connections should have been denied") unreachable!("All incoming connections should have been denied")
}, },
@ -91,7 +90,8 @@ fn main() {
// We now use the controller to dial to the address. // We now use the controller to dial to the address.
let (tx, rx) = oneshot::channel(); let (tx, rx) = oneshot::channel();
swarm_controller swarm_controller
.dial_custom_handler(target_addr.parse().expect("invalid multiaddr"), ping::Ping, .dial_custom_handler(target_addr.parse().expect("invalid multiaddr"),
transport.with_upgrade(ping::Ping),
|(mut pinger, future), _| { |(mut pinger, future), _| {
let ping = pinger.ping().map_err(|_| unreachable!()).inspect(|_| { let ping = pinger.ping().map_err(|_| unreachable!()).inspect(|_| {
println!("Received pong from the remote"); println!("Received pong from the remote");

View File

@ -39,17 +39,11 @@ mod topic;
pub use self::topic::{Topic, TopicBuilder, TopicHash}; pub use self::topic::{Topic, TopicBuilder, TopicHash};
use std::fmt;
use std::hash::{Hash, Hasher};
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::iter;
use std::sync::Arc;
use std::sync::atomic::{AtomicUsize, Ordering};
use bytes::{Bytes, BytesMut};
use byteorder::{BigEndian, WriteBytesExt}; use byteorder::{BigEndian, WriteBytesExt};
use bytes::{Bytes, BytesMut};
use fnv::{FnvHashMap, FnvHashSet, FnvHasher}; use fnv::{FnvHashMap, FnvHashSet, FnvHasher};
use futures::{future, Future, Poll, Sink, Stream};
use futures::sync::mpsc; use futures::sync::mpsc;
use futures::{future, Future, Poll, Sink, Stream};
use libp2p_peerstore::PeerId; use libp2p_peerstore::PeerId;
use libp2p_swarm::{ConnectionUpgrade, Endpoint}; use libp2p_swarm::{ConnectionUpgrade, Endpoint};
use log::Level; use log::Level;
@ -57,6 +51,12 @@ use multiaddr::{AddrComponent, Multiaddr};
use parking_lot::{Mutex, RwLock}; use parking_lot::{Mutex, RwLock};
use protobuf::Message as ProtobufMessage; use protobuf::Message as ProtobufMessage;
use smallvec::SmallVec; use smallvec::SmallVec;
use std::fmt;
use std::hash::{Hash, Hasher};
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::iter;
use std::sync::Arc;
use std::sync::atomic::{AtomicUsize, Ordering};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use varint::VarintCodec; use varint::VarintCodec;

View File

@ -29,8 +29,8 @@ use protobuf::repeated::RepeatedField;
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::iter; use std::iter;
use structs_proto; use structs_proto;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_io::codec::Framed; use tokio_io::codec::Framed;
use tokio_io::{AsyncRead, AsyncWrite};
use varint::VarintCodec; use varint::VarintCodec;
/// Configuration for an upgrade to the identity protocol. /// Configuration for an upgrade to the identity protocol.
@ -238,11 +238,11 @@ mod tests {
use self::libp2p_tcp_transport::TcpConfig; use self::libp2p_tcp_transport::TcpConfig;
use self::tokio_core::reactor::Core; use self::tokio_core::reactor::Core;
use {IdentifyInfo, IdentifyOutput, IdentifyProtocolConfig};
use futures::{Future, Stream}; use futures::{Future, Stream};
use libp2p_swarm::Transport; use libp2p_swarm::Transport;
use std::sync::mpsc; use std::sync::mpsc;
use std::thread; use std::thread;
use {IdentifyInfo, IdentifyOutput, IdentifyProtocolConfig};
#[test] #[test]
fn correct_transfer() { fn correct_transfer() {

View File

@ -26,6 +26,7 @@ use protocol::{IdentifyInfo, IdentifyOutput, IdentifyProtocolConfig};
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::ops::Deref; use std::ops::Deref;
use std::time::Duration; use std::time::Duration;
use tokio_io::{AsyncRead, AsyncWrite};
/// Implementation of `Transport`. See [the crate root description](index.html). /// Implementation of `Transport`. See [the crate root description](index.html).
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
@ -59,13 +60,14 @@ impl<Trans, PStoreRef> IdentifyTransport<Trans, PStoreRef> {
impl<Trans, PStore, PStoreRef> Transport for IdentifyTransport<Trans, PStoreRef> impl<Trans, PStore, PStoreRef> Transport for IdentifyTransport<Trans, PStoreRef>
where where
Trans: Transport + Clone + 'static, // TODO: 'static :( Trans: Transport + Clone + 'static, // TODO: 'static :(
Trans::Output: AsyncRead + AsyncWrite,
PStoreRef: Deref<Target = PStore> + Clone + 'static, // TODO: 'static :( PStoreRef: Deref<Target = PStore> + Clone + 'static, // TODO: 'static :(
for<'r> &'r PStore: Peerstore, for<'r> &'r PStore: Peerstore,
{ {
type RawConn = Trans::RawConn; type Output = Trans::Output;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>; type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
type ListenerUpgrade = Box<Future<Item = (Trans::RawConn, Multiaddr), Error = IoError>>; type ListenerUpgrade = Box<Future<Item = (Trans::Output, Multiaddr), Error = IoError>>;
type Dial = Box<Future<Item = (Trans::RawConn, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (Trans::Output, Multiaddr), Error = IoError>>;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
@ -275,11 +277,12 @@ where
impl<Trans, PStore, PStoreRef> MuxedTransport for IdentifyTransport<Trans, PStoreRef> impl<Trans, PStore, PStoreRef> MuxedTransport for IdentifyTransport<Trans, PStoreRef>
where where
Trans: MuxedTransport + Clone + 'static, Trans: MuxedTransport + Clone + 'static,
Trans::Output: AsyncRead + AsyncWrite,
PStoreRef: Deref<Target = PStore> + Clone + 'static, PStoreRef: Deref<Target = PStore> + Clone + 'static,
for<'r> &'r PStore: Peerstore, for<'r> &'r PStore: Peerstore,
{ {
type Incoming = Box<Future<Item = Self::IncomingUpgrade, Error = IoError>>; type Incoming = Box<Future<Item = Self::IncomingUpgrade, Error = IoError>>;
type IncomingUpgrade = Box<Future<Item = (Trans::RawConn, Multiaddr), Error = IoError>>; type IncomingUpgrade = Box<Future<Item = (Trans::Output, Multiaddr), Error = IoError>>;
#[inline] #[inline]
fn next_incoming(self) -> Self::Incoming { fn next_incoming(self) -> Self::Incoming {
@ -391,8 +394,8 @@ mod tests {
use self::tokio_core::reactor::Core; use self::tokio_core::reactor::Core;
use IdentifyTransport; use IdentifyTransport;
use futures::{Future, Stream}; use futures::{Future, Stream};
use libp2p_peerstore::{PeerAccess, PeerId, Peerstore};
use libp2p_peerstore::memory_peerstore::MemoryPeerstore; use libp2p_peerstore::memory_peerstore::MemoryPeerstore;
use libp2p_peerstore::{PeerAccess, PeerId, Peerstore};
use libp2p_swarm::Transport; use libp2p_swarm::Transport;
use multiaddr::{AddrComponent, Multiaddr}; use multiaddr::{AddrComponent, Multiaddr};
use std::io::Error as IoError; use std::io::Error as IoError;
@ -410,9 +413,9 @@ mod tests {
inner: TcpConfig, inner: TcpConfig,
} }
impl Transport for UnderlyingTrans { impl Transport for UnderlyingTrans {
type RawConn = <TcpConfig as Transport>::RawConn; type Output = <TcpConfig as Transport>::Output;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>; type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
type ListenerUpgrade = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; type ListenerUpgrade = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
type Dial = <TcpConfig as Transport>::Dial; type Dial = <TcpConfig as Transport>::Dial;
#[inline] #[inline]
fn listen_on( fn listen_on(

View File

@ -24,13 +24,12 @@
use bytes::Bytes; use bytes::Bytes;
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::{self, future, Future};
use futures::sync::oneshot; use futures::sync::oneshot;
use futures::{self, future, Future};
use kad_server::{KadServerInterface, KademliaServerConfig, KademliaServerController}; use kad_server::{KadServerInterface, KademliaServerConfig, KademliaServerController};
use kbucket::{KBucketsPeerId, KBucketsTable, UpdateOutcome}; use kbucket::{KBucketsPeerId, KBucketsTable, UpdateOutcome};
use libp2p_peerstore::{PeerAccess, PeerId, Peerstore}; use libp2p_peerstore::{PeerAccess, PeerId, Peerstore};
use libp2p_swarm::{Endpoint, MuxedTransport, SwarmController}; use libp2p_swarm::{ConnectionUpgrade, Endpoint, MuxedTransport, SwarmController, Transport};
use libp2p_swarm::ConnectionUpgrade;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use parking_lot::Mutex; use parking_lot::Mutex;
use protocol::ConnectionType; use protocol::ConnectionType;
@ -95,27 +94,28 @@ where
} }
/// Turns the prototype into an actual controller by feeding it a swarm. /// Turns the prototype into an actual controller by feeding it a swarm.
pub fn start<T, C>( pub fn start<T, K>(
self, self,
swarm: SwarmController<T, C>, swarm: SwarmController<T>,
kademlia_transport: K,
) -> ( ) -> (
KademliaController<P, R, T, C>, KademliaController<P, R, T, K>,
Box<Future<Item = (), Error = IoError>>, Box<Future<Item = (), Error = IoError>>,
) )
where where
P: Clone + Deref<Target = Pc> + 'static, // TODO: 'static :-/ P: Clone + Deref<Target = Pc> + 'static, // TODO: 'static :-/
for<'r> &'r Pc: Peerstore, for<'r> &'r Pc: Peerstore,
R: Clone + 'static, // TODO: 'static :-/ R: Clone + 'static, // TODO: 'static :-/
T: Clone + MuxedTransport + 'static, // TODO: 'static :-/ T: Clone + MuxedTransport + 'static, // TODO: 'static :-/
C: Clone + ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/ T::Output: From<KademliaProcessingFuture>,
C::NamesIter: Clone, K: Transport<Output = KademliaProcessingFuture> + Clone + 'static, // TODO: 'static :-/
C::Output: From<KademliaProcessingFuture>,
{ {
// TODO: initialization // TODO: initialization
let controller = KademliaController { let controller = KademliaController {
inner: self.inner.clone(), inner: self.inner.clone(),
swarm_controller: swarm, swarm_controller: swarm,
kademlia_transport,
}; };
let init_future = { let init_future = {
@ -142,40 +142,40 @@ where
/// Object that allows one to make queries on the Kademlia system. /// Object that allows one to make queries on the Kademlia system.
#[derive(Debug)] #[derive(Debug)]
pub struct KademliaController<P, R, T, C> pub struct KademliaController<P, R, T, K>
where where
T: MuxedTransport + 'static, // TODO: 'static :-/ T: MuxedTransport + 'static, // TODO: 'static :-/
C: ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/
{ {
inner: Arc<Inner<P, R>>, inner: Arc<Inner<P, R>>,
swarm_controller: SwarmController<T, C>, swarm_controller: SwarmController<T>,
kademlia_transport: K,
} }
impl<P, R, T, C> Clone for KademliaController<P, R, T, C> impl<P, R, T, K> Clone for KademliaController<P, R, T, K>
where where
T: Clone + MuxedTransport + 'static, // TODO: 'static :-/ T: Clone + MuxedTransport + 'static, // TODO: 'static :-/
C: Clone + ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/ K: Clone,
{ {
#[inline] #[inline]
fn clone(&self) -> Self { fn clone(&self) -> Self {
KademliaController { KademliaController {
inner: self.inner.clone(), inner: self.inner.clone(),
swarm_controller: self.swarm_controller.clone(), swarm_controller: self.swarm_controller.clone(),
kademlia_transport: self.kademlia_transport.clone(),
} }
} }
} }
impl<P, Pc, R, T, C> KademliaController<P, R, T, C> impl<P, Pc, R, T, K> KademliaController<P, R, T, K>
where where
P: Deref<Target = Pc>, P: Deref<Target = Pc>,
for<'r> &'r Pc: Peerstore, for<'r> &'r Pc: Peerstore,
R: Clone, R: Clone,
T: Clone + MuxedTransport + 'static, // TODO: 'static :-/ T: Clone + MuxedTransport + 'static, // TODO: 'static :-/
C: Clone + ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/
{ {
/// Performs an iterative find node query on the network. /// Performs an iterative find node query on the network.
/// ///
/// Will query the network for the peers that are the closest to `searched_key` and return /// Will query the network for the peers that4 are the closest to `searched_key` and return
/// the results. /// the results.
/// ///
/// The algorithm used is a standard Kademlia algorithm. The details are not documented, so /// The algorithm used is a standard Kademlia algorithm. The details are not documented, so
@ -188,8 +188,8 @@ where
where where
P: Clone + 'static, P: Clone + 'static,
R: 'static, R: 'static,
C::NamesIter: Clone, T::Output: From<KademliaProcessingFuture>,
C::Output: From<KademliaProcessingFuture>, K: Transport<Output = KademliaProcessingFuture> + Clone + 'static,
{ {
query::find_node(self.clone(), searched_key) query::find_node(self.clone(), searched_key)
} }
@ -214,10 +214,9 @@ impl<P, R> KademliaUpgrade<P, R> {
/// Builds a connection upgrade from the controller. /// Builds a connection upgrade from the controller.
#[inline] #[inline]
pub fn from_controller<T, C>(ctl: &KademliaController<P, R, T, C>) -> Self pub fn from_controller<T, K>(ctl: &KademliaController<P, R, T, K>) -> Self
where where
T: MuxedTransport, T: MuxedTransport,
C: ConnectionUpgrade<T::RawConn>,
{ {
KademliaUpgrade { KademliaUpgrade {
inner: ctl.inner.clone(), inner: ctl.inner.clone(),
@ -408,15 +407,14 @@ where
} }
} }
impl<R, P, Pc, T, C> query::QueryInterface for KademliaController<P, R, T, C> impl<R, P, Pc, T, K> query::QueryInterface for KademliaController<P, R, T, K>
where where
P: Clone + Deref<Target = Pc> + 'static, // TODO: 'static :-/ P: Clone + Deref<Target = Pc> + 'static, // TODO: 'static :-/
for<'r> &'r Pc: Peerstore, for<'r> &'r Pc: Peerstore,
R: Clone + 'static, // TODO: 'static :-/ R: Clone + 'static, // TODO: 'static :-/
T: Clone + MuxedTransport + 'static, // TODO: 'static :-/ T: Clone + MuxedTransport + 'static, // TODO: 'static :-/
C: Clone + ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/ T::Output: From<KademliaProcessingFuture>,
C::NamesIter: Clone, K: Transport<Output = KademliaProcessingFuture> + Clone + 'static,
C::Output: From<KademliaProcessingFuture>,
{ {
#[inline] #[inline]
fn local_id(&self) -> &PeerId { fn local_id(&self) -> &PeerId {
@ -469,11 +467,9 @@ where
} }
Entry::Vacant(entry) => { Entry::Vacant(entry) => {
// Need to open a connection. // Need to open a connection.
let proto = KademliaUpgrade { match self.swarm_controller
inner: self.inner.clone(), .dial_to_handler(addr, self.kademlia_transport.clone())
upgrade: KademliaServerConfig::new(self.inner.clone()), {
};
match self.swarm_controller.dial_to_handler(addr, proto) {
Ok(()) => (), Ok(()) => (),
Err(_addr) => { Err(_addr) => {
let fut = future::err(IoError::new( let fut = future::err(IoError::new(

View File

@ -36,8 +36,8 @@
//! `Arc` in order to be available whenever we need to request something from a node. //! `Arc` in order to be available whenever we need to request something from a node.
use bytes::Bytes; use bytes::Bytes;
use futures::{future, Future, Sink, Stream};
use futures::sync::{mpsc, oneshot}; use futures::sync::{mpsc, oneshot};
use futures::{future, Future, Sink, Stream};
use libp2p_peerstore::PeerId; use libp2p_peerstore::PeerId;
use libp2p_swarm::ConnectionUpgrade; use libp2p_swarm::ConnectionUpgrade;
use libp2p_swarm::Endpoint; use libp2p_swarm::Endpoint;

View File

@ -26,8 +26,8 @@
//! used to send messages. //! used to send messages.
use bytes::Bytes; use bytes::Bytes;
use futures::{Sink, Stream};
use futures::future; use futures::future;
use futures::{Sink, Stream};
use libp2p_peerstore::PeerId; use libp2p_peerstore::PeerId;
use libp2p_swarm::{ConnectionUpgrade, Endpoint, Multiaddr}; use libp2p_swarm::{ConnectionUpgrade, Endpoint, Multiaddr};
use protobuf::{self, Message}; use protobuf::{self, Message};
@ -162,9 +162,9 @@ where
} }
/// Custom trait that derives `Sink` and `Stream`, so that we can box it. /// Custom trait that derives `Sink` and `Stream`, so that we can box it.
pub trait KadStreamSink pub trait KadStreamSink:
: Stream<Item = KadMsg, Error = IoError> + Sink<SinkItem = KadMsg, SinkError = IoError> Stream<Item = KadMsg, Error = IoError> + Sink<SinkItem = KadMsg, SinkError = IoError>
{ {
} }
impl<T> KadStreamSink for T impl<T> KadStreamSink for T
where where

View File

@ -35,25 +35,25 @@ extern crate rand;
extern crate tokio_io; extern crate tokio_io;
extern crate varint; extern crate varint;
mod read;
mod write;
mod shared;
mod header; mod header;
mod read;
mod shared;
mod write;
use bytes::Bytes; use bytes::Bytes;
use circular_buffer::Array; use circular_buffer::Array;
use futures::{Async, Future, Poll};
use futures::future::{self, FutureResult}; use futures::future::{self, FutureResult};
use header::MultiplexHeader; use futures::{Async, Future, Poll};
use swarm::muxing::StreamMuxer;
use swarm::{ConnectionUpgrade, Endpoint, Multiaddr};
use futures_mutex::Mutex; use futures_mutex::Mutex;
use header::MultiplexHeader;
use read::{read_stream, MultiplexReadState}; use read::{read_stream, MultiplexReadState};
use shared::{buf_from_slice, ByteBuf, MultiplexShared}; use shared::{buf_from_slice, ByteBuf, MultiplexShared};
use std::iter;
use std::io::{self, Read, Write}; use std::io::{self, Read, Write};
use std::iter;
use std::sync::Arc; use std::sync::Arc;
use std::sync::atomic::{self, AtomicUsize}; use std::sync::atomic::{self, AtomicUsize};
use swarm::muxing::StreamMuxer;
use swarm::{ConnectionUpgrade, Endpoint, Multiaddr};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use write::write_stream; use write::write_stream;

View File

@ -18,14 +18,14 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use {bytes, varint}; use circular_buffer::Array;
use futures::Async; use futures::Async;
use futures::task; use futures::task;
use header::{MultiplexHeader, PacketType}; use header::{MultiplexHeader, PacketType};
use shared::SubstreamMetadata;
use std::io; use std::io;
use tokio_io::AsyncRead; use tokio_io::AsyncRead;
use shared::SubstreamMetadata; use {bytes, varint};
use circular_buffer::Array;
pub enum NextMultiplexState { pub enum NextMultiplexState {
NewStream(u32), NewStream(u32),

View File

@ -21,11 +21,11 @@
use read::MultiplexReadState; use read::MultiplexReadState;
use write::MultiplexWriteState; use write::MultiplexWriteState;
use circular_buffer::{Array, CircularBuffer};
use std::collections::HashMap;
use bytes::Bytes;
use arrayvec::ArrayVec; use arrayvec::ArrayVec;
use bytes::Bytes;
use circular_buffer::{Array, CircularBuffer};
use futures::task::Task; use futures::task::Task;
use std::collections::HashMap;
const BUF_SIZE: usize = 1024; const BUF_SIZE: usize = 1024;

View File

@ -18,14 +18,14 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use shared::{ByteBuf, MultiplexShared, SubstreamMetadata};
use header::MultiplexHeader; use header::MultiplexHeader;
use shared::{ByteBuf, MultiplexShared, SubstreamMetadata};
use circular_buffer; use circular_buffer;
use varint;
use futures::task; use futures::task;
use std::io; use std::io;
use tokio_io::AsyncWrite; use tokio_io::AsyncWrite;
use varint;
#[derive(Copy, Clone, PartialEq, Eq, Debug)] #[derive(Copy, Clone, PartialEq, Eq, Debug)]
pub enum RequestType { pub enum RequestType {

View File

@ -23,8 +23,8 @@
use ProtocolChoiceError; use ProtocolChoiceError;
use bytes::Bytes; use bytes::Bytes;
use futures::{Future, Sink, Stream};
use futures::future::{loop_fn, result, Loop}; use futures::future::{loop_fn, result, Loop};
use futures::{Future, Sink, Stream};
use protocol::Dialer; use protocol::Dialer;
use protocol::DialerToListenerMessage; use protocol::DialerToListenerMessage;

View File

@ -26,10 +26,10 @@
//! We purposely only support a frame length of under 64kiB. Frames most consist in a short //! We purposely only support a frame length of under 64kiB. Frames most consist in a short
//! protocol name, which is highly unlikely to be more than 64kiB long. //! protocol name, which is highly unlikely to be more than 64kiB long.
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::marker::PhantomData;
use futures::{Async, Poll, Sink, StartSend, Stream}; use futures::{Async, Poll, Sink, StartSend, Stream};
use smallvec::SmallVec; use smallvec::SmallVec;
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::marker::PhantomData;
use tokio_io::AsyncRead; use tokio_io::AsyncRead;
/// Wraps around a `AsyncRead` and implements `Stream`. /// Wraps around a `AsyncRead` and implements `Stream`.
@ -232,10 +232,10 @@ fn decode_length_prefix(buf: &[u8]) -> u16 {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use std::io::Cursor;
use std::io::ErrorKind;
use futures::{Future, Stream}; use futures::{Future, Stream};
use length_delimited::LengthDelimitedFramedRead; use length_delimited::LengthDelimitedFramedRead;
use std::io::Cursor;
use std::io::ErrorKind;
#[test] #[test]
fn basic_read() { fn basic_read() {

View File

@ -23,8 +23,8 @@
use ProtocolChoiceError; use ProtocolChoiceError;
use bytes::Bytes; use bytes::Bytes;
use futures::{Future, Sink, Stream};
use futures::future::{err, loop_fn, Loop}; use futures::future::{err, loop_fn, Loop};
use futures::{Future, Sink, Stream};
use protocol::DialerToListenerMessage; use protocol::DialerToListenerMessage;
use protocol::Listener; use protocol::Listener;

View File

@ -28,9 +28,9 @@ use protocol::ListenerToDialerMessage;
use protocol::MULTISTREAM_PROTOCOL_WITH_LF; use protocol::MULTISTREAM_PROTOCOL_WITH_LF;
use protocol::MultistreamSelectError; use protocol::MultistreamSelectError;
use std::io::{BufRead, Cursor, Read}; use std::io::{BufRead, Cursor, Read};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_io::codec::length_delimited::Builder as LengthDelimitedBuilder; use tokio_io::codec::length_delimited::Builder as LengthDelimitedBuilder;
use tokio_io::codec::length_delimited::FramedWrite as LengthDelimitedFramedWrite; use tokio_io::codec::length_delimited::FramedWrite as LengthDelimitedFramedWrite;
use tokio_io::{AsyncRead, AsyncWrite};
use varint; use varint;
/// Wraps around a `AsyncRead+AsyncWrite`. Assumes that we're on the dialer's side. Produces and /// Wraps around a `AsyncRead+AsyncWrite`. Assumes that we're on the dialer's side. Produces and
@ -191,12 +191,12 @@ where
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
extern crate tokio_core; extern crate tokio_core;
use bytes::Bytes;
use futures::{Sink, Stream};
use futures::Future;
use protocol::{Dialer, DialerToListenerMessage, MultistreamSelectError};
use self::tokio_core::net::{TcpListener, TcpStream}; use self::tokio_core::net::{TcpListener, TcpStream};
use self::tokio_core::reactor::Core; use self::tokio_core::reactor::Core;
use bytes::Bytes;
use futures::Future;
use futures::{Sink, Stream};
use protocol::{Dialer, DialerToListenerMessage, MultistreamSelectError};
#[test] #[test]
fn wrong_proto_name() { fn wrong_proto_name() {

View File

@ -27,9 +27,9 @@ use protocol::DialerToListenerMessage;
use protocol::ListenerToDialerMessage; use protocol::ListenerToDialerMessage;
use protocol::MULTISTREAM_PROTOCOL_WITH_LF; use protocol::MULTISTREAM_PROTOCOL_WITH_LF;
use protocol::MultistreamSelectError; use protocol::MultistreamSelectError;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_io::codec::length_delimited::Builder as LengthDelimitedBuilder; use tokio_io::codec::length_delimited::Builder as LengthDelimitedBuilder;
use tokio_io::codec::length_delimited::FramedWrite as LengthDelimitedFramedWrite; use tokio_io::codec::length_delimited::FramedWrite as LengthDelimitedFramedWrite;
use tokio_io::{AsyncRead, AsyncWrite};
use varint; use varint;
/// Wraps around a `AsyncRead+AsyncWrite`. Assumes that we're on the listener's side. Produces and /// Wraps around a `AsyncRead+AsyncWrite`. Assumes that we're on the listener's side. Produces and
@ -186,12 +186,12 @@ where
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
extern crate tokio_core; extern crate tokio_core;
use bytes::Bytes;
use futures::{Sink, Stream};
use futures::Future;
use protocol::{Dialer, Listener, ListenerToDialerMessage, MultistreamSelectError};
use self::tokio_core::net::{TcpListener, TcpStream}; use self::tokio_core::net::{TcpListener, TcpStream};
use self::tokio_core::reactor::Core; use self::tokio_core::reactor::Core;
use bytes::Bytes;
use futures::Future;
use futures::{Sink, Stream};
use protocol::{Dialer, Listener, ListenerToDialerMessage, MultistreamSelectError};
#[test] #[test]
fn wrong_proto_name() { fn wrong_proto_name() {

View File

@ -24,16 +24,16 @@
extern crate tokio_core; extern crate tokio_core;
use {dialer_select_proto, listener_select_proto};
use ProtocolChoiceError;
use bytes::Bytes;
use dialer_select::{dialer_select_proto_parallel, dialer_select_proto_serial};
use futures::{Sink, Stream};
use futures::Future;
use protocol::{Dialer, DialerToListenerMessage, Listener, ListenerToDialerMessage};
use self::tokio_core::net::TcpListener; use self::tokio_core::net::TcpListener;
use self::tokio_core::net::TcpStream; use self::tokio_core::net::TcpStream;
use self::tokio_core::reactor::Core; use self::tokio_core::reactor::Core;
use ProtocolChoiceError;
use bytes::Bytes;
use dialer_select::{dialer_select_proto_parallel, dialer_select_proto_serial};
use futures::Future;
use futures::{Sink, Stream};
use protocol::{Dialer, DialerToListenerMessage, Listener, ListenerToDialerMessage};
use {dialer_select_proto, listener_select_proto};
#[test] #[test]
fn negotiate_with_self_succeeds() { fn negotiate_with_self_succeeds() {

View File

@ -22,6 +22,7 @@
use super::TTL; use super::TTL;
use PeerId; use PeerId;
use bs58;
use datastore::{Datastore, JsonFileDatastore, JsonFileDatastoreEntry, Query}; use datastore::{Datastore, JsonFileDatastore, JsonFileDatastoreEntry, Query};
use futures::{Future, Stream}; use futures::{Future, Stream};
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
@ -31,7 +32,6 @@ use std::io::Error as IoError;
use std::iter; use std::iter;
use std::path::PathBuf; use std::path::PathBuf;
use std::vec::IntoIter as VecIntoIter; use std::vec::IntoIter as VecIntoIter;
use bs58;
/// Peerstore backend that uses a Json file. /// Peerstore backend that uses a Json file.
pub struct JsonPeerstore { pub struct JsonPeerstore {

View File

@ -28,9 +28,9 @@
use TTL; use TTL;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use serde::{Deserialize, Deserializer, Serialize, Serializer};
use serde::de::Error as DeserializerError; use serde::de::Error as DeserializerError;
use serde::ser::SerializeStruct; use serde::ser::SerializeStruct;
use serde::{Deserialize, Deserializer, Serialize, Serializer};
use std::cmp::Ordering; use std::cmp::Ordering;
use std::time::{Duration, SystemTime, UNIX_EPOCH}; use std::time::{Duration, SystemTime, UNIX_EPOCH};

View File

@ -18,8 +18,8 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use {PeerId, TTL};
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use {PeerId, TTL};
/// Implemented on objects that store peers. /// Implemented on objects that store peers.
/// ///

View File

@ -89,9 +89,9 @@ extern crate rand;
extern crate tokio_io; extern crate tokio_io;
use bytes::{BufMut, Bytes, BytesMut}; use bytes::{BufMut, Bytes, BytesMut};
use futures::{Future, Sink, Stream};
use futures::future::{loop_fn, FutureResult, IntoFuture, Loop}; use futures::future::{loop_fn, FutureResult, IntoFuture, Loop};
use futures::sync::{mpsc, oneshot}; use futures::sync::{mpsc, oneshot};
use futures::{Future, Sink, Stream};
use libp2p_swarm::{ConnectionUpgrade, Endpoint, Multiaddr}; use libp2p_swarm::{ConnectionUpgrade, Endpoint, Multiaddr};
use log::Level; use log::Level;
use parking_lot::Mutex; use parking_lot::Mutex;
@ -102,8 +102,8 @@ use std::error::Error;
use std::io::Error as IoError; use std::io::Error as IoError;
use std::iter; use std::iter;
use std::sync::Arc; use std::sync::Arc;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_io::codec::{Decoder, Encoder}; use tokio_io::codec::{Decoder, Encoder};
use tokio_io::{AsyncRead, AsyncWrite};
/// Represents a prototype for an upgrade to handle the ping protocol. /// Represents a prototype for an upgrade to handle the ping protocol.
/// ///
@ -306,9 +306,9 @@ mod tests {
use self::tokio_core::net::TcpStream; use self::tokio_core::net::TcpStream;
use self::tokio_core::reactor::Core; use self::tokio_core::reactor::Core;
use super::Ping; use super::Ping;
use futures::future::join_all;
use futures::Future; use futures::Future;
use futures::Stream; use futures::Stream;
use futures::future::join_all;
use libp2p_swarm::{ConnectionUpgrade, Endpoint}; use libp2p_swarm::{ConnectionUpgrade, Endpoint};
#[test] #[test]

View File

@ -140,8 +140,9 @@ pub struct ListenerUpgrade<T: Transport>(RateLimited<T::ListenerUpgrade>);
impl<T> Future for ListenerUpgrade<T> impl<T> Future for ListenerUpgrade<T>
where where
T: Transport + 'static, T: Transport + 'static,
T::Output: AsyncRead + AsyncWrite,
{ {
type Item = (Connection<T::RawConn>, Multiaddr); type Item = (Connection<T::Output>, Multiaddr);
type Error = io::Error; type Error = io::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> { fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
@ -157,9 +158,10 @@ pub struct Dial<T: Transport>(RateLimited<T::Dial>);
impl<T> IntoFuture for Dial<T> impl<T> IntoFuture for Dial<T>
where where
T: Transport + 'static, T: Transport + 'static,
T::Output: AsyncRead + AsyncWrite,
{ {
type Future = Box<Future<Item = Self::Item, Error = Self::Error>>; type Future = Box<Future<Item = Self::Item, Error = Self::Error>>;
type Item = (Connection<T::RawConn>, Multiaddr); type Item = (Connection<T::Output>, Multiaddr);
type Error = io::Error; type Error = io::Error;
fn into_future(self) -> Self::Future { fn into_future(self) -> Self::Future {
@ -176,8 +178,9 @@ where
impl<T> Transport for RateLimited<T> impl<T> Transport for RateLimited<T>
where where
T: Transport + 'static, T: Transport + 'static,
T::Output: AsyncRead + AsyncWrite,
{ {
type RawConn = Connection<T::RawConn>; type Output = Connection<T::Output>;
type Listener = Listener<T>; type Listener = Listener<T>;
type ListenerUpgrade = ListenerUpgrade<T>; type ListenerUpgrade = ListenerUpgrade<T>;
type Dial = Dial<T>; type Dial = Dial<T>;

View File

@ -34,12 +34,12 @@ extern crate bytes;
extern crate futures; extern crate futures;
extern crate tokio_io; extern crate tokio_io;
use bytes::{Buf, IntoBuf};
use futures::{Async, AsyncSink, Poll, Sink, Stream};
use std::cmp; use std::cmp;
use std::io::Error as IoError; use std::io::Error as IoError;
use std::io::ErrorKind as IoErrorKind; use std::io::ErrorKind as IoErrorKind;
use std::io::{Read, Write}; use std::io::{Read, Write};
use bytes::{Buf, IntoBuf};
use futures::{Async, AsyncSink, Poll, Sink, Stream};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
/// Wraps around a `Stream + Sink` whose items are buffers. Implements `AsyncRead` and `AsyncWrite`. /// Wraps around a `Stream + Sink` whose items are buffers. Implements `AsyncRead` and `AsyncWrite`.
@ -164,11 +164,11 @@ where
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use bytes::Bytes;
use futures::{Future, Poll, Sink, StartSend, Stream};
use futures::sync::mpsc::channel;
use std::io::Read;
use RwStreamSink; use RwStreamSink;
use bytes::Bytes;
use futures::sync::mpsc::channel;
use futures::{Future, Poll, Sink, StartSend, Stream};
use std::io::Read;
// This struct merges a stream and a sink and is quite useful for tests. // This struct merges a stream and a sink and is quite useful for tests.
struct Wrapper<St, Si>(St, Si); struct Wrapper<St, Si>(St, Si);

View File

@ -26,8 +26,8 @@ use self::encode::EncoderMiddleware;
use crypto::symmetriccipher::SynchronousStreamCipher; use crypto::symmetriccipher::SynchronousStreamCipher;
use ring::hmac; use ring::hmac;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_io::codec::length_delimited; use tokio_io::codec::length_delimited;
use tokio_io::{AsyncRead, AsyncWrite};
mod decode; mod decode;
mod encode; mod encode;
@ -59,6 +59,9 @@ where
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
extern crate tokio_core; extern crate tokio_core;
use self::tokio_core::net::TcpListener;
use self::tokio_core::net::TcpStream;
use self::tokio_core::reactor::Core;
use super::DecoderMiddleware; use super::DecoderMiddleware;
use super::EncoderMiddleware; use super::EncoderMiddleware;
use super::full_codec; use super::full_codec;
@ -66,16 +69,13 @@ mod tests {
use crypto::aessafe::AesSafe256Encryptor; use crypto::aessafe::AesSafe256Encryptor;
use crypto::blockmodes::CtrMode; use crypto::blockmodes::CtrMode;
use error::SecioError; use error::SecioError;
use futures::{Future, Sink, Stream};
use futures::sync::mpsc::channel; use futures::sync::mpsc::channel;
use futures::{Future, Sink, Stream};
use rand; use rand;
use ring::digest::SHA256; use ring::digest::SHA256;
use ring::hmac::SigningKey; use ring::hmac::SigningKey;
use ring::hmac::VerificationKey; use ring::hmac::VerificationKey;
use std::io::Error as IoError; use std::io::Error as IoError;
use self::tokio_core::net::TcpListener;
use self::tokio_core::net::TcpStream;
use self::tokio_core::reactor::Core;
use tokio_io::codec::length_delimited::Framed; use tokio_io::codec::length_delimited::Framed;
#[test] #[test]

View File

@ -30,19 +30,19 @@ use futures::stream::Stream;
use keys_proto::{KeyType as KeyTypeProtobuf, PublicKey as PublicKeyProtobuf}; use keys_proto::{KeyType as KeyTypeProtobuf, PublicKey as PublicKeyProtobuf};
use protobuf::Message as ProtobufMessage; use protobuf::Message as ProtobufMessage;
use protobuf::core::parse_from_bytes as protobuf_parse_from_bytes; use protobuf::core::parse_from_bytes as protobuf_parse_from_bytes;
use ring::{agreement, digest, rand};
use ring::agreement::EphemeralPrivateKey; use ring::agreement::EphemeralPrivateKey;
use ring::hmac::{SigningContext, SigningKey, VerificationKey}; use ring::hmac::{SigningContext, SigningKey, VerificationKey};
use ring::rand::SecureRandom; use ring::rand::SecureRandom;
use ring::signature::{RSAKeyPair, RSASigningState, RSA_PKCS1_2048_8192_SHA256, RSA_PKCS1_SHA256};
use ring::signature::verify as signature_verify; use ring::signature::verify as signature_verify;
use ring::signature::{RSAKeyPair, RSASigningState, RSA_PKCS1_2048_8192_SHA256, RSA_PKCS1_SHA256};
use ring::{agreement, digest, rand};
use std::cmp::{self, Ordering}; use std::cmp::{self, Ordering};
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::mem; use std::mem;
use std::sync::Arc; use std::sync::Arc;
use structs_proto::{Exchange, Propose}; use structs_proto::{Exchange, Propose};
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_io::codec::length_delimited; use tokio_io::codec::length_delimited;
use tokio_io::{AsyncRead, AsyncWrite};
use untrusted::Input as UntrustedInput; use untrusted::Input as UntrustedInput;
/// Performs a handshake on the given socket. /// Performs a handshake on the given socket.
@ -526,6 +526,9 @@ fn stretch_key(key: &SigningKey, result: &mut [u8]) {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
extern crate tokio_core; extern crate tokio_core;
use self::tokio_core::net::TcpListener;
use self::tokio_core::net::TcpStream;
use self::tokio_core::reactor::Core;
use super::handshake; use super::handshake;
use super::stretch_key; use super::stretch_key;
use futures::Future; use futures::Future;
@ -534,9 +537,6 @@ mod tests {
use ring::hmac::SigningKey; use ring::hmac::SigningKey;
use ring::signature::RSAKeyPair; use ring::signature::RSAKeyPair;
use std::sync::Arc; use std::sync::Arc;
use self::tokio_core::net::TcpListener;
use self::tokio_core::net::TcpStream;
use self::tokio_core::reactor::Core;
use untrusted::Input; use untrusted::Input;
#[test] #[test]

View File

@ -39,7 +39,7 @@
//! # fn main() { //! # fn main() {
//! use futures::Future; //! use futures::Future;
//! use libp2p_secio::{SecioConfig, SecioKeyPair}; //! use libp2p_secio::{SecioConfig, SecioKeyPair};
//! use libp2p_swarm::{Multiaddr, Transport}; //! use libp2p_swarm::{Multiaddr, Transport, upgrade};
//! use libp2p_tcp_transport::TcpConfig; //! use libp2p_tcp_transport::TcpConfig;
//! use tokio_core::reactor::Core; //! use tokio_core::reactor::Core;
//! use tokio_io::io::write_all; //! use tokio_io::io::write_all;
@ -52,10 +52,12 @@
//! //let private_key = include_bytes!("test-private-key.pk8"); //! //let private_key = include_bytes!("test-private-key.pk8");
//! # let public_key = vec![]; //! # let public_key = vec![];
//! //let public_key = include_bytes!("test-public-key.der").to_vec(); //! //let public_key = include_bytes!("test-public-key.der").to_vec();
//! SecioConfig { //! let upgrade = SecioConfig {
//! // See the documentation of `SecioKeyPair`. //! // See the documentation of `SecioKeyPair`.
//! key: SecioKeyPair::rsa_from_pkcs8(private_key, public_key).unwrap(), //! key: SecioKeyPair::rsa_from_pkcs8(private_key, public_key).unwrap(),
//! } //! };
//!
//! upgrade::map(upgrade, |(socket, _remote_key)| socket)
//! }); //! });
//! //!
//! let future = transport.dial("/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap()) //! let future = transport.dial("/ip4/127.0.0.1/tcp/12345".parse::<Multiaddr>().unwrap())
@ -95,8 +97,8 @@ extern crate untrusted;
pub use self::error::SecioError; pub use self::error::SecioError;
use bytes::{Bytes, BytesMut}; use bytes::{Bytes, BytesMut};
use futures::{Future, Poll, Sink, StartSend, Stream};
use futures::stream::MapErr as StreamMapErr; use futures::stream::MapErr as StreamMapErr;
use futures::{Future, Poll, Sink, StartSend, Stream};
use libp2p_swarm::Multiaddr; use libp2p_swarm::Multiaddr;
use ring::signature::RSAKeyPair; use ring::signature::RSAKeyPair;
use rw_stream_sink::RwStreamSink; use rw_stream_sink::RwStreamSink;
@ -110,8 +112,8 @@ use untrusted::Input;
mod algo_support; mod algo_support;
mod codec; mod codec;
mod error; mod error;
mod keys_proto;
mod handshake; mod handshake;
mod keys_proto;
mod structs_proto; mod structs_proto;
/// Implementation of the `ConnectionUpgrade` trait of `libp2p_swarm`. Automatically applies /// Implementation of the `ConnectionUpgrade` trait of `libp2p_swarm`. Automatically applies
@ -179,16 +181,19 @@ enum SecioKeyPairInner {
} }
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum SecioPublicKey<'a> { pub enum SecioPublicKey {
/// DER format. /// DER format.
Rsa(&'a [u8]), Rsa(Vec<u8>),
} }
impl<S> libp2p_swarm::ConnectionUpgrade<S> for SecioConfig impl<S> libp2p_swarm::ConnectionUpgrade<S> for SecioConfig
where where
S: AsyncRead + AsyncWrite + 'static, S: AsyncRead + AsyncWrite + 'static,
{ {
type Output = RwStreamSink<StreamMapErr<SecioMiddleware<S>, fn(SecioError) -> IoError>>; type Output = (
RwStreamSink<StreamMapErr<SecioMiddleware<S>, fn(SecioError) -> IoError>>,
SecioPublicKey,
);
type Future = Box<Future<Item = Self::Output, Error = IoError>>; type Future = Box<Future<Item = Self::Output, Error = IoError>>;
type NamesIter = iter::Once<(Bytes, ())>; type NamesIter = iter::Once<(Bytes, ())>;
type UpgradeIdentifier = (); type UpgradeIdentifier = ();
@ -209,9 +214,9 @@ where
info!(target: "libp2p-secio", "starting secio upgrade with {:?}", remote_addr); info!(target: "libp2p-secio", "starting secio upgrade with {:?}", remote_addr);
let fut = SecioMiddleware::handshake(incoming, self.key); let fut = SecioMiddleware::handshake(incoming, self.key);
let wrapped = fut.map(|stream_sink| { let wrapped = fut.map(|(stream_sink, pubkey)| {
let mapped = stream_sink.map_err(map_err as fn(_) -> _); let mapped = stream_sink.map_err(map_err as fn(_) -> _);
RwStreamSink::new(mapped) (RwStreamSink::new(mapped), pubkey)
}).map_err(map_err); }).map_err(map_err);
Box::new(wrapped) Box::new(wrapped)
} }
@ -229,7 +234,6 @@ fn map_err(err: SecioError) -> IoError {
/// individually, so you are encouraged to group data in few frames if possible. /// individually, so you are encouraged to group data in few frames if possible.
pub struct SecioMiddleware<S> { pub struct SecioMiddleware<S> {
inner: codec::FullCodec<S>, inner: codec::FullCodec<S>,
remote_pubkey_der: Vec<u8>,
} }
impl<S> SecioMiddleware<S> impl<S> SecioMiddleware<S>
@ -243,25 +247,18 @@ where
pub fn handshake<'a>( pub fn handshake<'a>(
socket: S, socket: S,
key_pair: SecioKeyPair, key_pair: SecioKeyPair,
) -> Box<Future<Item = SecioMiddleware<S>, Error = SecioError> + 'a> ) -> Box<Future<Item = (SecioMiddleware<S>, SecioPublicKey), Error = SecioError> + 'a>
where where
S: 'a, S: 'a,
{ {
let SecioKeyPairInner::Rsa { private, public } = key_pair.inner; let SecioKeyPairInner::Rsa { private, public } = key_pair.inner;
let fut = let fut = handshake::handshake(socket, public, private).map(|(inner, pubkey)| {
handshake::handshake(socket, public, private).map(|(inner, pubkey)| SecioMiddleware { let inner = SecioMiddleware { inner };
inner: inner, (inner, SecioPublicKey::Rsa(pubkey))
remote_pubkey_der: pubkey, });
});
Box::new(fut) Box::new(fut)
} }
/// Returns the public key of the remote in the `DER` format.
#[inline]
pub fn remote_public_key_der(&self) -> SecioPublicKey {
SecioPublicKey::Rsa(&self.remote_pubkey_der)
}
} }
impl<S> Sink for SecioMiddleware<S> impl<S> Sink for SecioMiddleware<S>

View File

@ -42,14 +42,15 @@
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::future::{self, Either, FutureResult, IntoFuture}; use futures::future::{self, Either, FutureResult, IntoFuture};
use futures::{Async, Future, Poll, Stream}; use futures::{Async, Future, Poll, Stream};
use futures::stream::FuturesUnordered;
use futures::stream::Fuse as StreamFuse; use futures::stream::Fuse as StreamFuse;
use futures::stream::FuturesUnordered;
use futures::sync::mpsc; use futures::sync::mpsc;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use muxing::StreamMuxer; use muxing::StreamMuxer;
use parking_lot::Mutex; use parking_lot::Mutex;
use std::io::{self, Error as IoError}; use std::io::{self, Error as IoError};
use std::sync::Arc; use std::sync::Arc;
use tokio_io::{AsyncRead, AsyncWrite};
use transport::{MuxedTransport, Transport, UpgradedNode}; use transport::{MuxedTransport, Transport, UpgradedNode};
use upgrade::ConnectionUpgrade; use upgrade::ConnectionUpgrade;
@ -62,7 +63,8 @@ use upgrade::ConnectionUpgrade;
pub struct ConnectionReuse<T, C> pub struct ConnectionReuse<T, C>
where where
T: Transport, T: Transport,
C: ConnectionUpgrade<T::RawConn>, T::Output: AsyncRead + AsyncWrite,
C: ConnectionUpgrade<T::Output>,
C::Output: StreamMuxer, C::Output: StreamMuxer,
{ {
// Underlying transport and connection upgrade for when we need to dial or listen. // Underlying transport and connection upgrade for when we need to dial or listen.
@ -94,7 +96,8 @@ where
impl<T, C> From<UpgradedNode<T, C>> for ConnectionReuse<T, C> impl<T, C> From<UpgradedNode<T, C>> for ConnectionReuse<T, C>
where where
T: Transport, T: Transport,
C: ConnectionUpgrade<T::RawConn>, T::Output: AsyncRead + AsyncWrite,
C: ConnectionUpgrade<T::Output>,
C::Output: StreamMuxer, C::Output: StreamMuxer,
{ {
#[inline] #[inline]
@ -115,16 +118,17 @@ where
impl<T, C> Transport for ConnectionReuse<T, C> impl<T, C> Transport for ConnectionReuse<T, C>
where where
T: Transport + 'static, // TODO: 'static :( T: Transport + 'static, // TODO: 'static :(
C: ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :( T::Output: AsyncRead + AsyncWrite,
C: ConnectionUpgrade<T::Output> + 'static, // TODO: 'static :(
C: Clone, C: Clone,
C::Output: StreamMuxer + Clone, C::Output: StreamMuxer + Clone,
C::NamesIter: Clone, // TODO: not elegant C::NamesIter: Clone, // TODO: not elegant
{ {
type RawConn = <C::Output as StreamMuxer>::Substream; type Output = <C::Output as StreamMuxer>::Substream;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>; type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
type ListenerUpgrade = FutureResult<(Self::RawConn, Multiaddr), IoError>; type ListenerUpgrade = FutureResult<(Self::Output, Multiaddr), IoError>;
type Dial = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
let (listener, new_addr) = match self.inner.listen_on(addr.clone()) { let (listener, new_addr) = match self.inner.listen_on(addr.clone()) {
@ -215,8 +219,9 @@ where
impl<T, C> MuxedTransport for ConnectionReuse<T, C> impl<T, C> MuxedTransport for ConnectionReuse<T, C>
where where
T: Transport + 'static, // TODO: 'static :( T: Transport + 'static, // TODO: 'static :(
C: ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :( T::Output: AsyncRead + AsyncWrite,
C: ConnectionUpgrade<T::Output> + 'static, // TODO: 'static :(
C: Clone, C: Clone,
C::Output: StreamMuxer + Clone, C::Output: StreamMuxer + Clone,
C::NamesIter: Clone, // TODO: not elegant C::NamesIter: Clone, // TODO: not elegant

View File

@ -188,11 +188,12 @@
//! let transport = libp2p_tcp_transport::TcpConfig::new(core.handle()) //! let transport = libp2p_tcp_transport::TcpConfig::new(core.handle())
//! .with_dummy_muxing(); //! .with_dummy_muxing();
//! //!
//! let (swarm_controller, swarm_future) = libp2p_swarm::swarm(transport, Ping, |(mut pinger, service), client_addr| { //! let (swarm_controller, swarm_future) = libp2p_swarm::swarm(transport.with_upgrade(Ping),
//! pinger.ping().map_err(|_| panic!()) //! |(mut pinger, service), client_addr| {
//! .select(service).map_err(|_| panic!()) //! pinger.ping().map_err(|_| panic!())
//! .map(|_| ()) //! .select(service).map_err(|_| panic!())
//! }); //! .map(|_| ())
//! });
//! //!
//! // The `swarm_controller` can then be used to do some operations. //! // The `swarm_controller` can then be used to do some operations.
//! swarm_controller.listen_on("/ip4/0.0.0.0/tcp/0".parse().unwrap()); //! swarm_controller.listen_on("/ip4/0.0.0.0/tcp/0".parse().unwrap());

View File

@ -18,13 +18,12 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use std::fmt;
use std::io::Error as IoError;
use futures::{future, Async, Future, IntoFuture, Poll, Stream};
use futures::stream::{FuturesUnordered, StreamFuture}; use futures::stream::{FuturesUnordered, StreamFuture};
use futures::sync::mpsc; use futures::sync::mpsc;
use transport::UpgradedNode; use futures::{future, Async, Future, IntoFuture, Poll, Stream};
use {ConnectionUpgrade, Multiaddr, MuxedTransport}; use std::fmt;
use std::io::Error as IoError;
use {Multiaddr, MuxedTransport, Transport};
/// Creates a swarm. /// Creates a swarm.
/// ///
@ -34,29 +33,24 @@ use {ConnectionUpgrade, Multiaddr, MuxedTransport};
/// Produces a `SwarmController` and an implementation of `Future`. The controller can be used to /// 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. /// control, and the `Future` must be driven to completion in order for things to work.
/// ///
pub fn swarm<T, C, H, F>( pub fn swarm<T, H, F>(
transport: T, transport: T,
upgrade: C,
handler: H, handler: H,
) -> (SwarmController<T, C>, SwarmFuture<T, C, H, F::Future>) ) -> (SwarmController<T>, SwarmFuture<T, H, F::Future>)
where where
T: MuxedTransport + Clone + 'static, // TODO: 'static :-/ T: MuxedTransport + Clone + 'static, // TODO: 'static :-/
C: ConnectionUpgrade<T::RawConn> + Clone + 'static, // TODO: 'static :-/ H: FnMut(T::Output, Multiaddr) -> F,
C::NamesIter: Clone, // TODO: not elegant
H: FnMut(C::Output, Multiaddr) -> F,
F: IntoFuture<Item = (), Error = IoError>, F: IntoFuture<Item = (), Error = IoError>,
{ {
let (new_dialers_tx, new_dialers_rx) = mpsc::unbounded(); let (new_dialers_tx, new_dialers_rx) = mpsc::unbounded();
let (new_listeners_tx, new_listeners_rx) = mpsc::unbounded(); let (new_listeners_tx, new_listeners_rx) = mpsc::unbounded();
let (new_toprocess_tx, new_toprocess_rx) = mpsc::unbounded(); let (new_toprocess_tx, new_toprocess_rx) = mpsc::unbounded();
let upgraded = transport.clone().with_upgrade(upgrade);
let future = SwarmFuture { let future = SwarmFuture {
upgraded: upgraded.clone(), transport: transport.clone(),
handler: handler, handler: handler,
new_listeners: new_listeners_rx, new_listeners: new_listeners_rx,
next_incoming: upgraded.clone().next_incoming(), next_incoming: transport.clone().next_incoming(),
listeners: FuturesUnordered::new(), listeners: FuturesUnordered::new(),
listeners_upgrade: FuturesUnordered::new(), listeners_upgrade: FuturesUnordered::new(),
dialers: FuturesUnordered::new(), dialers: FuturesUnordered::new(),
@ -67,7 +61,6 @@ where
let controller = SwarmController { let controller = SwarmController {
transport: transport, transport: transport,
upgraded: upgraded,
new_listeners: new_listeners_tx, new_listeners: new_listeners_tx,
new_dialers: new_dialers_tx, new_dialers: new_dialers_tx,
new_toprocess: new_toprocess_tx, new_toprocess: new_toprocess_tx,
@ -77,46 +70,34 @@ where
} }
/// Allows control of what the swarm is doing. /// Allows control of what the swarm is doing.
pub struct SwarmController<T, C> pub struct SwarmController<T>
where where
T: MuxedTransport + 'static, // TODO: 'static :-/ T: MuxedTransport + 'static, // TODO: 'static :-/
C: ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/
{ {
transport: T, transport: T,
upgraded: UpgradedNode<T, C>, new_listeners: mpsc::UnboundedSender<T::Listener>,
new_listeners: mpsc::UnboundedSender< new_dialers: mpsc::UnboundedSender<Box<Future<Item = (T::Output, Multiaddr), Error = IoError>>>,
Box<
Stream<
Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>,
Error = IoError,
>,
>,
>,
new_dialers: mpsc::UnboundedSender<Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>>,
new_toprocess: mpsc::UnboundedSender<Box<Future<Item = (), Error = IoError>>>, new_toprocess: mpsc::UnboundedSender<Box<Future<Item = (), Error = IoError>>>,
} }
impl<T, C> fmt::Debug for SwarmController<T, C> impl<T> fmt::Debug for SwarmController<T>
where where
T: fmt::Debug + MuxedTransport + 'static, // TODO: 'static :-/ T: fmt::Debug + MuxedTransport + 'static, // TODO: 'static :-/
C: fmt::Debug + ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/
{ {
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result { fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
fmt.debug_tuple("SwarmController") fmt.debug_tuple("SwarmController")
.field(&self.upgraded) .field(&self.transport)
.finish() .finish()
} }
} }
impl<T, C> Clone for SwarmController<T, C> impl<T> Clone for SwarmController<T>
where where
T: MuxedTransport + Clone + 'static, // TODO: 'static :-/ T: MuxedTransport + Clone + 'static, // TODO: 'static :-/
C: ConnectionUpgrade<T::RawConn> + 'static + Clone, // TODO: 'static :-/
{ {
fn clone(&self) -> SwarmController<T, C> { fn clone(&self) -> SwarmController<T> {
SwarmController { SwarmController {
transport: self.transport.clone(), transport: self.transport.clone(),
upgraded: self.upgraded.clone(),
new_listeners: self.new_listeners.clone(), new_listeners: self.new_listeners.clone(),
new_dialers: self.new_dialers.clone(), new_dialers: self.new_dialers.clone(),
new_toprocess: self.new_toprocess.clone(), new_toprocess: self.new_toprocess.clone(),
@ -124,31 +105,27 @@ where
} }
} }
impl<T, C> SwarmController<T, C> impl<T> SwarmController<T>
where where
T: MuxedTransport + Clone + 'static, // TODO: 'static :-/ T: MuxedTransport + Clone + 'static, // TODO: 'static :-/
C: ConnectionUpgrade<T::RawConn> + Clone + 'static, // TODO: 'static :-/
C::NamesIter: Clone, // TODO: not elegant
{ {
/// Asks the swarm to dial the node with the given multiaddress. The connection is then /// Asks the swarm to dial the node with the given multiaddress. The connection is then
/// upgraded using the `upgrade`, and the output is sent to the handler that was passed when /// upgraded using the `upgrade`, and the output is sent to the handler that was passed when
/// calling `swarm`. /// calling `swarm`.
// TODO: consider returning a future so that errors can be processed? // TODO: consider returning a future so that errors can be processed?
pub fn dial_to_handler<Du>(&self, multiaddr: Multiaddr, upgrade: Du) -> Result<(), Multiaddr> pub fn dial_to_handler<Du>(&self, multiaddr: Multiaddr, transport: Du) -> Result<(), Multiaddr>
where where
Du: ConnectionUpgrade<T::RawConn> + Clone + 'static, // TODO: 'static :-/ Du: Transport + 'static, // TODO: 'static :-/
Du::Output: Into<C::Output>, Du::Output: Into<T::Output>,
{ {
trace!(target: "libp2p-swarm", "Swarm dialing {}", multiaddr); trace!(target: "libp2p-swarm", "Swarm dialing {}", multiaddr);
match self.transport match transport.dial(multiaddr.clone()) {
.clone()
.with_upgrade(upgrade)
.dial(multiaddr.clone())
{
Ok(dial) => { Ok(dial) => {
let dial = Box::new(dial.map(|(d, client_addr)| (d.into(), client_addr))) let dial = Box::new(
as Box<Future<Item = _, Error = _>>; dial.into_future()
.map(|(d, client_addr)| (d.into(), client_addr)),
) as Box<Future<Item = _, Error = _>>;
// Ignoring errors if the receiver has been closed, because in that situation // Ignoring errors if the receiver has been closed, because in that situation
// nothing is going to be processed anyway. // nothing is going to be processed anyway.
let _ = self.new_dialers.unbounded_send(dial); let _ = self.new_dialers.unbounded_send(dial);
@ -167,19 +144,19 @@ where
pub fn dial_custom_handler<Du, Df, Dfu>( pub fn dial_custom_handler<Du, Df, Dfu>(
&self, &self,
multiaddr: Multiaddr, multiaddr: Multiaddr,
upgrade: Du, transport: Du,
and_then: Df, and_then: Df,
) -> Result<(), Multiaddr> ) -> Result<(), Multiaddr>
where where
Du: ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/ Du: Transport + 'static, // TODO: 'static :-/
Df: FnOnce(Du::Output, Multiaddr) -> Dfu + 'static, // TODO: 'static :-/ Df: FnOnce(Du::Output, Multiaddr) -> Dfu + 'static, // TODO: 'static :-/
Dfu: IntoFuture<Item = (), Error = IoError> + 'static, // TODO: 'static :-/ Dfu: IntoFuture<Item = (), Error = IoError> + 'static, // TODO: 'static :-/
{ {
trace!(target: "libp2p-swarm", "Swarm dialing {} with custom handler", multiaddr); trace!(target: "libp2p-swarm", "Swarm dialing {} with custom handler", multiaddr);
match self.transport.clone().with_upgrade(upgrade).dial(multiaddr) { match transport.dial(multiaddr) {
Ok(dial) => { Ok(dial) => {
let dial = Box::new(dial.and_then(|(d, m)| and_then(d, m))) as Box<_>; let dial = Box::new(dial.into_future().and_then(|(d, m)| and_then(d, m))) as Box<_>;
// Ignoring errors if the receiver has been closed, because in that situation // Ignoring errors if the receiver has been closed, because in that situation
// nothing is going to be processed anyway. // nothing is going to be processed anyway.
let _ = self.new_toprocess.unbounded_send(dial); let _ = self.new_toprocess.unbounded_send(dial);
@ -192,7 +169,7 @@ where
/// Adds a multiaddr to listen on. All the incoming connections will use the `upgrade` that /// Adds a multiaddr to listen on. All the incoming connections will use the `upgrade` that
/// was passed to `swarm`. /// was passed to `swarm`.
pub fn listen_on(&self, multiaddr: Multiaddr) -> Result<Multiaddr, Multiaddr> { pub fn listen_on(&self, multiaddr: Multiaddr) -> Result<Multiaddr, Multiaddr> {
match self.upgraded.clone().listen_on(multiaddr) { match self.transport.clone().listen_on(multiaddr) {
Ok((listener, new_addr)) => { Ok((listener, new_addr)) => {
trace!(target: "libp2p-swarm", "Swarm listening on {}", new_addr); trace!(target: "libp2p-swarm", "Swarm listening on {}", new_addr);
// Ignoring errors if the receiver has been closed, because in that situation // Ignoring errors if the receiver has been closed, because in that situation
@ -206,49 +183,37 @@ where
} }
/// Future that must be driven to completion in order for the swarm to work. /// Future that must be driven to completion in order for the swarm to work.
pub struct SwarmFuture<T, C, H, F> pub struct SwarmFuture<T, H, F>
where where
T: MuxedTransport + 'static, // TODO: 'static :-/ T: MuxedTransport + 'static, // TODO: 'static :-/
C: ConnectionUpgrade<T::RawConn> + 'static, // TODO: 'static :-/
{ {
upgraded: UpgradedNode<T, C>, transport: T,
handler: H, handler: H,
new_listeners: mpsc::UnboundedReceiver< new_listeners: mpsc::UnboundedReceiver<T::Listener>,
Box< next_incoming: T::Incoming,
Stream<
Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>,
Error = IoError,
>,
>,
>,
next_incoming: Box<
Future<Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>, Error = IoError>,
>,
listeners: FuturesUnordered< listeners: FuturesUnordered<
StreamFuture< StreamFuture<
Box< Box<
Stream< Stream<
Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>, Item = Box<Future<Item = (T::Output, Multiaddr), Error = IoError>>,
Error = IoError, Error = IoError,
>, >,
>, >,
>, >,
>, >,
listeners_upgrade: listeners_upgrade:
FuturesUnordered<Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>>, FuturesUnordered<Box<Future<Item = (T::Output, Multiaddr), Error = IoError>>>,
dialers: FuturesUnordered<Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>>, dialers: FuturesUnordered<Box<Future<Item = (T::Output, Multiaddr), Error = IoError>>>,
new_dialers: new_dialers:
mpsc::UnboundedReceiver<Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>>, mpsc::UnboundedReceiver<Box<Future<Item = (T::Output, Multiaddr), Error = IoError>>>,
to_process: FuturesUnordered<future::Either<F, Box<Future<Item = (), Error = IoError>>>>, to_process: FuturesUnordered<future::Either<F, Box<Future<Item = (), Error = IoError>>>>,
new_toprocess: mpsc::UnboundedReceiver<Box<Future<Item = (), Error = IoError>>>, new_toprocess: mpsc::UnboundedReceiver<Box<Future<Item = (), Error = IoError>>>,
} }
impl<T, C, H, If, F> Future for SwarmFuture<T, C, H, F> impl<T, H, If, F> Future for SwarmFuture<T, H, F>
where where
T: MuxedTransport + Clone + 'static, // TODO: 'static :-/, T: MuxedTransport + Clone + 'static, // TODO: 'static :-/,
C: ConnectionUpgrade<T::RawConn> + Clone + 'static, // TODO: 'static :-/ H: FnMut(T::Output, Multiaddr) -> If,
C::NamesIter: Clone, // TODO: not elegant
H: FnMut(C::Output, Multiaddr) -> If,
If: IntoFuture<Future = F, Item = (), Error = IoError>, If: IntoFuture<Future = F, Item = (), Error = IoError>,
F: Future<Item = (), Error = IoError>, F: Future<Item = (), Error = IoError>,
{ {
@ -262,19 +227,22 @@ where
Ok(Async::Ready(connec)) => { Ok(Async::Ready(connec)) => {
debug!(target: "libp2p-swarm", "Swarm received new multiplexed \ debug!(target: "libp2p-swarm", "Swarm received new multiplexed \
incoming connection"); incoming connection");
self.next_incoming = self.upgraded.clone().next_incoming(); self.next_incoming = self.transport.clone().next_incoming();
self.listeners_upgrade.push(connec); self.listeners_upgrade.push(Box::new(connec) as Box<_>);
} }
Ok(Async::NotReady) => {} Ok(Async::NotReady) => {}
Err(err) => { Err(err) => {
debug!(target: "libp2p-swarm", "Error in multiplexed incoming \ debug!(target: "libp2p-swarm", "Error in multiplexed incoming \
connection: {:?}", err); connection: {:?}", err);
self.next_incoming = self.upgraded.clone().next_incoming(); self.next_incoming = self.transport.clone().next_incoming();
} }
}; };
match self.new_listeners.poll() { match self.new_listeners.poll() {
Ok(Async::Ready(Some(new_listener))) => { Ok(Async::Ready(Some(new_listener))) => {
let new_listener = Box::new(
new_listener.map(|f| Box::new(f) as Box<Future<Item = _, Error = _>>),
) as Box<Stream<Item = _, Error = _>>;
self.listeners.push(new_listener.into_future()); self.listeners.push(new_listener.into_future());
} }
Ok(Async::Ready(None)) | Err(_) => { Ok(Async::Ready(None)) | Err(_) => {

View File

@ -0,0 +1,141 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use futures::prelude::*;
use multiaddr::Multiaddr;
use std::io::Error as IoError;
use transport::{MuxedTransport, Transport};
use upgrade::Endpoint;
/// See the `Transport::and_then` method.
#[inline]
pub fn and_then<T, C>(transport: T, upgrade: C) -> AndThen<T, C> {
AndThen { transport, upgrade }
}
/// See the `Transport::and_then` method.
#[derive(Debug, Clone)]
pub struct AndThen<T, C> {
transport: T,
upgrade: C,
}
impl<T, C, F, O> Transport for AndThen<T, C>
where
T: Transport + 'static,
C: FnOnce(T::Output, Endpoint, Multiaddr) -> F + Clone + 'static,
F: Future<Item = O, Error = IoError> + 'static,
{
type Output = O;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
type ListenerUpgrade = Box<Future<Item = (O, Multiaddr), Error = IoError>>;
type Dial = Box<Future<Item = (O, Multiaddr), Error = IoError>>;
#[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
let upgrade = self.upgrade;
let (listening_stream, new_addr) = match self.transport.listen_on(addr) {
Ok((l, new_addr)) => (l, new_addr),
Err((trans, addr)) => {
let builder = AndThen {
transport: trans,
upgrade: upgrade,
};
return Err((builder, addr));
}
};
// Try to negotiate the protocol.
// Note that failing to negotiate a protocol will never produce a future with an error.
// Instead the `stream` will produce `Ok(Err(...))`.
// `stream` can only produce an `Err` if `listening_stream` produces an `Err`.
let stream = listening_stream.map(move |connection| {
let upgrade = upgrade.clone();
let future = connection.and_then(move |(stream, client_addr)| {
upgrade(stream, Endpoint::Listener, client_addr.clone()).map(|o| (o, client_addr))
});
Box::new(future) as Box<_>
});
Ok((Box::new(stream), new_addr))
}
#[inline]
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
let upgrade = self.upgrade;
let dialed_fut = match self.transport.dial(addr.clone()) {
Ok(f) => f.into_future(),
Err((trans, addr)) => {
let builder = AndThen {
transport: trans,
upgrade: upgrade,
};
return Err((builder, addr));
}
};
let future = dialed_fut
// Try to negotiate the protocol.
.and_then(move |(connection, client_addr)| {
upgrade(connection, Endpoint::Dialer, client_addr.clone())
.map(|o| (o, client_addr))
});
Ok(Box::new(future))
}
#[inline]
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> {
self.transport.nat_traversal(server, observed)
}
}
impl<T, C, F, O> MuxedTransport for AndThen<T, C>
where
T: MuxedTransport + 'static,
C: FnOnce(T::Output, Endpoint, Multiaddr) -> F + Clone + 'static,
F: Future<Item = O, Error = IoError> + 'static,
{
type Incoming = Box<Future<Item = Self::IncomingUpgrade, Error = IoError>>;
type IncomingUpgrade = Box<Future<Item = (O, Multiaddr), Error = IoError>>;
#[inline]
fn next_incoming(self) -> Self::Incoming {
let upgrade = self.upgrade;
let future = self.transport.next_incoming().map(|future| {
// Try to negotiate the protocol.
let future = future.and_then(move |(connection, client_addr)| {
let upgrade = upgrade.clone();
upgrade(connection, Endpoint::Listener, client_addr.clone())
.map(|o| (o, client_addr))
});
Box::new(future) as Box<Future<Item = _, Error = _>>
});
Box::new(future) as Box<_>
}
}

View File

@ -39,7 +39,7 @@ where
A: Transport, A: Transport,
B: Transport, B: Transport,
{ {
type RawConn = EitherSocket<A::RawConn, B::RawConn>; type Output = EitherSocket<A::Output, B::Output>;
type Listener = EitherListenStream<A::Listener, B::Listener>; type Listener = EitherListenStream<A::Listener, B::Listener>;
type ListenerUpgrade = EitherListenUpgrade<A::ListenerUpgrade, B::ListenerUpgrade>; type ListenerUpgrade = EitherListenUpgrade<A::ListenerUpgrade, B::ListenerUpgrade>;
type Dial = type Dial =
@ -88,12 +88,12 @@ where
B::Incoming: 'static, // TODO: meh :-/ B::Incoming: 'static, // TODO: meh :-/
A::IncomingUpgrade: 'static, // TODO: meh :-/ A::IncomingUpgrade: 'static, // TODO: meh :-/
B::IncomingUpgrade: 'static, // TODO: meh :-/ B::IncomingUpgrade: 'static, // TODO: meh :-/
A::RawConn: 'static, // TODO: meh :-/ A::Output: 'static, // TODO: meh :-/
B::RawConn: 'static, // TODO: meh :-/ B::Output: 'static, // TODO: meh :-/
{ {
type Incoming = Box<Future<Item = Self::IncomingUpgrade, Error = IoError>>; type Incoming = Box<Future<Item = Self::IncomingUpgrade, Error = IoError>>;
type IncomingUpgrade = type IncomingUpgrade =
Box<Future<Item = (EitherSocket<A::RawConn, B::RawConn>, Multiaddr), Error = IoError>>; Box<Future<Item = (EitherSocket<A::Output, B::Output>, Multiaddr), Error = IoError>>;
#[inline] #[inline]
fn next_incoming(self) -> Self::Incoming { fn next_incoming(self) -> Self::Incoming {

View File

@ -22,8 +22,8 @@ use futures::future;
use futures::prelude::*; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::io::{self, Cursor}; use std::io::{self, Cursor};
use transport::Transport;
use transport::MuxedTransport; use transport::MuxedTransport;
use transport::Transport;
/// Dummy implementation of `Transport` that just denies every single attempt. /// Dummy implementation of `Transport` that just denies every single attempt.
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
@ -31,10 +31,10 @@ pub struct DeniedTransport;
impl Transport for DeniedTransport { impl Transport for DeniedTransport {
// TODO: could use `!` for associated types once stable // TODO: could use `!` for associated types once stable
type RawConn = Cursor<Vec<u8>>; type Output = Cursor<Vec<u8>>;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = io::Error>>; type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = io::Error>>;
type ListenerUpgrade = Box<Future<Item = (Self::RawConn, Multiaddr), Error = io::Error>>; type ListenerUpgrade = Box<Future<Item = (Self::Output, Multiaddr), Error = io::Error>>;
type Dial = Box<Future<Item = (Self::RawConn, Multiaddr), Error = io::Error>>; type Dial = Box<Future<Item = (Self::Output, Multiaddr), Error = io::Error>>;
#[inline] #[inline]
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
@ -54,7 +54,7 @@ impl Transport for DeniedTransport {
impl MuxedTransport for DeniedTransport { impl MuxedTransport for DeniedTransport {
type Incoming = future::Empty<Self::IncomingUpgrade, io::Error>; type Incoming = future::Empty<Self::IncomingUpgrade, io::Error>;
type IncomingUpgrade = future::Empty<(Self::RawConn, Multiaddr), io::Error>; type IncomingUpgrade = future::Empty<(Self::Output, Multiaddr), io::Error>;
#[inline] #[inline]
fn next_incoming(self) -> Self::Incoming { fn next_incoming(self) -> Self::Incoming {

View File

@ -40,7 +40,7 @@ where
T: Transport, T: Transport,
{ {
type Incoming = future::Empty<Self::IncomingUpgrade, IoError>; type Incoming = future::Empty<Self::IncomingUpgrade, IoError>;
type IncomingUpgrade = future::Empty<(T::RawConn, Multiaddr), IoError>; type IncomingUpgrade = future::Empty<(T::Output, Multiaddr), IoError>;
fn next_incoming(self) -> Self::Incoming fn next_incoming(self) -> Self::Incoming
where where
@ -54,7 +54,7 @@ impl<T> Transport for DummyMuxing<T>
where where
T: Transport, T: Transport,
{ {
type RawConn = T::RawConn; type Output = T::Output;
type Listener = T::Listener; type Listener = T::Listener;
type ListenerUpgrade = T::ListenerUpgrade; type ListenerUpgrade = T::ListenerUpgrade;
type Dial = T::Dial; type Dial = T::Dial;

108
swarm/src/transport/map.rs Normal file
View File

@ -0,0 +1,108 @@
// Copyright 2017 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use futures::prelude::*;
use multiaddr::Multiaddr;
use std::io::Error as IoError;
use transport::{MuxedTransport, Transport};
use Endpoint;
/// See `Transport::map`.
#[derive(Debug, Copy, Clone)]
pub struct Map<T, F> {
transport: T,
map: F,
}
impl<T, F> Map<T, F> {
/// Internal function that builds a `Map`.
#[inline]
pub(crate) fn new(transport: T, map: F) -> Map<T, F> {
Map { transport, map }
}
}
impl<T, F, D> Transport for Map<T, F>
where
T: Transport + 'static, // TODO: 'static :-/
F: FnOnce(T::Output, Endpoint, Multiaddr) -> D + Clone + 'static, // TODO: 'static :-/
{
type Output = D;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
type ListenerUpgrade = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
type Dial = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
let map = self.map;
match self.transport.listen_on(addr) {
Ok((stream, listen_addr)) => {
let stream = stream.map(move |future| {
let map = map.clone();
let future = future
.into_future()
.map(move |(output, addr)| (map(output, Endpoint::Listener, addr.clone()), addr));
Box::new(future) as Box<_>
});
Ok((Box::new(stream), listen_addr))
}
Err((transport, addr)) => Err((Map { transport, map }, addr)),
}
}
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
let map = self.map;
match self.transport.dial(addr) {
Ok(future) => {
let future = future
.into_future()
.map(move |(output, addr)| (map(output, Endpoint::Dialer, addr.clone()), addr));
Ok(Box::new(future))
}
Err((transport, addr)) => Err((Map { transport, map }, addr)),
}
}
#[inline]
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> {
self.transport.nat_traversal(server, observed)
}
}
impl<T, F, D> MuxedTransport for Map<T, F>
where
T: MuxedTransport + 'static, // TODO: 'static :-/
F: FnOnce(T::Output, Endpoint, Multiaddr) -> D + Clone + 'static, // TODO: 'static :-/
{
type Incoming = Box<Future<Item = Self::IncomingUpgrade, Error = IoError>>;
type IncomingUpgrade = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
fn next_incoming(self) -> Self::Incoming {
let map = self.map;
let future = self.transport.next_incoming().map(move |upgrade| {
let future = upgrade.map(move |(output, addr)| {
(map(output, Endpoint::Listener, addr.clone()), addr)
});
Box::new(future) as Box<_>
});
Box::new(future)
}
}

View File

@ -33,11 +33,13 @@ use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::io::Error as IoError; use std::io::Error as IoError;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::ConnectionUpgrade; use upgrade::{ConnectionUpgrade, Endpoint};
pub mod and_then;
pub mod choice; pub mod choice;
pub mod denied; pub mod denied;
pub mod dummy; pub mod dummy;
pub mod map;
pub mod muxed; pub mod muxed;
pub mod upgrade; pub mod upgrade;
@ -59,7 +61,7 @@ pub use self::upgrade::UpgradedNode;
/// > on `Foo`. /// > on `Foo`.
pub trait Transport { pub trait Transport {
/// The raw connection to a peer. /// The raw connection to a peer.
type RawConn: AsyncRead + AsyncWrite; type Output;
/// The listener produces incoming connections. /// The listener produces incoming connections.
/// ///
@ -71,10 +73,10 @@ pub trait Transport {
/// After a connection has been received, we may need to do some asynchronous pre-processing /// After a connection has been received, we may need to do some asynchronous pre-processing
/// on it (eg. an intermediary protocol negotiation). While this pre-processing takes place, we /// on it (eg. an intermediary protocol negotiation). While this pre-processing takes place, we
/// want to be able to continue polling on the listener. /// want to be able to continue polling on the listener.
type ListenerUpgrade: Future<Item = (Self::RawConn, Multiaddr), Error = IoError>; type ListenerUpgrade: Future<Item = (Self::Output, Multiaddr), Error = IoError>;
/// A future which indicates that we are currently dialing to a peer. /// A future which indicates that we are currently dialing to a peer.
type Dial: IntoFuture<Item = (Self::RawConn, Multiaddr), Error = IoError>; type Dial: IntoFuture<Item = (Self::Output, Multiaddr), Error = IoError>;
/// Listen on the given multiaddr. Returns a stream of incoming connections, plus a modified /// Listen on the given multiaddr. Returns a stream of incoming connections, plus a modified
/// version of the `Multiaddr`. This new `Multiaddr` is the one that that should be advertised /// version of the `Multiaddr`. This new `Multiaddr` is the one that that should be advertised
@ -108,6 +110,16 @@ pub trait Transport {
/// doesn't recognize the protocols, or if `server` and `observed` are related. /// doesn't recognize the protocols, or if `server` and `observed` are related.
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr>; fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr>;
/// Applies a function on the output of the `Transport`.
#[inline]
fn map<F, O>(self, map: F) -> map::Map<Self, F>
where
Self: Sized,
F: FnOnce(Self::Output, Endpoint, Multiaddr) -> O + Clone + 'static, // TODO: 'static :-/
{
map::Map::new(self, map)
}
/// Builds a new struct that implements `Transport` that contains both `self` and `other`. /// Builds a new struct that implements `Transport` that contains both `self` and `other`.
/// ///
/// The returned object will redirect its calls to `self`, except that if `listen_on` or `dial` /// The returned object will redirect its calls to `self`, except that if `listen_on` or `dial`
@ -129,11 +141,27 @@ pub trait Transport {
fn with_upgrade<U>(self, upgrade: U) -> UpgradedNode<Self, U> fn with_upgrade<U>(self, upgrade: U) -> UpgradedNode<Self, U>
where where
Self: Sized, Self: Sized,
U: ConnectionUpgrade<Self::RawConn>, Self::Output: AsyncRead + AsyncWrite,
U: ConnectionUpgrade<Self::Output>,
{ {
UpgradedNode::new(self, upgrade) UpgradedNode::new(self, upgrade)
} }
/// Wraps this transport inside an upgrade. Whenever a connection that uses this transport
/// is established, it is wrapped inside the upgrade.
///
/// > **Note**: The concept of an *upgrade* for example includes middlewares such *secio*
/// > (communication encryption), *multiplex*, but also a protocol handler.
#[inline]
fn and_then<C, F>(self, upgrade: C) -> and_then::AndThen<Self, C>
where
Self: Sized,
C: FnOnce(Self::Output, Endpoint, Multiaddr) -> F + Clone + 'static,
F: Future<Error = IoError> + 'static,
{
and_then::and_then(self, upgrade)
}
/// Builds a dummy implementation of `MuxedTransport` that uses this transport. /// Builds a dummy implementation of `MuxedTransport` that uses this transport.
/// ///
/// The resulting object will not actually use muxing. This means that dialing the same node /// The resulting object will not actually use muxing. This means that dialing the same node

View File

@ -30,7 +30,7 @@ pub trait MuxedTransport: Transport {
/// Future resolving to a future that will resolve to an incoming connection. /// Future resolving to a future that will resolve to an incoming connection.
type Incoming: Future<Item = Self::IncomingUpgrade, Error = IoError>; type Incoming: Future<Item = Self::IncomingUpgrade, Error = IoError>;
/// Future resolving to an incoming connection. /// Future resolving to an incoming connection.
type IncomingUpgrade: Future<Item = (Self::RawConn, Multiaddr), Error = IoError>; type IncomingUpgrade: Future<Item = (Self::Output, Multiaddr), Error = IoError>;
/// Returns the next incoming substream opened by a node that we dialed ourselves. /// Returns the next incoming substream opened by a node that we dialed ourselves.
/// ///

View File

@ -18,16 +18,14 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::Bytes;
use connection_reuse::ConnectionReuse; use connection_reuse::ConnectionReuse;
use futures::prelude::*; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use multistream_select;
use muxing::StreamMuxer; use muxing::StreamMuxer;
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::Error as IoError;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use transport::{MuxedTransport, Transport}; use transport::{MuxedTransport, Transport};
use upgrade::{ConnectionUpgrade, Endpoint}; use upgrade::{apply, ConnectionUpgrade, Endpoint};
/// Implements the `Transport` trait. Dials or listens, then upgrades any dialed or received /// Implements the `Transport` trait. Dials or listens, then upgrades any dialed or received
/// connection. /// connection.
@ -51,7 +49,8 @@ impl<T, C> UpgradedNode<T, C> {
impl<'a, T, C> UpgradedNode<T, C> impl<'a, T, C> UpgradedNode<T, C>
where where
T: Transport + 'a, T: Transport + 'a,
C: ConnectionUpgrade<T::RawConn> + 'a, T::Output: AsyncRead + AsyncWrite,
C: ConnectionUpgrade<T::Output> + 'a,
{ {
/// Turns this upgraded node into a `ConnectionReuse`. If the `Output` implements the /// Turns this upgraded node into a `ConnectionReuse`. If the `Output` implements the
/// `StreamMuxer` trait, the returned object will implement `Transport` and `MuxedTransport`. /// `StreamMuxer` trait, the returned object will implement `Transport` and `MuxedTransport`.
@ -79,6 +78,8 @@ where
self, self,
addr: Multiaddr, addr: Multiaddr,
) -> Result<Box<Future<Item = (C::Output, Multiaddr), Error = IoError> + 'a>, (Self, Multiaddr)> ) -> Result<Box<Future<Item = (C::Output, Multiaddr), Error = IoError> + 'a>, (Self, Multiaddr)>
where
C::NamesIter: Clone, // TODO: not elegant
{ {
let upgrade = self.upgrade; let upgrade = self.upgrade;
@ -97,39 +98,7 @@ where
let future = dialed_fut let future = dialed_fut
// Try to negotiate the protocol. // Try to negotiate the protocol.
.and_then(move |(connection, client_addr)| { .and_then(move |(connection, client_addr)| {
let iter = upgrade.protocol_names() apply(connection, upgrade, Endpoint::Dialer, client_addr)
.map(|(name, id)| (name, <Bytes as PartialEq>::eq, id));
debug!(target: "libp2p-swarm", "Starting protocol negotiation (dialer)");
let negotiated = multistream_select::dialer_select_proto(connection, iter)
.map_err(|err| IoError::new(IoErrorKind::Other, err));
negotiated.map(|(upgrade_id, conn)| (upgrade_id, conn, upgrade, client_addr))
})
.then(|negotiated| {
match negotiated {
Ok((_, _, _, ref client_addr)) => {
debug!(target: "libp2p-swarm", "Successfully negotiated protocol \
upgrade with {}", client_addr)
},
Err(ref err) => {
debug!(target: "libp2p-swarm", "Error while negotiated protocol \
upgrade: {:?}", err)
},
};
negotiated
})
.and_then(move |(upgrade_id, connection, upgrade, client_addr)| {
let f = upgrade.upgrade(connection, upgrade_id, Endpoint::Dialer, &client_addr);
debug!(target: "libp2p-swarm", "Trying to apply negotiated protocol with {}",
client_addr);
f.map(|v| (v, client_addr))
})
.then(|val| {
match val {
Ok(_) => debug!(target: "libp2p-swarm", "Successfully applied negotiated \
protocol"),
Err(_) => debug!(target: "libp2p-swarm", "Failed to apply negotiated protocol"),
}
val
}); });
Ok(Box::new(future)) Ok(Box::new(future))
@ -144,9 +113,9 @@ where
self, self,
) -> Box< ) -> Box<
Future< Future<
Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError> + 'a>, Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError> + 'a>,
Error = IoError, Error = IoError,
> >
+ 'a, + 'a,
> >
where where
@ -158,44 +127,9 @@ where
let future = self.transports.next_incoming().map(|future| { let future = self.transports.next_incoming().map(|future| {
// Try to negotiate the protocol. // Try to negotiate the protocol.
let future = future let future = future.and_then(move |(connection, client_addr)| {
.and_then(move |(connection, addr)| { apply(connection, upgrade, Endpoint::Listener, client_addr)
let iter = upgrade });
.protocol_names()
.map::<_, fn(_) -> _>(|(name, id)| (name, <Bytes as PartialEq>::eq, id));
debug!(target: "libp2p-swarm", "Starting protocol negotiation (incoming)");
let negotiated = multistream_select::listener_select_proto(connection, iter)
.map_err(|err| IoError::new(IoErrorKind::Other, err));
negotiated.map(move |(upgrade_id, conn)| (upgrade_id, conn, upgrade, addr))
})
.then(|negotiated| {
match negotiated {
Ok((_, _, _, ref client_addr)) => {
debug!(target: "libp2p-swarm", "Successfully negotiated protocol \
upgrade with {}", client_addr)
}
Err(ref err) => {
debug!(target: "libp2p-swarm", "Error while negotiated protocol \
upgrade: {:?}", err)
}
};
negotiated
})
.and_then(move |(upgrade_id, connection, upgrade, addr)| {
let upg = upgrade.upgrade(connection, upgrade_id, Endpoint::Listener, &addr);
debug!(target: "libp2p-swarm", "Trying to apply negotiated protocol with {}",
addr);
upg.map(|u| (u, addr))
})
.then(|val| {
match val {
Ok(_) => debug!(target: "libp2p-swarm", "Successfully applied negotiated \
protocol"),
Err(_) => debug!(target: "libp2p-swarm", "Failed to apply negotiated \
protocol"),
}
val
});
Box::new(future) as Box<Future<Item = _, Error = _>> Box::new(future) as Box<Future<Item = _, Error = _>>
}); });
@ -216,9 +150,9 @@ where
( (
Box< Box<
Stream< Stream<
Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError> + 'a>, Item = Box<Future<Item = (C::Output, Multiaddr), Error = IoError> + 'a>,
Error = IoError, Error = IoError,
> >
+ 'a, + 'a,
>, >,
Multiaddr, Multiaddr,
@ -250,38 +184,10 @@ where
let stream = listening_stream.map(move |connection| { let stream = listening_stream.map(move |connection| {
let upgrade = upgrade.clone(); let upgrade = upgrade.clone();
let connection = connection let connection = connection
// Try to negotiate the protocol // Try to negotiate the protocol.
.and_then(move |(connection, remote_addr)| { .and_then(move |(connection, client_addr)| {
let iter = upgrade.protocol_names() apply(connection, upgrade, Endpoint::Listener, client_addr)
.map::<_, fn(_) -> _>(|(n, t)| (n, <Bytes as PartialEq>::eq, t)); });
let remote_addr2 = remote_addr.clone();
debug!(target: "libp2p-swarm", "Starting protocol negotiation (listener)");
multistream_select::listener_select_proto(connection, iter)
.map_err(|err| IoError::new(IoErrorKind::Other, err))
.then(move |negotiated| {
match negotiated {
Ok(_) => {
debug!(target: "libp2p-swarm", "Successfully negotiated \
protocol upgrade with {}", remote_addr2)
},
Err(ref err) => {
debug!(target: "libp2p-swarm", "Error while negotiated \
protocol upgrade: {:?}", err)
},
};
negotiated
})
.and_then(move |(upgrade_id, connection)| {
let fut = upgrade.upgrade(
connection,
upgrade_id,
Endpoint::Listener,
&remote_addr,
);
fut.map(move |c| (c, remote_addr))
})
.into_future()
});
Box::new(connection) as Box<_> Box::new(connection) as Box<_>
}); });
@ -293,12 +199,12 @@ where
impl<T, C> Transport for UpgradedNode<T, C> impl<T, C> Transport for UpgradedNode<T, C>
where where
T: Transport + 'static, T: Transport + 'static,
C: ConnectionUpgrade<T::RawConn> + 'static, T::Output: AsyncRead + AsyncWrite,
C::Output: AsyncRead + AsyncWrite, C: ConnectionUpgrade<T::Output> + 'static,
C::NamesIter: Clone, // TODO: not elegant C::NamesIter: Clone, // TODO: not elegant
C: Clone, C: Clone,
{ {
type RawConn = C::Output; type Output = C::Output;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>; type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
type ListenerUpgrade = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>; type ListenerUpgrade = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>;
type Dial = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (C::Output, Multiaddr), Error = IoError>>;
@ -322,8 +228,8 @@ where
impl<T, C> MuxedTransport for UpgradedNode<T, C> impl<T, C> MuxedTransport for UpgradedNode<T, C>
where where
T: MuxedTransport + 'static, T: MuxedTransport + 'static,
C: ConnectionUpgrade<T::RawConn> + 'static, T::Output: AsyncRead + AsyncWrite,
C::Output: AsyncRead + AsyncWrite, C: ConnectionUpgrade<T::Output> + 'static,
C::NamesIter: Clone, // TODO: not elegant C::NamesIter: Clone, // TODO: not elegant
C: Clone, C: Clone,
{ {

View File

@ -0,0 +1,81 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use bytes::Bytes;
use futures::prelude::*;
use multiaddr::Multiaddr;
use multistream_select;
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint};
/// Applies a connection upgrade on a socket.
///
/// Returns a `Future` that returns the outcome of the connection upgrade.
#[inline]
pub fn apply<'a, C, U>(
connection: C,
upgrade: U,
endpoint: Endpoint,
remote_addr: Multiaddr,
) -> Box<Future<Item = (U::Output, Multiaddr), Error = IoError> + 'a>
where
U: ConnectionUpgrade<C> + 'a,
U::NamesIter: Clone, // TODO: not elegant
C: AsyncRead + AsyncWrite + 'a,
{
let iter = upgrade
.protocol_names()
.map::<_, fn(_) -> _>(|(n, t)| (n, <Bytes as PartialEq>::eq, t));
let remote_addr2 = remote_addr.clone();
debug!(target: "libp2p-swarm", "Starting protocol negotiation");
let negotiation = match endpoint {
Endpoint::Listener => multistream_select::listener_select_proto(connection, iter),
Endpoint::Dialer => multistream_select::dialer_select_proto(connection, iter),
};
let future = negotiation
.map_err(|err| IoError::new(IoErrorKind::Other, err))
.then(move |negotiated| {
match negotiated {
Ok(_) => debug!(target: "libp2p-swarm", "Successfully negotiated \
protocol upgrade with {}", remote_addr2),
Err(ref err) => debug!(target: "libp2p-swarm", "Error while negotiated \
protocol upgrade: {:?}", err),
};
negotiated
})
.and_then(move |(upgrade_id, connection)| {
let fut = upgrade.upgrade(connection, upgrade_id, endpoint, &remote_addr);
fut.map(move |c| (c, remote_addr))
})
.into_future()
.then(|val| {
match val {
Ok(_) => debug!(target: "libp2p-swarm", "Successfully applied negotiated \
protocol"),
Err(_) => debug!(target: "libp2p-swarm", "Failed to apply negotiated protocol"),
}
val
});
Box::new(future)
}

View File

@ -26,16 +26,19 @@ use std::io::Error as IoError;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint}; use upgrade::{ConnectionUpgrade, Endpoint};
/// See `transport::Transport::or_upgrade()`. /// Builds a new `ConnectionUpgrade` that chooses between `A` and `B`.
///
/// If both `A` and `B` are supported by the remote, then `A` will be chosen.
// TODO: write a test for this ^
#[inline]
pub fn or<A, B>(me: A, other: B) -> OrUpgrade<A, B> {
OrUpgrade(me, other)
}
/// See `upgrade::or`.
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]
pub struct OrUpgrade<A, B>(A, B); pub struct OrUpgrade<A, B>(A, B);
impl<A, B> OrUpgrade<A, B> {
pub fn new(a: A, b: B) -> OrUpgrade<A, B> {
OrUpgrade(a, b)
}
}
impl<C, A, B> ConnectionUpgrade<C> for OrUpgrade<A, B> impl<C, A, B> ConnectionUpgrade<C> for OrUpgrade<A, B>
where where
C: AsyncRead + AsyncWrite, C: AsyncRead + AsyncWrite,

View File

@ -19,11 +19,11 @@
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use bytes::Bytes; use bytes::Bytes;
use upgrade::{ConnectionUpgrade, Endpoint};
use futures::prelude::*; use futures::prelude::*;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::{io, iter}; use std::{io, iter};
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint};
/// Implementation of `ConnectionUpgrade` that always fails to negotiate. /// Implementation of `ConnectionUpgrade` that always fails to negotiate.
#[derive(Debug, Copy, Clone)] #[derive(Debug, Copy, Clone)]

66
swarm/src/upgrade/map.rs Normal file
View File

@ -0,0 +1,66 @@
// Copyright 2017 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
use futures::{future, prelude::*};
use multiaddr::Multiaddr;
use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::{ConnectionUpgrade, Endpoint};
/// Applies a closure on the output of a connection upgrade.
#[inline]
pub fn map<U, F>(upgrade: U, map: F) -> Map<U, F> {
Map { upgrade, map }
}
/// Application of a closure on the output of a connection upgrade.
#[derive(Debug, Copy, Clone)]
pub struct Map<U, F> {
upgrade: U,
map: F,
}
impl<C, U, F, O> ConnectionUpgrade<C> for Map<U, F>
where
U: ConnectionUpgrade<C>,
C: AsyncRead + AsyncWrite,
F: FnOnce(U::Output) -> O,
{
type NamesIter = U::NamesIter;
type UpgradeIdentifier = U::UpgradeIdentifier;
fn protocol_names(&self) -> Self::NamesIter {
self.upgrade.protocol_names()
}
type Output = O;
type Future = future::Map<U::Future, F>;
fn upgrade(
self,
socket: C,
id: Self::UpgradeIdentifier,
ty: Endpoint,
remote_addr: &Multiaddr,
) -> Self::Future {
self.upgrade
.upgrade(socket, id, ty, remote_addr)
.map(self.map)
}
}

View File

@ -18,14 +18,18 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
pub mod denied; pub mod apply;
pub mod traits;
pub mod choice; pub mod choice;
pub mod denied;
pub mod map;
pub mod plaintext; pub mod plaintext;
pub mod simple; pub mod simple;
pub mod traits;
pub use self::choice::OrUpgrade; pub use self::apply::apply;
pub use self::choice::{or, OrUpgrade};
pub use self::denied::DeniedConnectionUpgrade; pub use self::denied::DeniedConnectionUpgrade;
pub use self::map::map;
pub use self::plaintext::PlainTextConfig; pub use self::plaintext::PlainTextConfig;
pub use self::simple::SimpleProtocol; pub use self::simple::SimpleProtocol;
pub use self::traits::{ConnectionUpgrade, Endpoint, UpgradeExt}; pub use self::traits::{ConnectionUpgrade, Endpoint};

View File

@ -23,7 +23,6 @@ use futures::future::Future;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std::io::Error as IoError; use std::io::Error as IoError;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
use upgrade::choice::OrUpgrade;
/// Type of connection for the upgrade. /// Type of connection for the upgrade.
#[derive(Debug, Copy, Clone, PartialEq, Eq)] #[derive(Debug, Copy, Clone, PartialEq, Eq)]
@ -74,19 +73,3 @@ pub trait ConnectionUpgrade<C: AsyncRead + AsyncWrite> {
remote_addr: &Multiaddr, remote_addr: &Multiaddr,
) -> Self::Future; ) -> Self::Future;
} }
/// Extension trait for `ConnectionUpgrade`. Automatically implemented on everything.
pub trait UpgradeExt {
/// Builds a struct that will choose an upgrade between `self` and `other`, depending on what
/// the remote supports.
fn or_upgrade<T>(self, other: T) -> OrUpgrade<Self, T>
where
Self: Sized;
}
impl<T> UpgradeExt for T {
#[inline]
fn or_upgrade<U>(self, other: U) -> OrUpgrade<Self, U> {
OrUpgrade::new(self, other)
}
}

View File

@ -31,10 +31,10 @@ use futures::future::Future;
use futures::{Sink, Stream}; use futures::{Sink, Stream};
use libp2p_swarm::{Multiaddr, MuxedTransport, StreamMuxer, Transport}; use libp2p_swarm::{Multiaddr, MuxedTransport, StreamMuxer, Transport};
use libp2p_tcp_transport::TcpConfig; use libp2p_tcp_transport::TcpConfig;
use tokio_core::reactor::Core;
use tokio_io::codec::length_delimited::Framed;
use std::sync::{atomic, mpsc}; use std::sync::{atomic, mpsc};
use std::thread; use std::thread;
use tokio_core::reactor::Core;
use tokio_io::codec::length_delimited::Framed;
// Ensures that a transport is only ever used once for dialing. // Ensures that a transport is only ever used once for dialing.
#[derive(Debug)] #[derive(Debug)]
@ -53,7 +53,7 @@ impl<T: Clone> Clone for OnlyOnce<T> {
} }
} }
impl<T: Transport> Transport for OnlyOnce<T> { impl<T: Transport> Transport for OnlyOnce<T> {
type RawConn = T::RawConn; type Output = T::Output;
type Listener = T::Listener; type Listener = T::Listener;
type ListenerUpgrade = T::ListenerUpgrade; type ListenerUpgrade = T::ListenerUpgrade;
type Dial = T::Dial; type Dial = T::Dial;

View File

@ -57,15 +57,15 @@ extern crate multiaddr;
extern crate tokio_core; extern crate tokio_core;
extern crate tokio_io; extern crate tokio_io;
use std::io::Error as IoError;
use std::iter;
use std::net::SocketAddr;
use tokio_core::reactor::Handle;
use tokio_core::net::{TcpListener, TcpStream};
use futures::future::{self, Future, FutureResult, IntoFuture}; use futures::future::{self, Future, FutureResult, IntoFuture};
use futures::stream::Stream; use futures::stream::Stream;
use multiaddr::{AddrComponent, Multiaddr, ToMultiaddr}; use multiaddr::{AddrComponent, Multiaddr, ToMultiaddr};
use std::io::Error as IoError;
use std::iter;
use std::net::SocketAddr;
use swarm::Transport; use swarm::Transport;
use tokio_core::net::{TcpListener, TcpStream};
use tokio_core::reactor::Handle;
/// Represents the configuration for a TCP/IP transport capability for libp2p. /// Represents the configuration for a TCP/IP transport capability for libp2p.
/// ///
@ -87,9 +87,9 @@ impl TcpConfig {
} }
impl Transport for TcpConfig { impl Transport for TcpConfig {
type RawConn = TcpStream; type Output = TcpStream;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>; type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>;
type ListenerUpgrade = FutureResult<(Self::RawConn, Multiaddr), IoError>; type ListenerUpgrade = FutureResult<(Self::Output, Multiaddr), IoError>;
type Dial = Box<Future<Item = (TcpStream, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (TcpStream, Multiaddr), Error = IoError>>;
/// Listen on the given multi-addr. /// Listen on the given multi-addr.
@ -195,14 +195,14 @@ fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result<SocketAddr, ()> {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::{multiaddr_to_socketaddr, TcpConfig}; use super::{multiaddr_to_socketaddr, TcpConfig};
use std;
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
use tokio_core::reactor::Core;
use tokio_io;
use futures::Future; use futures::Future;
use futures::stream::Stream; use futures::stream::Stream;
use multiaddr::Multiaddr; use multiaddr::Multiaddr;
use std;
use std::net::{IpAddr, Ipv4Addr, SocketAddr};
use swarm::Transport; use swarm::Transport;
use tokio_core::reactor::Core;
use tokio_io;
#[test] #[test]
fn multiaddr_to_tcp_conversion() { fn multiaddr_to_tcp_conversion() {

View File

@ -35,12 +35,12 @@ use bytes::{BufMut, Bytes, BytesMut, IntoBuf};
use futures::{Async, Poll}; use futures::{Async, Poll};
use num_bigint::BigUint; use num_bigint::BigUint;
use num_traits::ToPrimitive; use num_traits::ToPrimitive;
use tokio_io::{AsyncRead, AsyncWrite};
use tokio_io::codec::{Decoder, Encoder};
use std::io; use std::io;
use std::io::prelude::*; use std::io::prelude::*;
use std::marker::PhantomData; use std::marker::PhantomData;
use std::mem; use std::mem;
use tokio_io::codec::{Decoder, Encoder};
use tokio_io::{AsyncRead, AsyncWrite};
mod errors { mod errors {
error_chain! { error_chain! {
@ -106,7 +106,7 @@ macro_rules! impl_bits {
(std::mem::size_of::<$t>() * 8) - self.leading_zeros() as usize (std::mem::size_of::<$t>() * 8) - self.leading_zeros() as usize
} }
} }
} };
} }
impl_bits!(usize); impl_bits!(usize);
@ -139,7 +139,9 @@ macro_rules! impl_decoderstate {
|a: $t, b| -> Option<$t> { a.checked_shl(b as u32) } |a: $t, b| -> Option<$t> { a.checked_shl(b as u32) }
); );
}; };
($t:ty, $make_fn:expr) => { impl_decoderstate!($t, $make_fn, $make_fn); }; ($t:ty, $make_fn:expr) => {
impl_decoderstate!($t, $make_fn, $make_fn);
};
($t:ty, $make_fn:expr, $shift_fn:expr) => { ($t:ty, $make_fn:expr, $shift_fn:expr) => {
impl DecoderHelper for $t { impl DecoderHelper for $t {
#[inline] #[inline]
@ -148,13 +150,14 @@ macro_rules! impl_decoderstate {
byte: u8, byte: u8,
) -> ::errors::Result<Option<$t>> { ) -> ::errors::Result<Option<$t>> {
let res = decoder.accumulator.take().and_then(|accumulator| { let res = decoder.accumulator.take().and_then(|accumulator| {
let out = accumulator | match $shift_fn( let out = accumulator
$make_fn(byte & 0x7F), | match $shift_fn(
decoder.shift * USABLE_BITS_PER_BYTE, $make_fn(byte & 0x7F),
) { decoder.shift * USABLE_BITS_PER_BYTE,
Some(a) => a, ) {
None => return Some(Err(ErrorKind::ParseError.into())), Some(a) => a,
}; None => return Some(Err(ErrorKind::ParseError.into())),
};
decoder.shift += 1; decoder.shift += 1;
if byte & 0x80 == 0 { if byte & 0x80 == 0 {
@ -174,7 +177,7 @@ macro_rules! impl_decoderstate {
fn read<R: AsyncRead>( fn read<R: AsyncRead>(
decoder: &mut DecoderState<Self>, decoder: &mut DecoderState<Self>,
mut input: R mut input: R,
) -> Poll<Option<Self>, Error> { ) -> Poll<Option<Self>, Error> {
if decoder.accumulator == AccumulatorState::Finished { if decoder.accumulator == AccumulatorState::Finished {
return Err(Error::with_chain( return Err(Error::with_chain(
@ -202,17 +205,19 @@ macro_rules! impl_decoderstate {
Err(inner) => if decoder.accumulator == AccumulatorState::NotStarted { Err(inner) => if decoder.accumulator == AccumulatorState::NotStarted {
break Ok(Async::Ready(None)); break Ok(Async::Ready(None));
} else { } else {
break Err(Error::with_chain(inner, ErrorKind::ParseError)) break Err(Error::with_chain(inner, ErrorKind::ParseError));
}, },
} }
} }
} }
} }
} };
} }
macro_rules! impl_encoderstate { macro_rules! impl_encoderstate {
($t:ty) => { impl_encoderstate!($t, <$t>::from); }; ($t:ty) => {
impl_encoderstate!($t, <$t>::from);
};
($t:ty, $make_fn:expr) => { ($t:ty, $make_fn:expr) => {
impl EncoderHelper for $t { impl EncoderHelper for $t {
/// Write as much as possible of the inner integer to the output `AsyncWrite` /// Write as much as possible of the inner integer to the output `AsyncWrite`
@ -227,11 +232,11 @@ macro_rules! impl_encoderstate {
return None; return None;
} }
let masked = (&encoder.source >> (encoder.cur_chunk * USABLE_BITS_PER_BYTE)) & let masked = (&encoder.source >> (encoder.cur_chunk * USABLE_BITS_PER_BYTE))
$make_fn((1 << USABLE_BITS_PER_BYTE) - 1usize); & $make_fn((1 << USABLE_BITS_PER_BYTE) - 1usize);
let masked = masked.to_u8().expect( let masked = masked
"Masked with 0b0111_1111, is less than u8::MAX, QED", .to_u8()
); .expect("Masked with 0b0111_1111, is less than u8::MAX, QED");
if encoder.cur_chunk == last_chunk { if encoder.cur_chunk == last_chunk {
Some(masked) Some(masked)
@ -258,9 +263,9 @@ macro_rules! impl_encoderstate {
Ok(Async::Ready(WriteState::Pending(written))) Ok(Async::Ready(WriteState::Pending(written)))
}; };
} }
Err(inner) => break Err( Err(inner) => {
Error::with_chain(inner, ErrorKind::WriteError) break Err(Error::with_chain(inner, ErrorKind::WriteError))
), }
} }
} else { } else {
break Ok(Async::Ready(WriteState::Done(written))); break Ok(Async::Ready(WriteState::Done(written)));
@ -268,7 +273,7 @@ macro_rules! impl_encoderstate {
} }
} }
} }
} };
} }
impl_encoderstate!(usize); impl_encoderstate!(usize);
@ -315,8 +320,8 @@ enum AccumulatorState<T> {
impl<T: Default> AccumulatorState<T> { impl<T: Default> AccumulatorState<T> {
fn take(&mut self) -> Option<T> { fn take(&mut self) -> Option<T> {
use std::mem;
use AccumulatorState::*; use AccumulatorState::*;
use std::mem;
match mem::replace(self, AccumulatorState::Finished) { match mem::replace(self, AccumulatorState::Finished) {
InProgress(inner) => Some(inner), InProgress(inner) => Some(inner),
@ -511,15 +516,15 @@ pub fn encode<T: EncoderHelper + Bits>(input: T) -> Bytes {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::{decode, EncoderState, VarintDecoder}; use super::{decode, EncoderState, VarintDecoder};
use tokio_io::codec::FramedRead;
use num_bigint::BigUint;
use futures::{Future, Stream}; use futures::{Future, Stream};
use num_bigint::BigUint;
use tokio_io::codec::FramedRead;
#[test] #[test]
fn large_number_fails() { fn large_number_fails() {
use std::io::Cursor;
use futures::Async;
use super::WriteState; use super::WriteState;
use futures::Async;
use std::io::Cursor;
let mut out = vec![0u8; 10]; let mut out = vec![0u8; 10];
@ -586,9 +591,9 @@ mod tests {
#[test] #[test]
fn can_encode_basic_biguint_async() { fn can_encode_basic_biguint_async() {
use std::io::Cursor;
use futures::Async;
use super::WriteState; use super::WriteState;
use futures::Async;
use std::io::Cursor;
let mut out = vec![0u8; 2]; let mut out = vec![0u8; 2];
@ -608,9 +613,9 @@ mod tests {
#[test] #[test]
fn can_encode_basic_usize_async() { fn can_encode_basic_usize_async() {
use std::io::Cursor;
use futures::Async;
use super::WriteState; use super::WriteState;
use futures::Async;
use std::io::Cursor;
let mut out = vec![0u8; 2]; let mut out = vec![0u8; 2];

View File

@ -18,17 +18,17 @@
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE. // DEALINGS IN THE SOFTWARE.
use futures::{Async, Future, Poll, Stream};
use futures::stream::Then as StreamThen; use futures::stream::Then as StreamThen;
use futures::sync::{mpsc, oneshot}; use futures::sync::{mpsc, oneshot};
use futures::{Async, Future, Poll, Stream};
use multiaddr::{AddrComponent, Multiaddr}; use multiaddr::{AddrComponent, Multiaddr};
use rw_stream_sink::RwStreamSink; use rw_stream_sink::RwStreamSink;
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::io::{Read, Write}; use std::io::{Read, Write};
use std::iter; use std::iter;
use std::sync::{Arc, Mutex}; use std::sync::{Arc, Mutex};
use stdweb::{self, Reference};
use stdweb::web::TypedArray; use stdweb::web::TypedArray;
use stdweb::{self, Reference};
use swarm::Transport; use swarm::Transport;
use tokio_io::{AsyncRead, AsyncWrite}; use tokio_io::{AsyncRead, AsyncWrite};
@ -52,10 +52,10 @@ impl BrowserWsConfig {
} }
impl Transport for BrowserWsConfig { impl Transport for BrowserWsConfig {
type RawConn = BrowserWsConn; type Output = BrowserWsConn;
type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>; // TODO: use `!` type Listener = Box<Stream<Item = Self::ListenerUpgrade, Error = IoError>>; // TODO: use `!`
type ListenerUpgrade = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; // TODO: use `!` type ListenerUpgrade = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>; // TODO: use `!`
type Dial = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
#[inline] #[inline]
fn listen_on(self, a: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> { fn listen_on(self, a: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {

View File

@ -23,6 +23,7 @@ use multiaddr::{AddrComponent, Multiaddr};
use rw_stream_sink::RwStreamSink; use rw_stream_sink::RwStreamSink;
use std::io::{Error as IoError, ErrorKind as IoErrorKind}; use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use swarm::Transport; use swarm::Transport;
use tokio_io::{AsyncRead, AsyncWrite};
use websocket::client::builder::ClientBuilder; use websocket::client::builder::ClientBuilder;
use websocket::message::OwnedMessage; use websocket::message::OwnedMessage;
use websocket::server::upgrade::async::IntoWs; use websocket::server::upgrade::async::IntoWs;
@ -59,13 +60,13 @@ where
// TODO: this 'static is pretty arbitrary and is necessary because of the websocket library // TODO: this 'static is pretty arbitrary and is necessary because of the websocket library
T: Transport + 'static, T: Transport + 'static,
// TODO: this Send is pretty arbitrary and is necessary because of the websocket library // TODO: this Send is pretty arbitrary and is necessary because of the websocket library
T::RawConn: Send, T::Output: AsyncRead + AsyncWrite + Send,
{ {
type RawConn = Box<AsyncStream>; type Output = Box<AsyncStream>;
type Listener = type Listener =
stream::Map<T::Listener, fn(<T as Transport>::ListenerUpgrade) -> Self::ListenerUpgrade>; stream::Map<T::Listener, fn(<T as Transport>::ListenerUpgrade) -> Self::ListenerUpgrade>;
type ListenerUpgrade = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; type ListenerUpgrade = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
type Dial = Box<Future<Item = (Self::RawConn, Multiaddr), Error = IoError>>; type Dial = Box<Future<Item = (Self::Output, Multiaddr), Error = IoError>>;
fn listen_on( fn listen_on(
self, self,

View File

@ -88,10 +88,10 @@ extern crate stdweb;
#[cfg(not(target_os = "emscripten"))] #[cfg(not(target_os = "emscripten"))]
extern crate websocket; extern crate websocket;
#[cfg(not(target_os = "emscripten"))]
mod desktop;
#[cfg(target_os = "emscripten")] #[cfg(target_os = "emscripten")]
mod browser; mod browser;
#[cfg(not(target_os = "emscripten"))]
mod desktop;
#[cfg(target_os = "emscripten")] #[cfg(target_os = "emscripten")]
pub use self::browser::{BrowserWsConfig, BrowserWsConn}; pub use self::browser::{BrowserWsConfig, BrowserWsConn};