mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-22 06:11:34 +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:
@ -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)
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
@ -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]
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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]
|
||||
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user