mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-27 00:31:35 +00:00
Switch to stable futures (#1196)
* Switch to stable futures * Remove from_fn * Fix secio * Fix core --lib tests
This commit is contained in:
@ -23,6 +23,7 @@ use futures::prelude::*;
|
||||
use libp2p_core::upgrade::{
|
||||
InboundUpgrade,
|
||||
OutboundUpgrade,
|
||||
ReadOneError,
|
||||
Negotiated
|
||||
};
|
||||
use libp2p_swarm::{
|
||||
@ -33,9 +34,8 @@ use libp2p_swarm::{
|
||||
ProtocolsHandlerUpgrErr
|
||||
};
|
||||
use smallvec::SmallVec;
|
||||
use std::{io, marker::PhantomData, time::Duration};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use wasm_timer::{Delay, Instant};
|
||||
use std::{marker::PhantomData, pin::Pin, task::Context, task::Poll, time::Duration};
|
||||
use wasm_timer::Delay;
|
||||
use void::Void;
|
||||
|
||||
/// Delay between the moment we connect and the first time we identify.
|
||||
@ -75,7 +75,7 @@ pub enum IdentifyHandlerEvent<TSubstream> {
|
||||
/// We received a request for identification.
|
||||
Identify(ReplySubstream<Negotiated<TSubstream>>),
|
||||
/// Failed to identify the remote.
|
||||
IdentificationError(ProtocolsHandlerUpgrErr<io::Error>),
|
||||
IdentificationError(ProtocolsHandlerUpgrErr<ReadOneError>),
|
||||
}
|
||||
|
||||
impl<TSubstream> IdentifyHandler<TSubstream> {
|
||||
@ -84,7 +84,7 @@ impl<TSubstream> IdentifyHandler<TSubstream> {
|
||||
IdentifyHandler {
|
||||
config: IdentifyProtocolConfig,
|
||||
events: SmallVec::new(),
|
||||
next_id: Delay::new(Instant::now() + DELAY_TO_FIRST_ID),
|
||||
next_id: Delay::new(DELAY_TO_FIRST_ID),
|
||||
keep_alive: KeepAlive::Yes,
|
||||
marker: PhantomData,
|
||||
}
|
||||
@ -93,11 +93,11 @@ impl<TSubstream> IdentifyHandler<TSubstream> {
|
||||
|
||||
impl<TSubstream> ProtocolsHandler for IdentifyHandler<TSubstream>
|
||||
where
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TSubstream: AsyncRead + AsyncWrite + Unpin + 'static,
|
||||
{
|
||||
type InEvent = Void;
|
||||
type OutEvent = IdentifyHandlerEvent<TSubstream>;
|
||||
type Error = wasm_timer::Error;
|
||||
type Error = ReadOneError;
|
||||
type Substream = TSubstream;
|
||||
type InboundProtocol = IdentifyProtocolConfig;
|
||||
type OutboundProtocol = IdentifyProtocolConfig;
|
||||
@ -134,38 +134,39 @@ where
|
||||
) {
|
||||
self.events.push(IdentifyHandlerEvent::IdentificationError(err));
|
||||
self.keep_alive = KeepAlive::No;
|
||||
self.next_id.reset(Instant::now() + TRY_AGAIN_ON_ERR);
|
||||
self.next_id.reset(TRY_AGAIN_ON_ERR);
|
||||
}
|
||||
|
||||
fn connection_keep_alive(&self) -> KeepAlive {
|
||||
self.keep_alive
|
||||
}
|
||||
|
||||
fn poll(&mut self) -> Poll<
|
||||
fn poll(&mut self, cx: &mut Context) -> Poll<
|
||||
ProtocolsHandlerEvent<
|
||||
Self::OutboundProtocol,
|
||||
Self::OutboundOpenInfo,
|
||||
IdentifyHandlerEvent<TSubstream>,
|
||||
Self::Error,
|
||||
>,
|
||||
Self::Error,
|
||||
> {
|
||||
if !self.events.is_empty() {
|
||||
return Ok(Async::Ready(ProtocolsHandlerEvent::Custom(
|
||||
return Poll::Ready(ProtocolsHandlerEvent::Custom(
|
||||
self.events.remove(0),
|
||||
)));
|
||||
));
|
||||
}
|
||||
|
||||
// Poll the future that fires when we need to identify the node again.
|
||||
match self.next_id.poll()? {
|
||||
Async::NotReady => Ok(Async::NotReady),
|
||||
Async::Ready(()) => {
|
||||
self.next_id.reset(Instant::now() + DELAY_TO_NEXT_ID);
|
||||
match Future::poll(Pin::new(&mut self.next_id), cx) {
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Ok(())) => {
|
||||
self.next_id.reset(DELAY_TO_NEXT_ID);
|
||||
let ev = ProtocolsHandlerEvent::OutboundSubstreamRequest {
|
||||
protocol: SubstreamProtocol::new(self.config.clone()),
|
||||
info: (),
|
||||
};
|
||||
Ok(Async::Ready(ev))
|
||||
Poll::Ready(ev)
|
||||
}
|
||||
Poll::Ready(Err(err)) => Poll::Ready(ProtocolsHandlerEvent::Close(err.into()))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -19,14 +19,14 @@
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use crate::handler::{IdentifyHandler, IdentifyHandlerEvent};
|
||||
use crate::protocol::{IdentifyInfo, ReplySubstream, ReplyFuture};
|
||||
use crate::protocol::{IdentifyInfo, ReplySubstream};
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::{
|
||||
ConnectedPoint,
|
||||
Multiaddr,
|
||||
PeerId,
|
||||
PublicKey,
|
||||
upgrade::{Negotiated, UpgradeError}
|
||||
upgrade::{Negotiated, ReadOneError, UpgradeError}
|
||||
};
|
||||
use libp2p_swarm::{
|
||||
NetworkBehaviour,
|
||||
@ -35,8 +35,7 @@ use libp2p_swarm::{
|
||||
ProtocolsHandler,
|
||||
ProtocolsHandlerUpgrErr
|
||||
};
|
||||
use std::{collections::HashMap, collections::VecDeque, io};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use std::{collections::HashMap, collections::VecDeque, io, pin::Pin, task::Context, task::Poll};
|
||||
use void::Void;
|
||||
|
||||
/// Network behaviour that automatically identifies nodes periodically, returns information
|
||||
@ -67,7 +66,7 @@ enum Reply<TSubstream> {
|
||||
/// The reply is being sent.
|
||||
Sending {
|
||||
peer: PeerId,
|
||||
io: ReplyFuture<Negotiated<TSubstream>>
|
||||
io: Pin<Box<dyn Future<Output = Result<(), io::Error>> + Send>>,
|
||||
}
|
||||
}
|
||||
|
||||
@ -87,7 +86,7 @@ impl<TSubstream> Identify<TSubstream> {
|
||||
|
||||
impl<TSubstream> NetworkBehaviour for Identify<TSubstream>
|
||||
where
|
||||
TSubstream: AsyncRead + AsyncWrite,
|
||||
TSubstream: AsyncRead + AsyncWrite + Unpin + Send + 'static,
|
||||
{
|
||||
type ProtocolsHandler = IdentifyHandler<TSubstream>;
|
||||
type OutEvent = IdentifyEvent;
|
||||
@ -154,15 +153,16 @@ where
|
||||
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context,
|
||||
params: &mut impl PollParameters,
|
||||
) -> Async<
|
||||
) -> Poll<
|
||||
NetworkBehaviourAction<
|
||||
<Self::ProtocolsHandler as ProtocolsHandler>::InEvent,
|
||||
Self::OutEvent,
|
||||
>,
|
||||
> {
|
||||
if let Some(event) = self.events.pop_front() {
|
||||
return Async::Ready(event);
|
||||
return Poll::Ready(event);
|
||||
}
|
||||
|
||||
if let Some(r) = self.pending_replies.pop_front() {
|
||||
@ -189,17 +189,17 @@ where
|
||||
listen_addrs: listen_addrs.clone(),
|
||||
protocols: protocols.clone(),
|
||||
};
|
||||
let io = io.send(info, &observed);
|
||||
let io = Box::pin(io.send(info, &observed));
|
||||
reply = Some(Reply::Sending { peer, io });
|
||||
}
|
||||
Some(Reply::Sending { peer, mut io }) => {
|
||||
sending += 1;
|
||||
match io.poll() {
|
||||
Ok(Async::Ready(())) => {
|
||||
match Future::poll(Pin::new(&mut io), cx) {
|
||||
Poll::Ready(Ok(())) => {
|
||||
let event = IdentifyEvent::Sent { peer_id: peer };
|
||||
return Async::Ready(NetworkBehaviourAction::GenerateEvent(event));
|
||||
return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event));
|
||||
},
|
||||
Ok(Async::NotReady) => {
|
||||
Poll::Pending => {
|
||||
self.pending_replies.push_back(Reply::Sending { peer, io });
|
||||
if sending == to_send {
|
||||
// All remaining futures are NotReady
|
||||
@ -208,12 +208,12 @@ where
|
||||
reply = self.pending_replies.pop_front();
|
||||
}
|
||||
}
|
||||
Err(err) => {
|
||||
Poll::Ready(Err(err)) => {
|
||||
let event = IdentifyEvent::Error {
|
||||
peer_id: peer,
|
||||
error: ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(err))
|
||||
error: ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(err.into()))
|
||||
};
|
||||
return Async::Ready(NetworkBehaviourAction::GenerateEvent(event));
|
||||
return Poll::Ready(NetworkBehaviourAction::GenerateEvent(event));
|
||||
},
|
||||
}
|
||||
}
|
||||
@ -222,7 +222,7 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
Async::NotReady
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
|
||||
@ -248,7 +248,7 @@ pub enum IdentifyEvent {
|
||||
/// The peer with whom the error originated.
|
||||
peer_id: PeerId,
|
||||
/// The error that occurred.
|
||||
error: ProtocolsHandlerUpgrErr<io::Error>,
|
||||
error: ProtocolsHandlerUpgrErr<ReadOneError>,
|
||||
},
|
||||
}
|
||||
|
||||
@ -326,7 +326,7 @@ mod tests {
|
||||
assert_eq!(info.agent_version, "d");
|
||||
assert!(!info.protocols.is_empty());
|
||||
assert!(info.listen_addrs.is_empty());
|
||||
return Ok(Async::Ready(()))
|
||||
return Ok(Poll::Ready(()))
|
||||
},
|
||||
Async::Ready(Some(IdentifyEvent::Sent { .. })) => (),
|
||||
Async::Ready(e) => panic!("{:?}", e),
|
||||
@ -340,7 +340,7 @@ mod tests {
|
||||
assert_eq!(info.agent_version, "b");
|
||||
assert!(!info.protocols.is_empty());
|
||||
assert_eq!(info.listen_addrs.len(), 1);
|
||||
return Ok(Async::Ready(()))
|
||||
return Ok(Poll::Ready(()))
|
||||
},
|
||||
Async::Ready(Some(IdentifyEvent::Sent { .. })) => (),
|
||||
Async::Ready(e) => panic!("{:?}", e),
|
||||
@ -348,7 +348,7 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
Ok(Async::NotReady)
|
||||
Ok(Poll::Pending)
|
||||
}))
|
||||
.unwrap();
|
||||
}
|
||||
|
@ -18,25 +18,19 @@
|
||||
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
|
||||
// DEALINGS IN THE SOFTWARE.
|
||||
|
||||
use bytes::BytesMut;
|
||||
use crate::structs_proto;
|
||||
use futures::{future::{self, FutureResult}, Async, AsyncSink, Future, Poll, Sink, Stream};
|
||||
use futures::try_ready;
|
||||
use futures::prelude::*;
|
||||
use libp2p_core::{
|
||||
Multiaddr,
|
||||
PublicKey,
|
||||
upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}
|
||||
upgrade::{self, InboundUpgrade, OutboundUpgrade, UpgradeInfo, Negotiated}
|
||||
};
|
||||
use log::{debug, trace};
|
||||
use protobuf::Message as ProtobufMessage;
|
||||
use protobuf::parse_from_bytes as protobuf_parse_from_bytes;
|
||||
use protobuf::RepeatedField;
|
||||
use std::convert::TryFrom;
|
||||
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
|
||||
use std::{fmt, iter};
|
||||
use tokio_codec::Framed;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use unsigned_varint::codec;
|
||||
use std::{fmt, io, iter, pin::Pin};
|
||||
|
||||
/// Configuration for an upgrade to the `Identify` protocol.
|
||||
#[derive(Debug, Clone)]
|
||||
@ -54,7 +48,7 @@ pub struct RemoteInfo {
|
||||
|
||||
/// The substream on which a reply is expected to be sent.
|
||||
pub struct ReplySubstream<T> {
|
||||
inner: Framed<T, codec::UviBytes<Vec<u8>>>,
|
||||
inner: T,
|
||||
}
|
||||
|
||||
impl<T> fmt::Debug for ReplySubstream<T> {
|
||||
@ -65,13 +59,15 @@ impl<T> fmt::Debug for ReplySubstream<T> {
|
||||
|
||||
impl<T> ReplySubstream<T>
|
||||
where
|
||||
T: AsyncWrite
|
||||
T: AsyncWrite + Unpin
|
||||
{
|
||||
/// Sends back the requested information on the substream.
|
||||
///
|
||||
/// Consumes the substream, returning a `ReplyFuture` that resolves
|
||||
/// when the reply has been sent on the underlying connection.
|
||||
pub fn send(self, info: IdentifyInfo, observed_addr: &Multiaddr) -> ReplyFuture<T> {
|
||||
pub fn send(mut self, info: IdentifyInfo, observed_addr: &Multiaddr)
|
||||
-> impl Future<Output = Result<(), io::Error>>
|
||||
{
|
||||
debug!("Sending identify info to client");
|
||||
trace!("Sending: {:?}", info);
|
||||
|
||||
@ -90,50 +86,15 @@ where
|
||||
message.set_observedAddr(observed_addr.to_vec());
|
||||
message.set_protocols(RepeatedField::from_vec(info.protocols));
|
||||
|
||||
let bytes = message
|
||||
.write_to_bytes()
|
||||
.expect("writing protobuf failed; should never happen");
|
||||
|
||||
ReplyFuture {
|
||||
inner: self.inner,
|
||||
item: Some(bytes),
|
||||
async move {
|
||||
let bytes = message
|
||||
.write_to_bytes()
|
||||
.expect("writing protobuf failed; should never happen");
|
||||
upgrade::write_one(&mut self.inner, &bytes).await
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Future returned by `IdentifySender::send()`. Must be processed to the end in order to send
|
||||
/// the information to the remote.
|
||||
// Note: we don't use a `futures::sink::Sink` because it requires `T` to implement `Sink`, which
|
||||
// means that we would require `T: AsyncWrite` in this struct definition. This requirement
|
||||
// would then propagate everywhere.
|
||||
#[must_use = "futures do nothing unless polled"]
|
||||
pub struct ReplyFuture<T> {
|
||||
/// The Sink where to send the data.
|
||||
inner: Framed<T, codec::UviBytes<Vec<u8>>>,
|
||||
/// Bytes to send, or `None` if we've already sent them.
|
||||
item: Option<Vec<u8>>,
|
||||
}
|
||||
|
||||
impl<T> Future for ReplyFuture<T>
|
||||
where T: AsyncWrite
|
||||
{
|
||||
type Item = ();
|
||||
type Error = IoError;
|
||||
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
if let Some(item) = self.item.take() {
|
||||
if let AsyncSink::NotReady(item) = self.inner.start_send(item)? {
|
||||
self.item = Some(item);
|
||||
return Ok(Async::NotReady);
|
||||
}
|
||||
}
|
||||
|
||||
// A call to `close()` implies flushing.
|
||||
try_ready!(self.inner.close());
|
||||
Ok(Async::Ready(()))
|
||||
}
|
||||
}
|
||||
|
||||
/// Information of a peer sent in `Identify` protocol responses.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct IdentifyInfo {
|
||||
@ -162,93 +123,60 @@ impl UpgradeInfo for IdentifyProtocolConfig {
|
||||
|
||||
impl<C> InboundUpgrade<C> for IdentifyProtocolConfig
|
||||
where
|
||||
C: AsyncRead + AsyncWrite,
|
||||
C: AsyncRead + AsyncWrite + Unpin,
|
||||
{
|
||||
type Output = ReplySubstream<Negotiated<C>>;
|
||||
type Error = IoError;
|
||||
type Future = FutureResult<Self::Output, IoError>;
|
||||
type Error = io::Error;
|
||||
type Future = future::Ready<Result<Self::Output, io::Error>>;
|
||||
|
||||
fn upgrade_inbound(self, socket: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
trace!("Upgrading inbound connection");
|
||||
let inner = Framed::new(socket, codec::UviBytes::default());
|
||||
future::ok(ReplySubstream { inner })
|
||||
future::ok(ReplySubstream { inner: socket })
|
||||
}
|
||||
}
|
||||
|
||||
impl<C> OutboundUpgrade<C> for IdentifyProtocolConfig
|
||||
where
|
||||
C: AsyncRead + AsyncWrite,
|
||||
C: AsyncRead + AsyncWrite + Unpin + 'static,
|
||||
{
|
||||
type Output = RemoteInfo;
|
||||
type Error = IoError;
|
||||
type Future = IdentifyOutboundFuture<Negotiated<C>>;
|
||||
type Error = upgrade::ReadOneError;
|
||||
type Future = Pin<Box<dyn Future<Output = Result<Self::Output, Self::Error>>>>;
|
||||
|
||||
fn upgrade_outbound(self, socket: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
IdentifyOutboundFuture {
|
||||
inner: Framed::new(socket, codec::UviBytes::<BytesMut>::default()),
|
||||
shutdown: false,
|
||||
}
|
||||
}
|
||||
}
|
||||
fn upgrade_outbound(self, mut socket: Negotiated<C>, _: Self::Info) -> Self::Future {
|
||||
Box::pin(async move {
|
||||
socket.close().await?;
|
||||
let msg = upgrade::read_one(&mut socket, 4096).await?;
|
||||
let (info, observed_addr) = match parse_proto_msg(msg) {
|
||||
Ok(v) => v,
|
||||
Err(err) => {
|
||||
debug!("Failed to parse protobuf message; error = {:?}", err);
|
||||
return Err(err.into())
|
||||
}
|
||||
};
|
||||
|
||||
/// Future returned by `OutboundUpgrade::upgrade_outbound`.
|
||||
pub struct IdentifyOutboundFuture<T> {
|
||||
inner: Framed<T, codec::UviBytes<BytesMut>>,
|
||||
/// If true, we have finished shutting down the writing part of `inner`.
|
||||
shutdown: bool,
|
||||
}
|
||||
trace!("Remote observes us as {:?}", observed_addr);
|
||||
trace!("Information received: {:?}", info);
|
||||
|
||||
impl<T> Future for IdentifyOutboundFuture<T>
|
||||
where T: AsyncRead + AsyncWrite,
|
||||
{
|
||||
type Item = RemoteInfo;
|
||||
type Error = IoError;
|
||||
|
||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
||||
if !self.shutdown {
|
||||
try_ready!(self.inner.close());
|
||||
self.shutdown = true;
|
||||
}
|
||||
|
||||
let msg = match try_ready!(self.inner.poll()) {
|
||||
Some(i) => i,
|
||||
None => {
|
||||
debug!("Identify protocol stream closed before receiving info");
|
||||
return Err(IoErrorKind::InvalidData.into());
|
||||
}
|
||||
};
|
||||
|
||||
debug!("Received identify message");
|
||||
|
||||
let (info, observed_addr) = match parse_proto_msg(msg) {
|
||||
Ok(v) => v,
|
||||
Err(err) => {
|
||||
debug!("Failed to parse protobuf message; error = {:?}", err);
|
||||
return Err(err)
|
||||
}
|
||||
};
|
||||
|
||||
trace!("Remote observes us as {:?}", observed_addr);
|
||||
trace!("Information received: {:?}", info);
|
||||
|
||||
Ok(Async::Ready(RemoteInfo {
|
||||
info,
|
||||
observed_addr: observed_addr.clone(),
|
||||
_priv: ()
|
||||
}))
|
||||
Ok(RemoteInfo {
|
||||
info,
|
||||
observed_addr: observed_addr.clone(),
|
||||
_priv: ()
|
||||
})
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// Turns a protobuf message into an `IdentifyInfo` and an observed address. If something bad
|
||||
// happens, turn it into an `IoError`.
|
||||
fn parse_proto_msg(msg: BytesMut) -> Result<(IdentifyInfo, Multiaddr), IoError> {
|
||||
match protobuf_parse_from_bytes::<structs_proto::Identify>(&msg) {
|
||||
// happens, turn it into an `io::Error`.
|
||||
fn parse_proto_msg(msg: impl AsRef<[u8]>) -> Result<(IdentifyInfo, Multiaddr), io::Error> {
|
||||
match protobuf_parse_from_bytes::<structs_proto::Identify>(msg.as_ref()) {
|
||||
Ok(mut msg) => {
|
||||
// Turn a `Vec<u8>` into a `Multiaddr`. If something bad happens, turn it into
|
||||
// an `IoError`.
|
||||
fn bytes_to_multiaddr(bytes: Vec<u8>) -> Result<Multiaddr, IoError> {
|
||||
// an `io::Error`.
|
||||
fn bytes_to_multiaddr(bytes: Vec<u8>) -> Result<Multiaddr, io::Error> {
|
||||
Multiaddr::try_from(bytes)
|
||||
.map_err(|err| IoError::new(IoErrorKind::InvalidData, err))
|
||||
.map_err(|err| io::Error::new(io::ErrorKind::InvalidData, err))
|
||||
}
|
||||
|
||||
let listen_addrs = {
|
||||
@ -260,7 +188,7 @@ fn parse_proto_msg(msg: BytesMut) -> Result<(IdentifyInfo, Multiaddr), IoError>
|
||||
};
|
||||
|
||||
let public_key = PublicKey::from_protobuf_encoding(msg.get_publicKey())
|
||||
.map_err(|e| IoError::new(IoErrorKind::InvalidData, e))?;
|
||||
.map_err(|e| io::Error::new(io::ErrorKind::InvalidData, e))?;
|
||||
|
||||
let observed_addr = bytes_to_multiaddr(msg.take_observedAddr())?;
|
||||
let info = IdentifyInfo {
|
||||
@ -274,7 +202,7 @@ fn parse_proto_msg(msg: BytesMut) -> Result<(IdentifyInfo, Multiaddr), IoError>
|
||||
Ok((info, observed_addr))
|
||||
}
|
||||
|
||||
Err(err) => Err(IoError::new(IoErrorKind::InvalidData, err)),
|
||||
Err(err) => Err(io::Error::new(io::ErrorKind::InvalidData, err)),
|
||||
}
|
||||
}
|
||||
|
||||
|
Reference in New Issue
Block a user