Switch to stable futures (#1196)

* Switch to stable futures

* Remove from_fn

* Fix secio

* Fix core --lib tests
This commit is contained in:
Pierre Krieger
2019-09-16 11:08:44 +02:00
committed by GitHub
parent 8c119269d6
commit 170d2d268f
105 changed files with 3193 additions and 5594 deletions

View File

@ -20,19 +20,14 @@
//! Individual messages decoding.
use bytes::BytesMut;
use super::{Hmac, StreamCipher};
use crate::error::SecioError;
use futures::sink::Sink;
use futures::stream::Stream;
use futures::Async;
use futures::Poll;
use futures::StartSend;
use futures::prelude::*;
use log::debug;
use std::cmp::min;
use std::{cmp::min, pin::Pin, task::Context, task::Poll};
/// Wraps around a `Stream<Item = BytesMut>`. The buffers produced by the underlying stream
/// Wraps around a `Stream<Item = Vec<u8>>`. The buffers produced by the underlying stream
/// are decoded using the cipher and hmac.
///
/// This struct implements `Stream`, whose stream item are frames of data without the length
@ -52,7 +47,6 @@ impl<S> DecoderMiddleware<S> {
///
/// The `nonce` parameter denotes a sequence of bytes which are expected to be found at the
/// beginning of the stream and are checked for equality.
#[inline]
pub fn new(raw_stream: S, cipher: StreamCipher, hmac: Hmac, nonce: Vec<u8>) -> DecoderMiddleware<S> {
DecoderMiddleware {
cipher_state: cipher,
@ -65,24 +59,22 @@ impl<S> DecoderMiddleware<S> {
impl<S> Stream for DecoderMiddleware<S>
where
S: Stream<Item = BytesMut>,
S: TryStream<Ok = bytes::BytesMut> + Unpin,
S::Error: Into<SecioError>,
{
type Item = Vec<u8>;
type Error = SecioError;
type Item = Result<Vec<u8>, SecioError>;
#[inline]
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
let frame = match self.raw_stream.poll() {
Ok(Async::Ready(Some(t))) => t,
Ok(Async::Ready(None)) => return Ok(Async::Ready(None)),
Ok(Async::NotReady) => return Ok(Async::NotReady),
Err(err) => return Err(err.into()),
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Option<Self::Item>> {
let frame = match TryStream::try_poll_next(Pin::new(&mut self.raw_stream), cx) {
Poll::Ready(Some(Ok(t))) => t,
Poll::Ready(None) => return Poll::Ready(None),
Poll::Pending => return Poll::Pending,
Poll::Ready(Some(Err(err))) => return Poll::Ready(Some(Err(err.into()))),
};
if frame.len() < self.hmac.num_bytes() {
debug!("frame too short when decoding secio frame");
return Err(SecioError::FrameTooShort);
return Poll::Ready(Some(Err(SecioError::FrameTooShort)));
}
let content_length = frame.len() - self.hmac.num_bytes();
{
@ -91,7 +83,7 @@ where
if self.hmac.verify(crypted_data, expected_hash).is_err() {
debug!("hmac mismatch when decoding secio frame");
return Err(SecioError::HmacNotMatching);
return Poll::Ready(Some(Err(SecioError::HmacNotMatching)));
}
}
@ -103,35 +95,35 @@ where
if !self.nonce.is_empty() {
let n = min(data_buf.len(), self.nonce.len());
if data_buf[.. n] != self.nonce[.. n] {
return Err(SecioError::NonceVerificationFailed)
return Poll::Ready(Some(Err(SecioError::NonceVerificationFailed)))
}
self.nonce.drain(.. n);
data_buf.drain(.. n);
}
Ok(Async::Ready(Some(data_buf)))
Poll::Ready(Some(Ok(data_buf)))
}
}
impl<S> Sink for DecoderMiddleware<S>
impl<S, I> Sink<I> for DecoderMiddleware<S>
where
S: Sink,
S: Sink<I> + Unpin,
{
type SinkItem = S::SinkItem;
type SinkError = S::SinkError;
type Error = S::Error;
#[inline]
fn start_send(&mut self, item: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> {
self.raw_stream.start_send(item)
fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_ready(Pin::new(&mut self.raw_stream), cx)
}
#[inline]
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
self.raw_stream.poll_complete()
fn start_send(mut self: Pin<&mut Self>, item: I) -> Result<(), Self::Error> {
Sink::start_send(Pin::new(&mut self.raw_stream), item)
}
#[inline]
fn close(&mut self) -> Poll<(), Self::SinkError> {
self.raw_stream.close()
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_flush(Pin::new(&mut self.raw_stream), cx)
}
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_close(Pin::new(&mut self.raw_stream), cx)
}
}

View File

@ -20,9 +20,9 @@
//! Individual messages encoding.
use bytes::BytesMut;
use super::{Hmac, StreamCipher};
use futures::prelude::*;
use std::{pin::Pin, task::Context, task::Poll};
/// Wraps around a `Sink`. Encodes the buffers passed to it and passes it to the underlying sink.
///
@ -35,7 +35,6 @@ pub struct EncoderMiddleware<S> {
cipher_state: StreamCipher,
hmac: Hmac,
raw_sink: S,
pending: Option<BytesMut> // buffer encrypted data which can not be sent right away
}
impl<S> EncoderMiddleware<S> {
@ -44,68 +43,44 @@ impl<S> EncoderMiddleware<S> {
cipher_state: cipher,
hmac,
raw_sink: raw,
pending: None
}
}
}
impl<S> Sink for EncoderMiddleware<S>
impl<S> Sink<Vec<u8>> for EncoderMiddleware<S>
where
S: Sink<SinkItem = BytesMut>,
S: Sink<Vec<u8>> + Unpin,
{
type SinkItem = BytesMut;
type SinkError = S::SinkError;
type Error = S::Error;
fn start_send(&mut self, mut data_buf: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> {
if let Some(data) = self.pending.take() {
if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data)? {
self.pending = Some(data);
return Ok(AsyncSink::NotReady(data_buf))
}
}
debug_assert!(self.pending.is_none());
fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_ready(Pin::new(&mut self.raw_sink), cx)
}
fn start_send(mut self: Pin<&mut Self>, mut data_buf: Vec<u8>) -> Result<(), Self::Error> {
// TODO if SinkError gets refactor to SecioError, then use try_apply_keystream
self.cipher_state.encrypt(&mut data_buf[..]);
let signature = self.hmac.sign(&data_buf[..]);
data_buf.extend_from_slice(signature.as_ref());
if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data_buf)? {
self.pending = Some(data)
}
Ok(AsyncSink::Ready)
Sink::start_send(Pin::new(&mut self.raw_sink), data_buf)
}
#[inline]
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
if let Some(data) = self.pending.take() {
if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data)? {
self.pending = Some(data);
return Ok(Async::NotReady)
}
}
self.raw_sink.poll_complete()
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_flush(Pin::new(&mut self.raw_sink), cx)
}
#[inline]
fn close(&mut self) -> Poll<(), Self::SinkError> {
if let Some(data) = self.pending.take() {
if let AsyncSink::NotReady(data) = self.raw_sink.start_send(data)? {
self.pending = Some(data);
return Ok(Async::NotReady)
}
}
self.raw_sink.close()
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_close(Pin::new(&mut self.raw_sink), cx)
}
}
impl<S> Stream for EncoderMiddleware<S>
where
S: Stream,
S: Stream + Unpin,
{
type Item = S::Item;
type Error = S::Error;
#[inline]
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
self.raw_sink.poll()
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Option<Self::Item>> {
Stream::poll_next(Pin::new(&mut self.raw_sink), cx)
}
}

View File

@ -24,18 +24,18 @@
use self::decode::DecoderMiddleware;
use self::encode::EncoderMiddleware;
use aes_ctr::stream_cipher;
use crate::algo_support::Digest;
use futures::prelude::*;
use aes_ctr::stream_cipher;
use hmac::{self, Mac};
use sha2::{Sha256, Sha512};
use tokio_io::codec::length_delimited;
use tokio_io::{AsyncRead, AsyncWrite};
use unsigned_varint::codec::UviBytes;
mod decode;
mod encode;
/// Type returned by `full_codec`.
pub type FullCodec<S> = DecoderMiddleware<EncoderMiddleware<length_delimited::Framed<S>>>;
pub type FullCodec<S> = DecoderMiddleware<EncoderMiddleware<futures_codec::Framed<S, UviBytes<Vec<u8>>>>>;
pub type StreamCipher = Box<dyn stream_cipher::StreamCipher + Send>;
@ -108,7 +108,7 @@ impl Hmac {
/// The conversion between the stream/sink items and the socket is done with the given cipher and
/// hash algorithm (which are generally decided during the handshake).
pub fn full_codec<S>(
socket: length_delimited::Framed<S>,
socket: futures_codec::Framed<S, unsigned_varint::codec::UviBytes<Vec<u8>>>,
cipher_encoding: StreamCipher,
encoding_hmac: Hmac,
cipher_decoder: StreamCipher,
@ -116,7 +116,7 @@ pub fn full_codec<S>(
remote_nonce: Vec<u8>
) -> FullCodec<S>
where
S: AsyncRead + AsyncWrite,
S: AsyncRead + AsyncWrite + Unpin,
{
let encoder = EncoderMiddleware::new(socket, cipher_encoding, encoding_hmac);
DecoderMiddleware::new(encoder, cipher_decoder, decoding_hmac, remote_nonce)
@ -124,56 +124,45 @@ where
#[cfg(test)]
mod tests {
use tokio::runtime::current_thread::Runtime;
use tokio_tcp::{TcpListener, TcpStream};
use crate::stream_cipher::{ctr, Cipher};
use super::full_codec;
use super::DecoderMiddleware;
use super::EncoderMiddleware;
use super::Hmac;
use super::{full_codec, DecoderMiddleware, EncoderMiddleware, Hmac};
use crate::algo_support::Digest;
use crate::stream_cipher::{ctr, Cipher};
use crate::error::SecioError;
use async_std::net::{TcpListener, TcpStream};
use bytes::BytesMut;
use futures::sync::mpsc::channel;
use futures::{Future, Sink, Stream, stream};
use rand;
use std::io::Error as IoError;
use tokio_io::codec::length_delimited::Framed;
use futures::{prelude::*, channel::mpsc, channel::oneshot};
use futures_codec::Framed;
use unsigned_varint::codec::UviBytes;
const NULL_IV : [u8; 16] = [0;16];
const NULL_IV : [u8; 16] = [0; 16];
#[test]
fn raw_encode_then_decode() {
let (data_tx, data_rx) = channel::<BytesMut>(256);
let data_tx = data_tx.sink_map_err::<_, IoError>(|_| panic!());
let data_rx = data_rx.map_err::<IoError, _>(|_| panic!());
let (data_tx, data_rx) = mpsc::channel::<Vec<u8>>(256);
let data_rx = data_rx.map(BytesMut::from);
let cipher_key: [u8; 32] = rand::random();
let hmac_key: [u8; 32] = rand::random();
let encoder = EncoderMiddleware::new(
let mut encoder = EncoderMiddleware::new(
data_tx,
ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key),
);
let decoder = DecoderMiddleware::new(
data_rx,
let mut decoder = DecoderMiddleware::new(
data_rx.map(|v| Ok::<_, SecioError>(v)),
ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key),
Vec::new()
);
let data = b"hello world";
let data_sent = encoder.send(BytesMut::from(data.to_vec())).from_err();
let data_received = decoder.into_future().map(|(n, _)| n).map_err(|(e, _)| e);
let mut rt = Runtime::new().unwrap();
let (_, decoded) = rt.block_on(data_sent.join(data_received))
.map_err(|_| ())
.unwrap();
assert_eq!(&decoded.unwrap()[..], &data[..]);
futures::executor::block_on(async move {
encoder.send(data.to_vec()).await.unwrap();
let rx = decoder.next().await.unwrap().unwrap();
assert_eq!(rx, data);
});
}
fn full_codec_encode_then_decode(cipher: Cipher) {
@ -185,53 +174,44 @@ mod tests {
let data = b"hello world";
let data_clone = data.clone();
let nonce = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9];
let listener = TcpListener::bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let listener_addr = listener.local_addr().unwrap();
let (l_a_tx, l_a_rx) = oneshot::channel();
let nonce2 = nonce.clone();
let server = listener.incoming()
.into_future()
.map_err(|(e, _)| e)
.map(move |(connec, _)| {
full_codec(
Framed::new(connec.unwrap()),
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key),
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key),
nonce2
)
},
);
let server = async {
let listener = TcpListener::bind(&"127.0.0.1:0").await.unwrap();
let listener_addr = listener.local_addr().unwrap();
l_a_tx.send(listener_addr).unwrap();
let client = TcpStream::connect(&listener_addr)
.map_err(|e| e.into())
.map(move |stream| {
full_codec(
Framed::new(stream),
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
Vec::new()
)
});
let (connec, _) = listener.accept().await.unwrap();
let codec = full_codec(
Framed::new(connec, UviBytes::default()),
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key),
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key),
nonce2.clone()
);
let fin = server
.join(client)
.from_err::<SecioError>()
.and_then(|(server, client)| {
client
.send_all(stream::iter_ok::<_, IoError>(vec![nonce.into(), data_clone[..].into()]))
.map(move |_| server)
.from_err()
})
.and_then(|server| server.concat2().from_err());
let outcome = codec.map(|v| v.unwrap()).concat().await;
assert_eq!(outcome, data_clone);
};
let mut rt = Runtime::new().unwrap();
let received = rt.block_on(fin).unwrap();
assert_eq!(received, data);
let client = async {
let listener_addr = l_a_rx.await.unwrap();
let stream = TcpStream::connect(&listener_addr).await.unwrap();
let mut codec = full_codec(
Framed::new(stream, UviBytes::default()),
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
Vec::new()
);
codec.send(nonce.into()).await.unwrap();
codec.send(data.to_vec().into()).await.unwrap();
};
futures::executor::block_on(future::join(client, server));
}
#[test]

View File

@ -43,7 +43,7 @@ pub type AgreementPrivateKey = ring_agreement::EphemeralPrivateKey;
/// Generates a new key pair as part of the exchange.
///
/// Returns the opaque private key and the corresponding public key.
pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future<Item = (AgreementPrivateKey, Vec<u8>), Error = SecioError> {
pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future<Output = Result<(AgreementPrivateKey, Vec<u8>), SecioError>> {
let rng = ring_rand::SystemRandom::new();
match ring_agreement::EphemeralPrivateKey::generate(algorithm.into(), &rng) {
@ -51,22 +51,22 @@ pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future<Item = (Agreem
let r = tmp_priv_key.compute_public_key()
.map_err(|_| SecioError::EphemeralKeyGenerationFailed)
.map(move |tmp_pub_key| (tmp_priv_key, tmp_pub_key.as_ref().to_vec()));
future::result(r)
future::ready(r)
},
Err(_) => {
debug!("failed to generate ECDH key");
future::err(SecioError::EphemeralKeyGenerationFailed)
future::ready(Err(SecioError::EphemeralKeyGenerationFailed))
},
}
}
/// Finish the agreement. On success, returns the shared key that both remote agreed upon.
pub fn agree(algorithm: KeyAgreement, my_private_key: AgreementPrivateKey, other_public_key: &[u8], _out_size: usize)
-> impl Future<Item = Vec<u8>, Error = SecioError>
-> impl Future<Output = Result<Vec<u8>, SecioError>>
{
ring_agreement::agree_ephemeral(my_private_key, algorithm.into(),
UntrustedInput::from(other_public_key),
SecioError::SecretGenerationFailed,
|key_material| Ok(key_material.to_vec()))
.into_future()
let ret = ring_agreement::agree_ephemeral(my_private_key, algorithm.into(),
UntrustedInput::from(other_public_key),
SecioError::SecretGenerationFailed,
|key_material| Ok(key_material.to_vec()));
future::ready(ret)
}

View File

@ -44,14 +44,14 @@ pub struct AgreementPrivateKey(platform::AgreementPrivateKey);
///
/// Returns the opaque private key and the corresponding public key.
#[inline]
pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future<Item = (AgreementPrivateKey, Vec<u8>), Error = SecioError> {
platform::generate_agreement(algorithm).map(|(pr, pu)| (AgreementPrivateKey(pr), pu))
pub fn generate_agreement(algorithm: KeyAgreement) -> impl Future<Output = Result<(AgreementPrivateKey, Vec<u8>), SecioError>> {
platform::generate_agreement(algorithm).map_ok(|(pr, pu)| (AgreementPrivateKey(pr), pu))
}
/// Finish the agreement. On success, returns the shared key that both remote agreed upon.
#[inline]
pub fn agree(algorithm: KeyAgreement, my_private_key: AgreementPrivateKey, other_public_key: &[u8], out_size: usize)
-> impl Future<Item = Vec<u8>, Error = SecioError>
-> impl Future<Output = Result<Vec<u8>, SecioError>>
{
platform::agree(algorithm, my_private_key.0, other_public_key, out_size)
}

View File

@ -19,15 +19,11 @@
// DEALINGS IN THE SOFTWARE.
use crate::algo_support;
use bytes::BytesMut;
use crate::codec::{full_codec, FullCodec, Hmac};
use crate::stream_cipher::{Cipher, ctr};
use crate::stream_cipher::ctr;
use crate::error::SecioError;
use crate::exchange;
use futures::future;
use futures::sink::Sink;
use futures::stream::Stream;
use futures::Future;
use futures::prelude::*;
use libp2p_core::PublicKey;
use log::{debug, trace};
use protobuf::parse_from_bytes as protobuf_parse_from_bytes;
@ -37,447 +33,291 @@ use sha2::{Digest as ShaDigestTrait, Sha256};
use std::cmp::{self, Ordering};
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use crate::structs_proto::{Exchange, Propose};
use tokio_io::codec::length_delimited;
use tokio_io::{AsyncRead, AsyncWrite};
use crate::{KeyAgreement, SecioConfig};
// This struct contains the whole context of a handshake, and is filled progressively
// throughout the various parts of the handshake.
struct HandshakeContext<T> {
config: SecioConfig,
state: T
}
// HandshakeContext<()> --with_local-> HandshakeContext<Local>
struct Local {
// Locally-generated random number. The array size can be changed without any repercussion.
nonce: [u8; 16],
// Our encoded local public key
public_key_encoded: Vec<u8>,
// Our local proposition's raw bytes:
proposition_bytes: Vec<u8>
}
// HandshakeContext<Local> --with_remote-> HandshakeContext<Remote>
struct Remote {
local: Local,
// The remote's proposition's raw bytes:
proposition_bytes: BytesMut,
// The remote's public key:
public_key: PublicKey,
// The remote's `nonce`.
// If the NONCE size is actually part of the protocol, we can change this to a fixed-size
// array instead of a `Vec`.
nonce: Vec<u8>,
// Set to `ordering(
// hash(concat(remote-pubkey, local-none)),
// hash(concat(local-pubkey, remote-none))
// )`.
// `Ordering::Equal` is an invalid value (as it would mean we're talking to ourselves).
//
// Since everything is symmetrical, this value is used to determine what should be ours
// and what should be the remote's.
hashes_ordering: Ordering,
// Crypto algorithms chosen for the communication:
chosen_exchange: KeyAgreement,
chosen_cipher: Cipher,
chosen_hash: algo_support::Digest,
}
// HandshakeContext<Remote> --with_ephemeral-> HandshakeContext<Ephemeral>
struct Ephemeral {
remote: Remote,
// Ephemeral keypair generated for the handshake:
local_tmp_priv_key: exchange::AgreementPrivateKey,
local_tmp_pub_key: Vec<u8>
}
// HandshakeContext<Ephemeral> --take_private_key-> HandshakeContext<PubEphemeral>
struct PubEphemeral {
remote: Remote,
local_tmp_pub_key: Vec<u8>
}
impl HandshakeContext<()> {
fn new(config: SecioConfig) -> Self {
HandshakeContext {
config,
state: ()
}
}
// Setup local proposition.
fn with_local(self) -> Result<HandshakeContext<Local>, SecioError> {
let mut nonce = [0; 16];
rand::thread_rng()
.try_fill_bytes(&mut nonce)
.map_err(|_| SecioError::NonceGenerationFailed)?;
let public_key_encoded = self.config.key.public().into_protobuf_encoding();
// Send our proposition with our nonce, public key and supported protocols.
let mut proposition = Propose::new();
proposition.set_rand(nonce.to_vec());
proposition.set_pubkey(public_key_encoded.clone());
if let Some(ref p) = self.config.agreements_prop {
trace!("agreements proposition: {}", p);
proposition.set_exchanges(p.clone())
} else {
trace!("agreements proposition: {}", algo_support::DEFAULT_AGREEMENTS_PROPOSITION);
proposition.set_exchanges(algo_support::DEFAULT_AGREEMENTS_PROPOSITION.into())
}
if let Some(ref p) = self.config.ciphers_prop {
trace!("ciphers proposition: {}", p);
proposition.set_ciphers(p.clone())
} else {
trace!("ciphers proposition: {}", algo_support::DEFAULT_CIPHERS_PROPOSITION);
proposition.set_ciphers(algo_support::DEFAULT_CIPHERS_PROPOSITION.into())
}
if let Some(ref p) = self.config.digests_prop {
trace!("digests proposition: {}", p);
proposition.set_hashes(p.clone())
} else {
trace!("digests proposition: {}", algo_support::DEFAULT_DIGESTS_PROPOSITION);
proposition.set_hashes(algo_support::DEFAULT_DIGESTS_PROPOSITION.into())
}
let proposition_bytes = proposition.write_to_bytes()?;
Ok(HandshakeContext {
config: self.config,
state: Local {
nonce,
public_key_encoded,
proposition_bytes
}
})
}
}
impl HandshakeContext<Local> {
// Process remote proposition.
fn with_remote(self, b: BytesMut) -> Result<HandshakeContext<Remote>, SecioError> {
let mut prop = match protobuf_parse_from_bytes::<Propose>(&b) {
Ok(prop) => prop,
Err(_) => {
debug!("failed to parse remote's proposition protobuf message");
return Err(SecioError::HandshakeParsingFailure);
}
};
let public_key_encoded = prop.take_pubkey();
let nonce = prop.take_rand();
let pubkey = match PublicKey::from_protobuf_encoding(&public_key_encoded) {
Ok(p) => p,
Err(_) => {
debug!("failed to parse remote's proposition's pubkey protobuf");
return Err(SecioError::HandshakeParsingFailure);
},
};
// In order to determine which protocols to use, we compute two hashes and choose
// based on which hash is larger.
let hashes_ordering = {
let oh1 = {
let mut ctx = Sha256::new();
ctx.input(&public_key_encoded);
ctx.input(&self.state.nonce);
ctx.result()
};
let oh2 = {
let mut ctx = Sha256::new();
ctx.input(&self.state.public_key_encoded);
ctx.input(&nonce);
ctx.result()
};
oh1.as_ref().cmp(&oh2.as_ref())
};
let chosen_exchange = {
let ours = self.config.agreements_prop.as_ref()
.map(|s| s.as_ref())
.unwrap_or(algo_support::DEFAULT_AGREEMENTS_PROPOSITION);
let theirs = &prop.get_exchanges();
match algo_support::select_agreement(hashes_ordering, ours, theirs) {
Ok(a) => a,
Err(err) => {
debug!("failed to select an exchange protocol");
return Err(err);
}
}
};
let chosen_cipher = {
let ours = self.config.ciphers_prop.as_ref()
.map(|s| s.as_ref())
.unwrap_or(algo_support::DEFAULT_CIPHERS_PROPOSITION);
let theirs = &prop.get_ciphers();
match algo_support::select_cipher(hashes_ordering, ours, theirs) {
Ok(a) => {
debug!("selected cipher: {:?}", a);
a
}
Err(err) => {
debug!("failed to select a cipher protocol");
return Err(err);
}
}
};
let chosen_hash = {
let ours = self.config.digests_prop.as_ref()
.map(|s| s.as_ref())
.unwrap_or(algo_support::DEFAULT_DIGESTS_PROPOSITION);
let theirs = &prop.get_hashes();
match algo_support::select_digest(hashes_ordering, ours, theirs) {
Ok(a) => {
debug!("selected hash: {:?}", a);
a
}
Err(err) => {
debug!("failed to select a hash protocol");
return Err(err);
}
}
};
Ok(HandshakeContext {
config: self.config,
state: Remote {
local: self.state,
proposition_bytes: b,
public_key: pubkey,
nonce,
hashes_ordering,
chosen_exchange,
chosen_cipher,
chosen_hash
}
})
}
}
impl HandshakeContext<Remote> {
fn with_ephemeral(self, sk: exchange::AgreementPrivateKey, pk: Vec<u8>) -> HandshakeContext<Ephemeral> {
HandshakeContext {
config: self.config,
state: Ephemeral {
remote: self.state,
local_tmp_priv_key: sk,
local_tmp_pub_key: pk
}
}
}
}
impl HandshakeContext<Ephemeral> {
fn take_private_key(self) -> (HandshakeContext<PubEphemeral>, exchange::AgreementPrivateKey) {
let context = HandshakeContext {
config: self.config,
state: PubEphemeral {
remote: self.state.remote,
local_tmp_pub_key: self.state.local_tmp_pub_key
}
};
(context, self.state.local_tmp_priv_key)
}
}
use crate::SecioConfig;
/// Performs a handshake on the given socket.
///
/// This function expects that the remote is identified with `remote_public_key`, and the remote
/// will expect that we are identified with `local_key`.Any mismatch somewhere will produce a
/// will expect that we are identified with `local_key`. Any mismatch somewhere will produce a
/// `SecioError`.
///
/// On success, returns an object that implements the `Sink` and `Stream` trait whose items are
/// buffers of data, plus the public key of the remote, plus the ephemeral public key used during
/// negotiation.
pub fn handshake<'a, S: 'a>(socket: S, config: SecioConfig)
-> impl Future<Item = (FullCodec<S>, PublicKey, Vec<u8>), Error = SecioError>
pub async fn handshake<'a, S: 'a>(socket: S, config: SecioConfig)
-> Result<(FullCodec<S>, PublicKey, Vec<u8>), SecioError>
where
S: AsyncRead + AsyncWrite + Send,
S: AsyncRead + AsyncWrite + Send + Unpin,
{
// The handshake messages all start with a 4-bytes message length prefix.
let socket = length_delimited::Builder::new()
.big_endian()
.length_field_length(4)
.new_framed(socket);
// The handshake messages all start with a variable-length integer indicating the size.
let mut socket = futures_codec::Framed::new(
socket,
unsigned_varint::codec::UviBytes::<Vec<u8>>::default()
);
future::ok::<_, SecioError>(HandshakeContext::new(config))
.and_then(|context| {
// Generate our nonce.
let context = context.with_local()?;
trace!("starting handshake; local nonce = {:?}", context.state.nonce);
Ok(context)
})
.and_then(|context| {
trace!("sending proposition to remote");
socket.send(BytesMut::from(context.state.proposition_bytes.clone()))
.from_err()
.map(|s| (s, context))
})
// Receive the remote's proposition.
.and_then(move |(socket, context)| {
socket.into_future()
.map_err(|(e, _)| e.into())
.and_then(move |(prop_raw, socket)| {
let context = match prop_raw {
Some(p) => context.with_remote(p)?,
None => {
let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof");
debug!("unexpected eof while waiting for remote's proposition");
return Err(err.into())
},
};
trace!("received proposition from remote; pubkey = {:?}; nonce = {:?}",
context.state.public_key, context.state.nonce);
Ok((socket, context))
})
})
// Generate an ephemeral key for the negotiation.
.and_then(|(socket, context)| {
exchange::generate_agreement(context.state.chosen_exchange)
.map(move |(tmp_priv_key, tmp_pub_key)| (socket, context, tmp_priv_key, tmp_pub_key))
})
// Send the ephemeral pub key to the remote in an `Exchange` struct. The `Exchange` also
// contains a signature of the two propositions encoded with our static public key.
.and_then(|(socket, context, tmp_priv, tmp_pub_key)| {
let context = context.with_ephemeral(tmp_priv, tmp_pub_key.clone());
let exchange = {
let mut data_to_sign = context.state.remote.local.proposition_bytes.clone();
data_to_sign.extend_from_slice(&context.state.remote.proposition_bytes);
data_to_sign.extend_from_slice(&tmp_pub_key);
let local_nonce = {
let mut local_nonce = [0; 16];
rand::thread_rng()
.try_fill_bytes(&mut local_nonce)
.map_err(|_| SecioError::NonceGenerationFailed)?;
local_nonce
};
let mut exchange = Exchange::new();
exchange.set_epubkey(tmp_pub_key);
match context.config.key.sign(&data_to_sign) {
Ok(sig) => exchange.set_signature(sig),
Err(_) => return Err(SecioError::SigningFailure)
}
exchange
};
let local_exch = exchange.write_to_bytes()?;
Ok((BytesMut::from(local_exch), socket, context))
})
// Send our local `Exchange`.
.and_then(|(local_exch, socket, context)| {
trace!("sending exchange to remote");
socket.send(local_exch)
.from_err()
.map(|s| (s, context))
})
// Receive the remote's `Exchange`.
.and_then(move |(socket, context)| {
socket.into_future()
.map_err(|(e, _)| e.into())
.and_then(move |(raw, socket)| {
let raw = match raw {
Some(r) => r,
None => {
let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof");
debug!("unexpected eof while waiting for remote's exchange");
return Err(err.into())
},
};
let local_public_key_encoded = config.key.public().into_protobuf_encoding();
let remote_exch = match protobuf_parse_from_bytes::<Exchange>(&raw) {
Ok(e) => e,
Err(err) => {
debug!("failed to parse remote's exchange protobuf; {:?}", err);
return Err(SecioError::HandshakeParsingFailure);
}
};
// Send our proposition with our nonce, public key and supported protocols.
let mut local_proposition = Propose::new();
local_proposition.set_rand(local_nonce.to_vec());
local_proposition.set_pubkey(local_public_key_encoded.clone());
trace!("received and decoded the remote's exchange");
Ok((remote_exch, socket, context))
})
})
// Check the validity of the remote's `Exchange`. This verifies that the remote was really
// the sender of its proposition, and that it is the owner of both its global and ephemeral
// keys.
.and_then(|(remote_exch, socket, context)| {
let mut data_to_verify = context.state.remote.proposition_bytes.clone();
data_to_verify.extend_from_slice(&context.state.remote.local.proposition_bytes);
data_to_verify.extend_from_slice(remote_exch.get_epubkey());
if let Some(ref p) = config.agreements_prop {
trace!("agreements proposition: {}", p);
local_proposition.set_exchanges(p.clone())
} else {
trace!("agreements proposition: {}", algo_support::DEFAULT_AGREEMENTS_PROPOSITION);
local_proposition.set_exchanges(algo_support::DEFAULT_AGREEMENTS_PROPOSITION.into())
}
if !context.state.remote.public_key.verify(&data_to_verify, remote_exch.get_signature()) {
return Err(SecioError::SignatureVerificationFailed)
if let Some(ref p) = config.ciphers_prop {
trace!("ciphers proposition: {}", p);
local_proposition.set_ciphers(p.clone())
} else {
trace!("ciphers proposition: {}", algo_support::DEFAULT_CIPHERS_PROPOSITION);
local_proposition.set_ciphers(algo_support::DEFAULT_CIPHERS_PROPOSITION.into())
}
if let Some(ref p) = config.digests_prop {
trace!("digests proposition: {}", p);
local_proposition.set_hashes(p.clone())
} else {
trace!("digests proposition: {}", algo_support::DEFAULT_DIGESTS_PROPOSITION);
local_proposition.set_hashes(algo_support::DEFAULT_DIGESTS_PROPOSITION.into())
}
let local_proposition_bytes = local_proposition.write_to_bytes()?;
trace!("starting handshake; local nonce = {:?}", local_nonce);
trace!("sending proposition to remote");
socket.send(local_proposition_bytes.clone()).await?;
// Receive the remote's proposition.
let remote_proposition_bytes = match socket.next().await {
Some(b) => b?,
None => {
let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof");
debug!("unexpected eof while waiting for remote's proposition");
return Err(err.into())
},
};
let mut remote_proposition = match protobuf_parse_from_bytes::<Propose>(&remote_proposition_bytes) {
Ok(prop) => prop,
Err(_) => {
debug!("failed to parse remote's proposition protobuf message");
return Err(SecioError::HandshakeParsingFailure);
}
};
let remote_public_key_encoded = remote_proposition.take_pubkey();
let remote_nonce = remote_proposition.take_rand();
let remote_public_key = match PublicKey::from_protobuf_encoding(&remote_public_key_encoded) {
Ok(p) => p,
Err(_) => {
debug!("failed to parse remote's proposition's pubkey protobuf");
return Err(SecioError::HandshakeParsingFailure);
},
};
trace!("received proposition from remote; pubkey = {:?}; nonce = {:?}",
remote_public_key, remote_nonce);
// In order to determine which protocols to use, we compute two hashes and choose
// based on which hash is larger.
let hashes_ordering = {
let oh1 = {
let mut ctx = Sha256::new();
ctx.input(&remote_public_key_encoded);
ctx.input(&local_nonce);
ctx.result()
};
let oh2 = {
let mut ctx = Sha256::new();
ctx.input(&local_public_key_encoded);
ctx.input(&remote_nonce);
ctx.result()
};
oh1.as_ref().cmp(&oh2.as_ref())
};
let chosen_exchange = {
let ours = config.agreements_prop.as_ref()
.map(|s| s.as_ref())
.unwrap_or(algo_support::DEFAULT_AGREEMENTS_PROPOSITION);
let theirs = &remote_proposition.get_exchanges();
match algo_support::select_agreement(hashes_ordering, ours, theirs) {
Ok(a) => a,
Err(err) => {
debug!("failed to select an exchange protocol");
return Err(err);
}
}
};
trace!("successfully verified the remote's signature");
Ok((remote_exch, socket, context))
})
// Generate a key from the local ephemeral private key and the remote ephemeral public key,
// derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder.
.and_then(|(remote_exch, socket, context)| {
let (context, local_priv_key) = context.take_private_key();
let key_size = context.state.remote.chosen_hash.num_bytes();
exchange::agree(context.state.remote.chosen_exchange, local_priv_key, remote_exch.get_epubkey(), key_size)
.map(move |key_material| (socket, context, key_material))
})
// Generate a key from the local ephemeral private key and the remote ephemeral public key,
// derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder.
.and_then(|(socket, context, key_material)| {
let chosen_cipher = context.state.remote.chosen_cipher;
let cipher_key_size = chosen_cipher.key_size();
let iv_size = chosen_cipher.iv_size();
let chosen_cipher = {
let ours = config.ciphers_prop.as_ref()
.map(|s| s.as_ref())
.unwrap_or(algo_support::DEFAULT_CIPHERS_PROPOSITION);
let theirs = &remote_proposition.get_ciphers();
match algo_support::select_cipher(hashes_ordering, ours, theirs) {
Ok(a) => {
debug!("selected cipher: {:?}", a);
a
}
Err(err) => {
debug!("failed to select a cipher protocol");
return Err(err);
}
}
};
let key = Hmac::from_key(context.state.remote.chosen_hash, &key_material);
let mut longer_key = vec![0u8; 2 * (iv_size + cipher_key_size + 20)];
stretch_key(key, &mut longer_key);
let chosen_hash = {
let ours = config.digests_prop.as_ref()
.map(|s| s.as_ref())
.unwrap_or(algo_support::DEFAULT_DIGESTS_PROPOSITION);
let theirs = &remote_proposition.get_hashes();
match algo_support::select_digest(hashes_ordering, ours, theirs) {
Ok(a) => {
debug!("selected hash: {:?}", a);
a
}
Err(err) => {
debug!("failed to select a hash protocol");
return Err(err);
}
}
};
let (local_infos, remote_infos) = {
let (first_half, second_half) = longer_key.split_at(longer_key.len() / 2);
match context.state.remote.hashes_ordering {
Ordering::Equal => {
let msg = "equal digest of public key and nonce for local and remote";
return Err(SecioError::InvalidProposition(msg))
}
Ordering::Less => (second_half, first_half),
Ordering::Greater => (first_half, second_half),
// Generate an ephemeral key for the negotiation.
let (tmp_priv_key, tmp_pub_key) = exchange::generate_agreement(chosen_exchange).await?;
// Send the ephemeral pub key to the remote in an `Exchange` struct. The `Exchange` also
// contains a signature of the two propositions encoded with our static public key.
let local_exchange = {
let mut data_to_sign = local_proposition_bytes.clone();
data_to_sign.extend_from_slice(&remote_proposition_bytes);
data_to_sign.extend_from_slice(&tmp_pub_key);
let mut exchange = Exchange::new();
exchange.set_epubkey(tmp_pub_key.clone());
match config.key.sign(&data_to_sign) {
Ok(sig) => exchange.set_signature(sig),
Err(_) => return Err(SecioError::SigningFailure)
}
exchange
};
let local_exch = local_exchange.write_to_bytes()?;
// Send our local `Exchange`.
trace!("sending exchange to remote");
socket.send(local_exch).await?;
// Receive the remote's `Exchange`.
let remote_exch = {
let raw = match socket.next().await {
Some(r) => r?,
None => {
let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof");
debug!("unexpected eof while waiting for remote's exchange");
return Err(err.into())
},
};
match protobuf_parse_from_bytes::<Exchange>(&raw) {
Ok(e) => {
trace!("received and decoded the remote's exchange");
e
},
Err(err) => {
debug!("failed to parse remote's exchange protobuf; {:?}", err);
return Err(SecioError::HandshakeParsingFailure);
}
}
};
// Check the validity of the remote's `Exchange`. This verifies that the remote was really
// the sender of its proposition, and that it is the owner of both its global and ephemeral
// keys.
{
let mut data_to_verify = remote_proposition_bytes.clone();
data_to_verify.extend_from_slice(&local_proposition_bytes);
data_to_verify.extend_from_slice(remote_exch.get_epubkey());
if !remote_public_key.verify(&data_to_verify, remote_exch.get_signature()) {
return Err(SecioError::SignatureVerificationFailed)
}
trace!("successfully verified the remote's signature");
}
// Generate a key from the local ephemeral private key and the remote ephemeral public key,
// derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder.
let key_material = exchange::agree(chosen_exchange, tmp_priv_key, remote_exch.get_epubkey(), chosen_hash.num_bytes()).await?;
// Generate a key from the local ephemeral private key and the remote ephemeral public key,
// derive from it a cipher key, an iv, and a hmac key, and build the encoder/decoder.
let mut codec = {
let cipher_key_size = chosen_cipher.key_size();
let iv_size = chosen_cipher.iv_size();
let key = Hmac::from_key(chosen_hash, &key_material);
let mut longer_key = vec![0u8; 2 * (iv_size + cipher_key_size + 20)];
stretch_key(key, &mut longer_key);
let (local_infos, remote_infos) = {
let (first_half, second_half) = longer_key.split_at(longer_key.len() / 2);
match hashes_ordering {
Ordering::Equal => {
let msg = "equal digest of public key and nonce for local and remote";
return Err(SecioError::InvalidProposition(msg))
}
};
Ordering::Less => (second_half, first_half),
Ordering::Greater => (first_half, second_half),
}
};
let (encoding_cipher, encoding_hmac) = {
let (iv, rest) = local_infos.split_at(iv_size);
let (cipher_key, mac_key) = rest.split_at(cipher_key_size);
let hmac = Hmac::from_key(context.state.remote.chosen_hash, mac_key);
let cipher = ctr(chosen_cipher, cipher_key, iv);
(cipher, hmac)
};
let (encoding_cipher, encoding_hmac) = {
let (iv, rest) = local_infos.split_at(iv_size);
let (cipher_key, mac_key) = rest.split_at(cipher_key_size);
let hmac = Hmac::from_key(chosen_hash, mac_key);
let cipher = ctr(chosen_cipher, cipher_key, iv);
(cipher, hmac)
};
let (decoding_cipher, decoding_hmac) = {
let (iv, rest) = remote_infos.split_at(iv_size);
let (cipher_key, mac_key) = rest.split_at(cipher_key_size);
let hmac = Hmac::from_key(context.state.remote.chosen_hash, mac_key);
let cipher = ctr(chosen_cipher, cipher_key, iv);
(cipher, hmac)
};
let (decoding_cipher, decoding_hmac) = {
let (iv, rest) = remote_infos.split_at(iv_size);
let (cipher_key, mac_key) = rest.split_at(cipher_key_size);
let hmac = Hmac::from_key(chosen_hash, mac_key);
let cipher = ctr(chosen_cipher, cipher_key, iv);
(cipher, hmac)
};
let codec = full_codec(
socket,
encoding_cipher,
encoding_hmac,
decoding_cipher,
decoding_hmac,
context.state.remote.local.nonce.to_vec()
);
Ok((codec, context))
})
// We send back their nonce to check if the connection works.
.and_then(|(codec, context)| {
let remote_nonce = context.state.remote.nonce.clone();
trace!("checking encryption by sending back remote's nonce");
codec.send(BytesMut::from(remote_nonce))
.map(|s| (s, context.state.remote.public_key, context.state.local_tmp_pub_key))
.from_err()
})
full_codec(
socket,
encoding_cipher,
encoding_hmac,
decoding_cipher,
decoding_hmac,
local_nonce.to_vec()
)
};
// We send back their nonce to check if the connection works.
trace!("checking encryption by sending back remote's nonce");
codec.send(remote_nonce).await?;
Ok((codec, remote_public_key, tmp_pub_key))
}
/// Custom algorithm translated from reference implementations. Needs to be the same algorithm
@ -522,16 +362,10 @@ where D: ::hmac::digest::Input + ::hmac::digest::BlockInput +
#[cfg(test)]
mod tests {
use bytes::BytesMut;
use super::{handshake, stretch_key};
use crate::{algo_support::Digest, codec::Hmac, SecioConfig};
use libp2p_core::identity;
use tokio::runtime::current_thread::Runtime;
use tokio_tcp::{TcpListener, TcpStream};
use crate::{SecioConfig, SecioError};
use super::handshake;
use super::stretch_key;
use crate::algo_support::Digest;
use crate::codec::Hmac;
use futures::prelude::*;
use futures::{prelude::*, channel::oneshot};
#[test]
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
@ -573,38 +407,30 @@ mod tests {
}
fn handshake_with_self_succeeds(key1: SecioConfig, key2: SecioConfig) {
let listener = TcpListener::bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
let listener_addr = listener.local_addr().unwrap();
let (l_a_tx, l_a_rx) = oneshot::channel();
let server = listener
.incoming()
.into_future()
.map_err(|(e, _)| e.into())
.and_then(move |(connec, _)| handshake(connec.unwrap(), key1))
.and_then(|(connec, _, _)| {
let (sink, stream) = connec.split();
stream
.filter(|v| !v.is_empty())
.forward(sink.with(|v| Ok::<_, SecioError>(BytesMut::from(v))))
});
async_std::task::spawn(async move {
let listener = async_std::net::TcpListener::bind(&"127.0.0.1:0").await.unwrap();
l_a_tx.send(listener.local_addr().unwrap()).unwrap();
let connec = listener.accept().await.unwrap().0;
let mut codec = handshake(connec, key1).await.unwrap().0;
while let Some(packet) = codec.next().await {
let packet = packet.unwrap();
if !packet.is_empty() {
codec.send(packet.into()).await.unwrap();
}
}
});
let client = TcpStream::connect(&listener_addr)
.map_err(|e| e.into())
.and_then(move |stream| handshake(stream, key2))
.and_then(|(connec, _, _)| {
connec.send("hello".into())
.from_err()
.and_then(|connec| {
connec.filter(|v| !v.is_empty())
.into_future()
.map(|(v, _)| v)
.map_err(|(e, _)| e)
})
.map(|v| assert_eq!(b"hello", &v.unwrap()[..]))
});
let mut rt = Runtime::new().unwrap();
let _ = rt.block_on(server.join(client)).unwrap();
futures::executor::block_on(async move {
let listen_addr = l_a_rx.await.unwrap();
let connec = async_std::net::TcpStream::connect(&listen_addr).await.unwrap();
let mut codec = handshake(connec, key2).await.unwrap().0;
codec.send(b"hello".to_vec().into()).await.unwrap();
let mut packets_stream = codec.filter(|p| future::ready(!p.as_ref().unwrap().is_empty()));
let packet = packets_stream.next().await.unwrap();
assert_eq!(packet.unwrap(), b"hello");
});
}
#[test]

View File

@ -29,7 +29,7 @@
//!
//! ```no_run
//! # fn main() {
//! use futures::Future;
//! use futures::prelude::*;
//! use libp2p_secio::{SecioConfig, SecioOutput};
//! use libp2p_core::{PeerId, Multiaddr, identity};
//! use libp2p_core::transport::Transport;
@ -57,20 +57,12 @@
pub use self::error::SecioError;
use bytes::BytesMut;
use futures::stream::MapErr as StreamMapErr;
use futures::{Future, Poll, Sink, StartSend, Stream};
use libp2p_core::{
PeerId,
PublicKey,
identity,
upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade, Negotiated}
};
use futures::{prelude::*, io::Initializer};
use libp2p_core::{PeerId, PublicKey, identity, upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade, Negotiated}};
use log::debug;
use rw_stream_sink::RwStreamSink;
use std::io;
use std::iter;
use tokio_io::{AsyncRead, AsyncWrite};
use std::{io, iter, pin::Pin, task::Context, task::Poll};
mod algo_support;
mod codec;
@ -134,13 +126,13 @@ impl SecioConfig {
self
}
fn handshake<T>(self, socket: T) -> impl Future<Item=(PeerId, SecioOutput<T>), Error=SecioError>
fn handshake<T>(self, socket: T) -> impl Future<Output = Result<(PeerId, SecioOutput<T>), SecioError>>
where
T: AsyncRead + AsyncWrite + Send + 'static
T: AsyncRead + AsyncWrite + Unpin + Send + 'static
{
debug!("Starting secio upgrade");
SecioMiddleware::handshake(socket, self)
.map(|(stream_sink, pubkey, ephemeral)| {
.map_ok(|(stream_sink, pubkey, ephemeral)| {
let mapped = stream_sink.map_err(map_err as fn(_) -> _);
let peer = pubkey.clone().into_peer_id();
let io = SecioOutput {
@ -177,55 +169,59 @@ impl UpgradeInfo for SecioConfig {
impl<T> InboundUpgrade<T> for SecioConfig
where
T: AsyncRead + AsyncWrite + Send + 'static
T: AsyncRead + AsyncWrite + Unpin + Send + 'static
{
type Output = (PeerId, SecioOutput<Negotiated<T>>);
type Error = SecioError;
type Future = Box<dyn Future<Item = Self::Output, Error = Self::Error> + Send>;
type Future = Pin<Box<dyn Future<Output = Result<Self::Output, Self::Error>> + Send>>;
fn upgrade_inbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
Box::new(self.handshake(socket))
Box::pin(self.handshake(socket))
}
}
impl<T> OutboundUpgrade<T> for SecioConfig
where
T: AsyncRead + AsyncWrite + Send + 'static
T: AsyncRead + AsyncWrite + Unpin + Send + 'static
{
type Output = (PeerId, SecioOutput<Negotiated<T>>);
type Error = SecioError;
type Future = Box<dyn Future<Item = Self::Output, Error = Self::Error> + Send>;
type Future = Pin<Box<dyn Future<Output = Result<Self::Output, Self::Error>> + Send>>;
fn upgrade_outbound(self, socket: Negotiated<T>, _: Self::Info) -> Self::Future {
Box::new(self.handshake(socket))
Box::pin(self.handshake(socket))
}
}
impl<S: AsyncRead + AsyncWrite> io::Read for SecioOutput<S> {
fn read(&mut self, buf: &mut [u8]) -> io::Result<usize> {
self.stream.read(buf)
impl<S: AsyncRead + AsyncWrite + Unpin> AsyncRead for SecioOutput<S> {
fn poll_read(mut self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8])
-> Poll<Result<usize, io::Error>>
{
AsyncRead::poll_read(Pin::new(&mut self.stream), cx, buf)
}
unsafe fn initializer(&self) -> Initializer {
self.stream.initializer()
}
}
impl<S: AsyncRead + AsyncWrite> AsyncRead for SecioOutput<S> {
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool {
self.stream.prepare_uninitialized_buffer(buf)
}
}
impl<S: AsyncRead + AsyncWrite> io::Write for SecioOutput<S> {
fn write(&mut self, buf: &[u8]) -> io::Result<usize> {
self.stream.write(buf)
impl<S: AsyncRead + AsyncWrite + Unpin> AsyncWrite for SecioOutput<S> {
fn poll_write(mut self: Pin<&mut Self>, cx: &mut Context, buf: &[u8])
-> Poll<Result<usize, io::Error>>
{
AsyncWrite::poll_write(Pin::new(&mut self.stream), cx, buf)
}
fn flush(&mut self) -> io::Result<()> {
self.stream.flush()
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context)
-> Poll<Result<(), io::Error>>
{
AsyncWrite::poll_flush(Pin::new(&mut self.stream), cx)
}
}
impl<S: AsyncRead + AsyncWrite> AsyncWrite for SecioOutput<S> {
fn shutdown(&mut self) -> Poll<(), io::Error> {
self.stream.shutdown()
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context)
-> Poll<Result<(), io::Error>>
{
AsyncWrite::poll_close(Pin::new(&mut self.stream), cx)
}
}
@ -244,54 +240,52 @@ pub struct SecioMiddleware<S> {
impl<S> SecioMiddleware<S>
where
S: AsyncRead + AsyncWrite + Send,
S: AsyncRead + AsyncWrite + Send + Unpin + 'static,
{
/// Attempts to perform a handshake on the given socket.
///
/// On success, produces a `SecioMiddleware` that can then be used to encode/decode
/// communications, plus the public key of the remote, plus the ephemeral public key.
pub fn handshake(socket: S, config: SecioConfig)
-> impl Future<Item = (SecioMiddleware<S>, PublicKey, Vec<u8>), Error = SecioError>
-> impl Future<Output = Result<(SecioMiddleware<S>, PublicKey, Vec<u8>), SecioError>>
{
handshake::handshake(socket, config).map(|(inner, pubkey, ephemeral)| {
handshake::handshake(socket, config).map_ok(|(inner, pubkey, ephemeral)| {
let inner = SecioMiddleware { inner };
(inner, pubkey, ephemeral)
})
}
}
impl<S> Sink for SecioMiddleware<S>
impl<S> Sink<Vec<u8>> for SecioMiddleware<S>
where
S: AsyncRead + AsyncWrite,
S: AsyncRead + AsyncWrite + Unpin,
{
type SinkItem = BytesMut;
type SinkError = io::Error;
type Error = io::Error;
#[inline]
fn start_send(&mut self, item: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> {
self.inner.start_send(item)
fn poll_ready(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_ready(Pin::new(&mut self.inner), cx)
}
#[inline]
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
self.inner.poll_complete()
fn start_send(mut self: Pin<&mut Self>, item: Vec<u8>) -> Result<(), Self::Error> {
Sink::start_send(Pin::new(&mut self.inner), item)
}
#[inline]
fn close(&mut self) -> Poll<(), Self::SinkError> {
self.inner.close()
fn poll_flush(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_flush(Pin::new(&mut self.inner), cx)
}
fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Result<(), Self::Error>> {
Sink::poll_close(Pin::new(&mut self.inner), cx)
}
}
impl<S> Stream for SecioMiddleware<S>
where
S: AsyncRead + AsyncWrite,
S: AsyncRead + AsyncWrite + Unpin,
{
type Item = Vec<u8>;
type Error = SecioError;
type Item = Result<Vec<u8>, SecioError>;
#[inline]
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
self.inner.poll()
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll<Option<Self::Item>> {
Stream::poll_next(Pin::new(&mut self.inner), cx)
}
}