mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-13 01:51:23 +00:00
Wrap multistream-select streams under a Negotiated (#1001)
This commit is contained in:
@ -53,10 +53,10 @@ where
|
||||
{
|
||||
type Output = FloodsubRpc;
|
||||
type Error = FloodsubDecodeError;
|
||||
type Future = upgrade::ReadOneThen<TSocket, (), fn(Vec<u8>, ()) -> Result<FloodsubRpc, FloodsubDecodeError>>;
|
||||
type Future = upgrade::ReadOneThen<upgrade::Negotiated<TSocket>, (), fn(Vec<u8>, ()) -> Result<FloodsubRpc, FloodsubDecodeError>>;
|
||||
|
||||
#[inline]
|
||||
fn upgrade_inbound(self, socket: TSocket, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: upgrade::Negotiated<TSocket>, _: Self::Info) -> Self::Future {
|
||||
upgrade::read_one_then(socket, 2048, (), |packet, ()| {
|
||||
let mut rpc: rpc_proto::RPC = protobuf::parse_from_bytes(&packet)?;
|
||||
|
||||
@ -168,10 +168,10 @@ where
|
||||
{
|
||||
type Output = ();
|
||||
type Error = io::Error;
|
||||
type Future = upgrade::WriteOne<TSocket>;
|
||||
type Future = upgrade::WriteOne<upgrade::Negotiated<TSocket>>;
|
||||
|
||||
#[inline]
|
||||
fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, socket: upgrade::Negotiated<TSocket>, _: Self::Info) -> Self::Future {
|
||||
let bytes = self.into_bytes();
|
||||
upgrade::write_one(socket, bytes)
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ use crate::protocol::{IdentifyInfo, IdentifySender, IdentifySenderFuture};
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::protocols_handler::{ProtocolsHandler, ProtocolsHandlerSelect, ProtocolsHandlerUpgrErr};
|
||||
use libp2p_core::swarm::{ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters};
|
||||
use libp2p_core::{Multiaddr, PeerId, PublicKey, either::EitherOutput};
|
||||
use libp2p_core::{Multiaddr, PeerId, PublicKey, either::EitherOutput, upgrade::Negotiated};
|
||||
use smallvec::SmallVec;
|
||||
use std::{collections::HashMap, collections::VecDeque, io};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
@ -42,9 +42,9 @@ pub struct Identify<TSubstream> {
|
||||
/// For each peer we're connected to, the observed address to send back to it.
|
||||
observed_addresses: HashMap<PeerId, Multiaddr>,
|
||||
/// List of senders to answer, with the observed multiaddr.
|
||||
to_answer: SmallVec<[(PeerId, IdentifySender<TSubstream>, Multiaddr); 4]>,
|
||||
to_answer: SmallVec<[(PeerId, IdentifySender<Negotiated<TSubstream>>, Multiaddr); 4]>,
|
||||
/// List of futures that send back information back to remotes.
|
||||
futures: SmallVec<[(PeerId, IdentifySenderFuture<TSubstream>); 4]>,
|
||||
futures: SmallVec<[(PeerId, IdentifySenderFuture<Negotiated<TSubstream>>); 4]>,
|
||||
/// Events that need to be produced outside when polling..
|
||||
events: VecDeque<NetworkBehaviourAction<EitherOutput<Void, Void>, IdentifyEvent>>,
|
||||
}
|
||||
|
@ -22,7 +22,7 @@ use crate::protocol::{IdentifySender, IdentifyProtocolConfig};
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::{
|
||||
protocols_handler::{KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr},
|
||||
upgrade::{DeniedUpgrade, InboundUpgrade, OutboundUpgrade}
|
||||
upgrade::{DeniedUpgrade, InboundUpgrade, OutboundUpgrade, Negotiated}
|
||||
};
|
||||
use smallvec::SmallVec;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
@ -34,7 +34,7 @@ pub struct IdentifyListenHandler<TSubstream> {
|
||||
config: IdentifyProtocolConfig,
|
||||
|
||||
/// List of senders to yield to the user.
|
||||
pending_result: SmallVec<[IdentifySender<TSubstream>; 4]>,
|
||||
pending_result: SmallVec<[IdentifySender<Negotiated<TSubstream>>; 4]>,
|
||||
}
|
||||
|
||||
impl<TSubstream> IdentifyListenHandler<TSubstream> {
|
||||
@ -53,7 +53,7 @@ where
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
{
|
||||
type InEvent = Void;
|
||||
type OutEvent = IdentifySender<TSubstream>;
|
||||
type OutEvent = IdentifySender<Negotiated<TSubstream>>;
|
||||
type Error = Void;
|
||||
type Substream = TSubstream;
|
||||
type InboundProtocol = IdentifyProtocolConfig;
|
||||
|
@ -24,7 +24,7 @@ use futures::{future::{self, FutureResult}, Async, AsyncSink, Future, Poll, Sink
|
||||
use futures::try_ready;
|
||||
use libp2p_core::{
|
||||
Multiaddr, PublicKey,
|
||||
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}
|
||||
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}
|
||||
};
|
||||
use log::{debug, trace};
|
||||
use protobuf::Message as ProtobufMessage;
|
||||
@ -150,11 +150,11 @@ impl<C> InboundUpgrade<C> for IdentifyProtocolConfig
|
||||
where
|
||||
C: AsyncRead + AsyncWrite,
|
||||
{
|
||||
type Output = IdentifySender<C>;
|
||||
type Output = IdentifySender<Negotiated<C>>;
|
||||
type Error = IoError;
|
||||
type Future = FutureResult<Self::Output, IoError>;
|
||||
|
||||
fn upgrade_inbound(self, socket: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
trace!("Upgrading inbound connection");
|
||||
let socket = Framed::new(socket, codec::UviBytes::default());
|
||||
let sender = IdentifySender { inner: socket };
|
||||
@ -168,9 +168,9 @@ where
|
||||
{
|
||||
type Output = RemoteInfo;
|
||||
type Error = IoError;
|
||||
type Future = IdentifyOutboundFuture<C>;
|
||||
type Future = IdentifyOutboundFuture<Negotiated<C>>;
|
||||
|
||||
fn upgrade_outbound(self, socket: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, socket: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
IdentifyOutboundFuture {
|
||||
inner: Framed::new(socket, codec::UviBytes::<BytesMut>::default()),
|
||||
shutdown: false,
|
||||
|
@ -24,7 +24,7 @@ use crate::protocol::{
|
||||
};
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::protocols_handler::{KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr};
|
||||
use libp2p_core::{upgrade, either::EitherOutput, InboundUpgrade, OutboundUpgrade, PeerId};
|
||||
use libp2p_core::{upgrade, either::EitherOutput, InboundUpgrade, OutboundUpgrade, PeerId, upgrade::Negotiated};
|
||||
use multihash::Multihash;
|
||||
use std::{error, fmt, io, time::Duration, time::Instant};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
@ -49,7 +49,7 @@ where
|
||||
next_connec_unique_id: UniqueConnecId,
|
||||
|
||||
/// List of active substreams with the state they are in.
|
||||
substreams: Vec<SubstreamState<TSubstream, TUserData>>,
|
||||
substreams: Vec<SubstreamState<Negotiated<TSubstream>, TUserData>>,
|
||||
|
||||
/// Until when to keep the connection alive.
|
||||
keep_alive: KeepAlive,
|
||||
|
@ -29,7 +29,7 @@
|
||||
use bytes::BytesMut;
|
||||
use crate::protobuf_structs;
|
||||
use futures::{future, sink, stream, Sink, Stream};
|
||||
use libp2p_core::{InboundUpgrade, Multiaddr, OutboundUpgrade, PeerId, UpgradeInfo};
|
||||
use libp2p_core::{InboundUpgrade, Multiaddr, OutboundUpgrade, PeerId, UpgradeInfo, upgrade::Negotiated};
|
||||
use multihash::Multihash;
|
||||
use protobuf::{self, Message};
|
||||
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
|
||||
@ -153,12 +153,12 @@ impl<C> InboundUpgrade<C> for KademliaProtocolConfig
|
||||
where
|
||||
C: AsyncRead + AsyncWrite,
|
||||
{
|
||||
type Output = KadInStreamSink<C>;
|
||||
type Output = KadInStreamSink<Negotiated<C>>;
|
||||
type Future = future::FutureResult<Self::Output, IoError>;
|
||||
type Error = IoError;
|
||||
|
||||
#[inline]
|
||||
fn upgrade_inbound(self, incoming: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, incoming: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
let mut codec = codec::UviBytes::default();
|
||||
codec.set_max_len(4096);
|
||||
|
||||
@ -182,12 +182,12 @@ impl<C> OutboundUpgrade<C> for KademliaProtocolConfig
|
||||
where
|
||||
C: AsyncRead + AsyncWrite,
|
||||
{
|
||||
type Output = KadOutStreamSink<C>;
|
||||
type Output = KadOutStreamSink<Negotiated<C>>;
|
||||
type Future = future::FutureResult<Self::Output, IoError>;
|
||||
type Error = IoError;
|
||||
|
||||
#[inline]
|
||||
fn upgrade_outbound(self, incoming: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, incoming: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
let mut codec = codec::UviBytes::default();
|
||||
codec.set_max_len(4096);
|
||||
|
||||
|
@ -58,7 +58,7 @@ pub use io::NoiseOutput;
|
||||
pub use protocol::{Keypair, PublicKey, Protocol, ProtocolParams, IX, IK, XX};
|
||||
pub use protocol::x25519::X25519;
|
||||
|
||||
use libp2p_core::{UpgradeInfo, InboundUpgrade, OutboundUpgrade};
|
||||
use libp2p_core::{UpgradeInfo, InboundUpgrade, OutboundUpgrade, upgrade::Negotiated};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use zeroize::Zeroize;
|
||||
|
||||
@ -127,11 +127,11 @@ where
|
||||
NoiseConfig<IX, C>: UpgradeInfo,
|
||||
C: Protocol<C> + AsRef<[u8]> + Zeroize
|
||||
{
|
||||
type Output = (PublicKey<C>, NoiseOutput<T>);
|
||||
type Output = (PublicKey<C>, NoiseOutput<Negotiated<T>>);
|
||||
type Error = NoiseError;
|
||||
type Future = rt1::NoiseInboundFuture<T, C>;
|
||||
type Future = rt1::NoiseInboundFuture<Negotiated<T>, C>;
|
||||
|
||||
fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
let session = self.params.into_builder()
|
||||
.local_private_key(self.keys.secret().as_ref())
|
||||
.build_responder()
|
||||
@ -146,11 +146,11 @@ where
|
||||
NoiseConfig<IX, C>: UpgradeInfo,
|
||||
C: Protocol<C> + AsRef<[u8]> + Zeroize
|
||||
{
|
||||
type Output = (PublicKey<C>, NoiseOutput<T>);
|
||||
type Output = (PublicKey<C>, NoiseOutput<Negotiated<T>>);
|
||||
type Error = NoiseError;
|
||||
type Future = rt1::NoiseOutboundFuture<T, C>;
|
||||
type Future = rt1::NoiseOutboundFuture<Negotiated<T>, C>;
|
||||
|
||||
fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
let session = self.params.into_builder()
|
||||
.local_private_key(self.keys.secret().as_ref())
|
||||
.build_initiator()
|
||||
@ -167,11 +167,11 @@ where
|
||||
NoiseConfig<XX, C>: UpgradeInfo,
|
||||
C: Protocol<C> + AsRef<[u8]> + Zeroize
|
||||
{
|
||||
type Output = (PublicKey<C>, NoiseOutput<T>);
|
||||
type Output = (PublicKey<C>, NoiseOutput<Negotiated<T>>);
|
||||
type Error = NoiseError;
|
||||
type Future = rt15::NoiseInboundFuture<T, C>;
|
||||
type Future = rt15::NoiseInboundFuture<Negotiated<T>, C>;
|
||||
|
||||
fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
let session = self.params.into_builder()
|
||||
.local_private_key(self.keys.secret().as_ref())
|
||||
.build_responder()
|
||||
@ -186,11 +186,11 @@ where
|
||||
NoiseConfig<XX, C>: UpgradeInfo,
|
||||
C: Protocol<C> + AsRef<[u8]> + Zeroize
|
||||
{
|
||||
type Output = (PublicKey<C>, NoiseOutput<T>);
|
||||
type Output = (PublicKey<C>, NoiseOutput<Negotiated<T>>);
|
||||
type Error = NoiseError;
|
||||
type Future = rt15::NoiseOutboundFuture<T, C>;
|
||||
type Future = rt15::NoiseOutboundFuture<Negotiated<T>, C>;
|
||||
|
||||
fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
let session = self.params.into_builder()
|
||||
.local_private_key(self.keys.secret().as_ref())
|
||||
.build_initiator()
|
||||
@ -207,11 +207,11 @@ where
|
||||
NoiseConfig<IK, C>: UpgradeInfo,
|
||||
C: Protocol<C> + AsRef<[u8]> + Zeroize
|
||||
{
|
||||
type Output = (PublicKey<C>, NoiseOutput<T>);
|
||||
type Output = (PublicKey<C>, NoiseOutput<Negotiated<T>>);
|
||||
type Error = NoiseError;
|
||||
type Future = rt1::NoiseInboundFuture<T, C>;
|
||||
type Future = rt1::NoiseInboundFuture<Negotiated<T>, C>;
|
||||
|
||||
fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
let session = self.params.into_builder()
|
||||
.local_private_key(self.keys.secret().as_ref())
|
||||
.build_responder()
|
||||
@ -226,11 +226,11 @@ where
|
||||
NoiseConfig<IK, C, PublicKey<C>>: UpgradeInfo,
|
||||
C: Protocol<C> + AsRef<[u8]> + Zeroize
|
||||
{
|
||||
type Output = (PublicKey<C>, NoiseOutput<T>);
|
||||
type Output = (PublicKey<C>, NoiseOutput<Negotiated<T>>);
|
||||
type Error = NoiseError;
|
||||
type Future = rt1::NoiseOutboundFuture<T, C>;
|
||||
type Future = rt1::NoiseOutboundFuture<Negotiated<T>, C>;
|
||||
|
||||
fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
let session = self.params.into_builder()
|
||||
.local_private_key(self.keys.secret().as_ref())
|
||||
.remote_public_key(self.remote.as_ref())
|
||||
|
@ -23,7 +23,7 @@
|
||||
|
||||
use bytes::Bytes;
|
||||
use futures::{future, prelude::*};
|
||||
use libp2p_core::{Multiaddr, upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}};
|
||||
use libp2p_core::{Multiaddr, upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}};
|
||||
use std::{io, iter};
|
||||
use tokio_codec::{FramedRead, FramedWrite};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
@ -51,11 +51,11 @@ impl<C> InboundUpgrade<C> for Observed
|
||||
where
|
||||
C: AsyncRead + AsyncWrite + Send + 'static
|
||||
{
|
||||
type Output = Sender<C>;
|
||||
type Output = Sender<Negotiated<C>>;
|
||||
type Error = io::Error;
|
||||
type Future = Box<dyn Future<Item=Self::Output, Error=Self::Error> + Send>;
|
||||
|
||||
fn upgrade_inbound(self, conn: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, conn: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
let io = FramedWrite::new(conn, UviBytes::default());
|
||||
Box::new(future::ok(Sender { io }))
|
||||
}
|
||||
@ -69,10 +69,10 @@ where
|
||||
type Error = io::Error;
|
||||
type Future = Box<dyn Future<Item=Self::Output, Error=Self::Error> + Send>;
|
||||
|
||||
fn upgrade_outbound(self, conn: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, conn: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
let io = FramedRead::new(conn, UviBytes::default());
|
||||
let future = io.into_future()
|
||||
.map_err(|(e, _): (io::Error, FramedRead<C, UviBytes>)| e)
|
||||
.map_err(|(e, _): (io::Error, FramedRead<Negotiated<C>, UviBytes>)| e)
|
||||
.and_then(move |(bytes, _)| {
|
||||
if let Some(b) = bytes {
|
||||
let ma = Multiaddr::from_bytes(b.to_vec())
|
||||
@ -100,7 +100,7 @@ impl<C: AsyncWrite> Sender<C> {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use libp2p_core::{Multiaddr, upgrade::{InboundUpgrade, OutboundUpgrade}};
|
||||
use libp2p_core::{Multiaddr, upgrade::{apply_inbound, apply_outbound}};
|
||||
use tokio::runtime::current_thread;
|
||||
use tokio::net::{TcpListener, TcpStream};
|
||||
use super::*;
|
||||
@ -115,17 +115,19 @@ mod tests {
|
||||
|
||||
let server = server.incoming()
|
||||
.into_future()
|
||||
.map_err(|(e, _)| e.into())
|
||||
.map_err(|_| panic!())
|
||||
.and_then(move |(conn, _)| {
|
||||
Observed::new().upgrade_inbound(conn.unwrap(), b"/paritytech/observed-address/0.1.0")
|
||||
apply_inbound(conn.unwrap(), Observed::new())
|
||||
})
|
||||
.map_err(|_| panic!())
|
||||
.and_then(move |sender| sender.send_address(observed_addr1));
|
||||
|
||||
let client = TcpStream::connect(&server_addr)
|
||||
.map_err(|e| e.into())
|
||||
.map_err(|_| panic!())
|
||||
.and_then(|conn| {
|
||||
Observed::new().upgrade_outbound(conn, b"/paritytech/observed-address/0.1.0")
|
||||
apply_outbound(conn, Observed::new())
|
||||
})
|
||||
.map_err(|_| panic!())
|
||||
.map(move |addr| {
|
||||
eprintln!("{} {}", addr, observed_addr2);
|
||||
assert_eq!(addr, observed_addr2)
|
||||
@ -133,7 +135,7 @@ mod tests {
|
||||
|
||||
current_thread::block_on_all(future::lazy(move || {
|
||||
current_thread::spawn(server.map_err(|e| panic!("server error: {}", e)).map(|_| ()));
|
||||
client.map_err(|e| panic!("client error: {}", e))
|
||||
client
|
||||
}))
|
||||
.unwrap();
|
||||
}
|
||||
|
@ -19,7 +19,7 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use futures::{prelude::*, future, try_ready};
|
||||
use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
||||
use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, upgrade::Negotiated};
|
||||
use log::debug;
|
||||
use rand::{distributions::Standard, prelude::*, rngs::EntropyRng};
|
||||
use std::{io, iter, time::Duration, time::Instant};
|
||||
@ -53,10 +53,10 @@ where
|
||||
{
|
||||
type Output = ();
|
||||
type Error = io::Error;
|
||||
type Future = future::Map<future::AndThen<future::AndThen<future::AndThen<tokio_io::io::ReadExact<TSocket, [u8; 32]>, tokio_io::io::WriteAll<TSocket, [u8; 32]>, fn((TSocket, [u8; 32])) -> tokio_io::io::WriteAll<TSocket, [u8; 32]>>, tokio_io::io::Flush<TSocket>, fn((TSocket, [u8; 32])) -> tokio_io::io::Flush<TSocket>>, tokio_io::io::Shutdown<TSocket>, fn(TSocket) -> tokio_io::io::Shutdown<TSocket>>, fn(TSocket) -> ()>;
|
||||
type Future = future::Map<future::AndThen<future::AndThen<future::AndThen<tokio_io::io::ReadExact<Negotiated<TSocket>, [u8; 32]>, tokio_io::io::WriteAll<Negotiated<TSocket>, [u8; 32]>, fn((Negotiated<TSocket>, [u8; 32])) -> tokio_io::io::WriteAll<Negotiated<TSocket>, [u8; 32]>>, tokio_io::io::Flush<Negotiated<TSocket>>, fn((Negotiated<TSocket>, [u8; 32])) -> tokio_io::io::Flush<Negotiated<TSocket>>>, tokio_io::io::Shutdown<Negotiated<TSocket>>, fn(Negotiated<TSocket>) -> tokio_io::io::Shutdown<Negotiated<TSocket>>>, fn(Negotiated<TSocket>) -> ()>;
|
||||
|
||||
#[inline]
|
||||
fn upgrade_inbound(self, socket: TSocket, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: Negotiated<TSocket>, _: Self::Info) -> Self::Future {
|
||||
tokio_io::io::read_exact(socket, [0; 32])
|
||||
.and_then::<fn(_) -> _, _>(|(socket, buffer)| tokio_io::io::write_all(socket, buffer))
|
||||
.and_then::<fn(_) -> _, _>(|(socket, _)| tokio_io::io::flush(socket))
|
||||
@ -71,10 +71,10 @@ where
|
||||
{
|
||||
type Output = Duration;
|
||||
type Error = io::Error;
|
||||
type Future = PingDialer<TSocket>;
|
||||
type Future = PingDialer<Negotiated<TSocket>>;
|
||||
|
||||
#[inline]
|
||||
fn upgrade_outbound(self, socket: TSocket, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, socket: Negotiated<TSocket>, _: Self::Info) -> Self::Future {
|
||||
let payload: [u8; 32] = EntropyRng::default().sample(Standard);
|
||||
debug!("Preparing for ping with payload {:?}", payload);
|
||||
|
||||
|
@ -19,7 +19,7 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use futures::future::{self, FutureResult};
|
||||
use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
||||
use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, upgrade::Negotiated};
|
||||
use std::iter;
|
||||
use void::Void;
|
||||
|
||||
@ -36,21 +36,21 @@ impl UpgradeInfo for PlainTextConfig {
|
||||
}
|
||||
|
||||
impl<C> InboundUpgrade<C> for PlainTextConfig {
|
||||
type Output = C;
|
||||
type Output = Negotiated<C>;
|
||||
type Error = Void;
|
||||
type Future = FutureResult<C, Self::Error>;
|
||||
type Future = FutureResult<Negotiated<C>, Self::Error>;
|
||||
|
||||
fn upgrade_inbound(self, i: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, i: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
future::ok(i)
|
||||
}
|
||||
}
|
||||
|
||||
impl<C> OutboundUpgrade<C> for PlainTextConfig {
|
||||
type Output = C;
|
||||
type Output = Negotiated<C>;
|
||||
type Error = Void;
|
||||
type Future = FutureResult<C, Self::Error>;
|
||||
type Future = FutureResult<Negotiated<C>, Self::Error>;
|
||||
|
||||
fn upgrade_outbound(self, i: C, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, i: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
future::ok(i)
|
||||
}
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ pub use self::error::SecioError;
|
||||
use bytes::BytesMut;
|
||||
use futures::stream::MapErr as StreamMapErr;
|
||||
use futures::{Future, Poll, Sink, StartSend, Stream};
|
||||
use libp2p_core::{PublicKey, identity, upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade}};
|
||||
use libp2p_core::{PublicKey, identity, upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade, Negotiated}};
|
||||
use log::debug;
|
||||
use rw_stream_sink::RwStreamSink;
|
||||
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
|
||||
@ -195,11 +195,11 @@ impl<T> InboundUpgrade<T> for SecioConfig
|
||||
where
|
||||
T: AsyncRead + AsyncWrite + Send + 'static
|
||||
{
|
||||
type Output = SecioOutput<T>;
|
||||
type Output = SecioOutput<Negotiated<T>>;
|
||||
type Error = SecioError;
|
||||
type Future = Box<dyn Future<Item = Self::Output, Error = Self::Error> + Send>;
|
||||
|
||||
fn upgrade_inbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_inbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
Box::new(self.handshake(socket))
|
||||
}
|
||||
}
|
||||
@ -208,11 +208,11 @@ impl<T> OutboundUpgrade<T> for SecioConfig
|
||||
where
|
||||
T: AsyncRead + AsyncWrite + Send + 'static
|
||||
{
|
||||
type Output = SecioOutput<T>;
|
||||
type Output = SecioOutput<Negotiated<T>>;
|
||||
type Error = SecioError;
|
||||
type Future = Box<dyn Future<Item = Self::Output, Error = Self::Error> + Send>;
|
||||
|
||||
fn upgrade_outbound(self, socket: T, _: Self::Info) -> Self::Future {
|
||||
fn upgrade_outbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
|
||||
Box::new(self.handshake(socket))
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user