mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-05-20 22:51:20 +00:00
Remove several unused transports. (#673)
This commit is contained in:
parent
f833c5d9ff
commit
d130302e15
@ -1,51 +0,0 @@
|
|||||||
// 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::{self, Cursor};
|
|
||||||
use transport::Transport;
|
|
||||||
|
|
||||||
/// Dummy implementation of `Transport` that just denies every single attempt.
|
|
||||||
#[derive(Debug, Copy, Clone)]
|
|
||||||
pub struct DeniedTransport;
|
|
||||||
|
|
||||||
impl Transport for DeniedTransport {
|
|
||||||
// TODO: could use `!` for associated types once stable
|
|
||||||
type Output = Cursor<Vec<u8>>;
|
|
||||||
type Listener = Box<Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = io::Error> + Send + Sync>;
|
|
||||||
type ListenerUpgrade = Box<Future<Item = Self::Output, Error = io::Error> + Send + Sync>;
|
|
||||||
type Dial = Box<Future<Item = Self::Output, Error = io::Error> + Send + Sync>;
|
|
||||||
|
|
||||||
#[inline]
|
|
||||||
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
|
|
||||||
Err((DeniedTransport, addr))
|
|
||||||
}
|
|
||||||
|
|
||||||
#[inline]
|
|
||||||
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
|
|
||||||
Err((DeniedTransport, addr))
|
|
||||||
}
|
|
||||||
|
|
||||||
#[inline]
|
|
||||||
fn nat_traversal(&self, _: &Multiaddr, _: &Multiaddr) -> Option<Multiaddr> {
|
|
||||||
None
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,108 +0,0 @@
|
|||||||
// 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::*, sync::oneshot};
|
|
||||||
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
|
|
||||||
use transport::Transport;
|
|
||||||
use Multiaddr;
|
|
||||||
|
|
||||||
/// See `Transport::interruptible`.
|
|
||||||
#[derive(Debug, Clone)]
|
|
||||||
pub struct Interruptible<T> {
|
|
||||||
transport: T,
|
|
||||||
rx: future::Shared<oneshot::Receiver<()>>,
|
|
||||||
}
|
|
||||||
|
|
||||||
impl<T> Interruptible<T> {
|
|
||||||
/// Internal function that builds an `Interruptible`.
|
|
||||||
#[inline]
|
|
||||||
pub(crate) fn new(transport: T) -> (Interruptible<T>, Interrupt) {
|
|
||||||
let (_tx, rx) = oneshot::channel();
|
|
||||||
let transport = Interruptible { transport, rx: rx.shared() };
|
|
||||||
let int = Interrupt { _tx };
|
|
||||||
(transport, int)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
impl<T> Transport for Interruptible<T>
|
|
||||||
where
|
|
||||||
T: Transport,
|
|
||||||
{
|
|
||||||
type Output = T::Output;
|
|
||||||
type Listener = T::Listener;
|
|
||||||
type ListenerUpgrade = T::ListenerUpgrade;
|
|
||||||
type Dial = InterruptibleDial<T::Dial>;
|
|
||||||
|
|
||||||
#[inline]
|
|
||||||
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
|
|
||||||
match self.transport.listen_on(addr) {
|
|
||||||
Ok(val) => Ok(val),
|
|
||||||
Err((transport, addr)) => Err((Interruptible { transport, rx: self.rx }, addr)),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
#[inline]
|
|
||||||
fn dial(self, addr: Multiaddr) -> Result<Self::Dial, (Self, Multiaddr)> {
|
|
||||||
match self.transport.dial(addr) {
|
|
||||||
Ok(future) => {
|
|
||||||
Ok(InterruptibleDial {
|
|
||||||
inner: future,
|
|
||||||
rx: self.rx,
|
|
||||||
})
|
|
||||||
}
|
|
||||||
Err((transport, addr)) => Err((Interruptible { transport, rx: self.rx }, addr)),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
#[inline]
|
|
||||||
fn nat_traversal(&self, server: &Multiaddr, observed: &Multiaddr) -> Option<Multiaddr> {
|
|
||||||
self.transport.nat_traversal(server, observed)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Dropping this object interrupts the dialing of the corresponding `Interruptible`.
|
|
||||||
pub struct Interrupt {
|
|
||||||
_tx: oneshot::Sender<()>,
|
|
||||||
}
|
|
||||||
|
|
||||||
#[must_use = "futures do nothing unless polled"]
|
|
||||||
pub struct InterruptibleDial<F> {
|
|
||||||
inner: F,
|
|
||||||
rx: future::Shared<oneshot::Receiver<()>>,
|
|
||||||
}
|
|
||||||
|
|
||||||
impl<F> Future for InterruptibleDial<F>
|
|
||||||
where F: Future<Error = IoError>
|
|
||||||
{
|
|
||||||
type Item = F::Item;
|
|
||||||
type Error = IoError;
|
|
||||||
|
|
||||||
#[inline]
|
|
||||||
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
|
|
||||||
match self.rx.poll() {
|
|
||||||
Ok(Async::Ready(_)) | Err(_) => {
|
|
||||||
return Err(IoError::new(IoErrorKind::ConnectionAborted, "connection interrupted"));
|
|
||||||
},
|
|
||||||
Ok(Async::NotReady) => (),
|
|
||||||
};
|
|
||||||
|
|
||||||
self.inner.poll()
|
|
||||||
}
|
|
||||||
}
|
|
@ -29,18 +29,15 @@
|
|||||||
//! `UpgradedNode::or_upgrade` methods, you can combine multiple transports and/or upgrades
|
//! `UpgradedNode::or_upgrade` methods, you can combine multiple transports and/or upgrades
|
||||||
//! together in a complex chain of protocols negotiation.
|
//! together in a complex chain of protocols negotiation.
|
||||||
|
|
||||||
use crate::{InboundUpgrade, OutboundUpgrade, Endpoint};
|
use crate::{InboundUpgrade, OutboundUpgrade, Endpoint, nodes::raw_swarm::ConnectedPoint};
|
||||||
use futures::prelude::*;
|
use futures::prelude::*;
|
||||||
use multiaddr::Multiaddr;
|
use multiaddr::Multiaddr;
|
||||||
use nodes::raw_swarm::ConnectedPoint;
|
|
||||||
use std::io::Error as IoError;
|
use std::io::Error as IoError;
|
||||||
use tokio_io::{AsyncRead, AsyncWrite};
|
use tokio_io::{AsyncRead, AsyncWrite};
|
||||||
|
|
||||||
pub mod and_then;
|
pub mod and_then;
|
||||||
pub mod boxed;
|
pub mod boxed;
|
||||||
pub mod choice;
|
pub mod choice;
|
||||||
pub mod denied;
|
|
||||||
pub mod interruptible;
|
|
||||||
pub mod map;
|
pub mod map;
|
||||||
pub mod map_err;
|
pub mod map_err;
|
||||||
pub mod map_err_dial;
|
pub mod map_err_dial;
|
||||||
@ -48,7 +45,6 @@ pub mod memory;
|
|||||||
pub mod upgrade;
|
pub mod upgrade;
|
||||||
|
|
||||||
pub use self::choice::OrTransport;
|
pub use self::choice::OrTransport;
|
||||||
pub use self::denied::DeniedTransport;
|
|
||||||
pub use self::memory::connector;
|
pub use self::memory::connector;
|
||||||
pub use self::upgrade::Upgrade;
|
pub use self::upgrade::Upgrade;
|
||||||
|
|
||||||
@ -202,13 +198,4 @@ pub trait Transport {
|
|||||||
{
|
{
|
||||||
and_then::and_then(self, upgrade)
|
and_then::and_then(self, upgrade)
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Wraps around the `Transport` and makes it interruptible.
|
|
||||||
#[inline]
|
|
||||||
fn interruptible(self) -> (interruptible::Interruptible<Self>, interruptible::Interrupt)
|
|
||||||
where
|
|
||||||
Self: Sized,
|
|
||||||
{
|
|
||||||
interruptible::Interruptible::new(self)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -20,7 +20,7 @@
|
|||||||
|
|
||||||
use bytes::Bytes;
|
use bytes::Bytes;
|
||||||
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
use crate::upgrade::{InboundUpgrade, OutboundUpgrade, UpgradeInfo};
|
||||||
use futures::future::FutureResult;
|
use futures::future;
|
||||||
use std::iter;
|
use std::iter;
|
||||||
use void::{unreachable, Void};
|
use void::{unreachable, Void};
|
||||||
|
|
||||||
@ -41,7 +41,7 @@ impl UpgradeInfo for DeniedUpgrade {
|
|||||||
impl<C> InboundUpgrade<C> for DeniedUpgrade {
|
impl<C> InboundUpgrade<C> for DeniedUpgrade {
|
||||||
type Output = Void;
|
type Output = Void;
|
||||||
type Error = Void;
|
type Error = Void;
|
||||||
type Future = FutureResult<Self::Output, Self::Error>;
|
type Future = future::Empty<Self::Output, Self::Error>;
|
||||||
|
|
||||||
fn upgrade_inbound(self, _: C, id: Self::UpgradeId) -> Self::Future {
|
fn upgrade_inbound(self, _: C, id: Self::UpgradeId) -> Self::Future {
|
||||||
unreachable(id)
|
unreachable(id)
|
||||||
@ -51,7 +51,7 @@ impl<C> InboundUpgrade<C> for DeniedUpgrade {
|
|||||||
impl<C> OutboundUpgrade<C> for DeniedUpgrade {
|
impl<C> OutboundUpgrade<C> for DeniedUpgrade {
|
||||||
type Output = Void;
|
type Output = Void;
|
||||||
type Error = Void;
|
type Error = Void;
|
||||||
type Future = FutureResult<Self::Output, Self::Error>;
|
type Future = future::Empty<Self::Output, Self::Error>;
|
||||||
|
|
||||||
fn upgrade_outbound(self, _: C, id: Self::UpgradeId) -> Self::Future {
|
fn upgrade_outbound(self, _: C, id: Self::UpgradeId) -> Self::Future {
|
||||||
unreachable(id)
|
unreachable(id)
|
||||||
|
@ -24,7 +24,7 @@ use crate::{
|
|||||||
protocol,
|
protocol,
|
||||||
utility::{Peer, RelayAddr}
|
utility::{Peer, RelayAddr}
|
||||||
};
|
};
|
||||||
use futures::{stream, prelude::*};
|
use futures::{future, stream, prelude::*};
|
||||||
use libp2p_core::{transport::Transport, upgrade::apply_outbound};
|
use libp2p_core::{transport::Transport, upgrade::apply_outbound};
|
||||||
use log::{debug, info, trace};
|
use log::{debug, info, trace};
|
||||||
use multiaddr::Multiaddr;
|
use multiaddr::Multiaddr;
|
||||||
@ -45,16 +45,14 @@ impl<T, P, S> Transport for RelayTransport<T, P>
|
|||||||
where
|
where
|
||||||
T: Transport + Send + Clone + 'static,
|
T: Transport + Send + Clone + 'static,
|
||||||
T::Dial: Send,
|
T::Dial: Send,
|
||||||
T::Listener: Send,
|
|
||||||
T::ListenerUpgrade: Send,
|
|
||||||
T::Output: AsyncRead + AsyncWrite + Send,
|
T::Output: AsyncRead + AsyncWrite + Send,
|
||||||
P: Deref<Target=S> + Clone + 'static,
|
P: Deref<Target=S> + Clone + 'static,
|
||||||
S: 'static,
|
S: 'static,
|
||||||
for<'a> &'a S: Peerstore
|
for<'a> &'a S: Peerstore
|
||||||
{
|
{
|
||||||
type Output = T::Output;
|
type Output = T::Output;
|
||||||
type Listener = Box<Stream<Item=(Self::ListenerUpgrade, Multiaddr), Error=io::Error> + Send>;
|
type Listener = stream::Empty<(Self::ListenerUpgrade, Multiaddr), io::Error>;
|
||||||
type ListenerUpgrade = Box<Future<Item=Self::Output, Error=io::Error> + Send>;
|
type ListenerUpgrade = future::Empty<Self::Output, io::Error>;
|
||||||
type Dial = Box<Future<Item=Self::Output, Error=io::Error> + Send>;
|
type Dial = Box<Future<Item=Self::Output, Error=io::Error> + Send>;
|
||||||
|
|
||||||
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
|
fn listen_on(self, addr: Multiaddr) -> Result<(Self::Listener, Multiaddr), (Self, Multiaddr)> {
|
||||||
@ -92,8 +90,6 @@ impl<T, P, S> RelayTransport<T, P>
|
|||||||
where
|
where
|
||||||
T: Transport + Clone + 'static,
|
T: Transport + Clone + 'static,
|
||||||
T::Dial: Send,
|
T::Dial: Send,
|
||||||
T::Listener: Send,
|
|
||||||
T::ListenerUpgrade: Send,
|
|
||||||
T::Output: AsyncRead + AsyncWrite + Send,
|
T::Output: AsyncRead + AsyncWrite + Send,
|
||||||
P: Deref<Target=S> + Clone + 'static,
|
P: Deref<Target=S> + Clone + 'static,
|
||||||
for<'a> &'a S: Peerstore
|
for<'a> &'a S: Peerstore
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
// 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::{future, 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 futures::{Async, Future, Poll, Stream};
|
||||||
@ -25,7 +26,6 @@ use multiaddr::{Protocol, 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::sync::{Arc, Mutex};
|
use std::sync::{Arc, Mutex};
|
||||||
use stdweb::web::TypedArray;
|
use stdweb::web::TypedArray;
|
||||||
use stdweb::{self, Reference};
|
use stdweb::{self, Reference};
|
||||||
@ -53,8 +53,8 @@ impl BrowserWsConfig {
|
|||||||
|
|
||||||
impl Transport for BrowserWsConfig {
|
impl Transport for BrowserWsConfig {
|
||||||
type Output = BrowserWsConn;
|
type Output = BrowserWsConn;
|
||||||
type Listener = Box<Stream<Item = (Self::ListenerUpgrade, Multiaddr), Error = IoError> + Send>; // TODO: use `!`
|
type Listener = stream::Empty<(Self::ListenerUpgrade, Multiaddr), IoError>;
|
||||||
type ListenerUpgrade = Box<Future<Item = Self::Output, Error = IoError> + Send>; // TODO: use `!`
|
type ListenerUpgrade = future::Empty<Self::Output, IoError>;
|
||||||
type Dial = Box<Future<Item = Self::Output, Error = IoError> + Send>;
|
type Dial = Box<Future<Item = Self::Output, Error = IoError> + Send>;
|
||||||
|
|
||||||
#[inline]
|
#[inline]
|
||||||
|
Loading…
x
Reference in New Issue
Block a user