mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-04-26 11:12:18 +00:00
secio: Defer nonce verification. (#667)
This commit is contained in:
parent
31962f5920
commit
a5fee2dbac
@ -29,6 +29,7 @@ use futures::stream::Stream;
|
|||||||
use futures::Async;
|
use futures::Async;
|
||||||
use futures::Poll;
|
use futures::Poll;
|
||||||
use futures::StartSend;
|
use futures::StartSend;
|
||||||
|
use std::cmp::min;
|
||||||
|
|
||||||
/// Wraps around a `Stream<Item = BytesMut>`. The buffers produced by the underlying stream
|
/// Wraps around a `Stream<Item = BytesMut>`. The buffers produced by the underlying stream
|
||||||
/// are decoded using the cipher and hmac.
|
/// are decoded using the cipher and hmac.
|
||||||
@ -42,19 +43,21 @@ pub struct DecoderMiddleware<S> {
|
|||||||
cipher_state: StreamCipher,
|
cipher_state: StreamCipher,
|
||||||
hmac: Hmac,
|
hmac: Hmac,
|
||||||
raw_stream: S,
|
raw_stream: S,
|
||||||
|
nonce: Vec<u8>
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<S> DecoderMiddleware<S> {
|
impl<S> DecoderMiddleware<S> {
|
||||||
|
/// Create a new decoder for the given stream, using the provided cipher and HMAC.
|
||||||
|
///
|
||||||
|
/// 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]
|
#[inline]
|
||||||
pub fn new(
|
pub fn new(raw_stream: S, cipher: StreamCipher, hmac: Hmac, nonce: Vec<u8>) -> DecoderMiddleware<S> {
|
||||||
raw_stream: S,
|
|
||||||
cipher: StreamCipher,
|
|
||||||
hmac: Hmac,
|
|
||||||
) -> DecoderMiddleware<S> {
|
|
||||||
DecoderMiddleware {
|
DecoderMiddleware {
|
||||||
cipher_state: cipher,
|
cipher_state: cipher,
|
||||||
hmac,
|
hmac,
|
||||||
raw_stream,
|
raw_stream,
|
||||||
|
nonce
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -97,6 +100,15 @@ where
|
|||||||
.try_apply_keystream(&mut data_buf)
|
.try_apply_keystream(&mut data_buf)
|
||||||
.map_err::<SecioError,_>(|e|e.into())?;
|
.map_err::<SecioError,_>(|e|e.into())?;
|
||||||
|
|
||||||
|
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)
|
||||||
|
}
|
||||||
|
self.nonce.drain(.. n);
|
||||||
|
data_buf.drain(.. n);
|
||||||
|
}
|
||||||
|
|
||||||
Ok(Async::Ready(Some(data_buf)))
|
Ok(Async::Ready(Some(data_buf)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -109,12 +109,13 @@ pub fn full_codec<S>(
|
|||||||
encoding_hmac: Hmac,
|
encoding_hmac: Hmac,
|
||||||
cipher_decoder: StreamCipher,
|
cipher_decoder: StreamCipher,
|
||||||
decoding_hmac: Hmac,
|
decoding_hmac: Hmac,
|
||||||
|
remote_nonce: Vec<u8>
|
||||||
) -> FullCodec<S>
|
) -> FullCodec<S>
|
||||||
where
|
where
|
||||||
S: AsyncRead + AsyncWrite,
|
S: AsyncRead + AsyncWrite,
|
||||||
{
|
{
|
||||||
let encoder = EncoderMiddleware::new(socket, cipher_encoding, encoding_hmac);
|
let encoder = EncoderMiddleware::new(socket, cipher_encoding, encoding_hmac);
|
||||||
DecoderMiddleware::new(encoder, cipher_decoder, decoding_hmac)
|
DecoderMiddleware::new(encoder, cipher_decoder, decoding_hmac, remote_nonce)
|
||||||
}
|
}
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
@ -133,7 +134,7 @@ mod tests {
|
|||||||
use bytes::BytesMut;
|
use bytes::BytesMut;
|
||||||
use error::SecioError;
|
use error::SecioError;
|
||||||
use futures::sync::mpsc::channel;
|
use futures::sync::mpsc::channel;
|
||||||
use futures::{Future, Sink, Stream};
|
use futures::{Future, Sink, Stream, stream};
|
||||||
use rand;
|
use rand;
|
||||||
use std::io::Error as IoError;
|
use std::io::Error as IoError;
|
||||||
use tokio_io::codec::length_delimited::Framed;
|
use tokio_io::codec::length_delimited::Framed;
|
||||||
@ -159,6 +160,7 @@ mod tests {
|
|||||||
data_rx,
|
data_rx,
|
||||||
ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]),
|
ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]),
|
||||||
Hmac::from_key(Digest::Sha256, &hmac_key),
|
Hmac::from_key(Digest::Sha256, &hmac_key),
|
||||||
|
Vec::new()
|
||||||
);
|
);
|
||||||
|
|
||||||
let data = b"hello world";
|
let data = b"hello world";
|
||||||
@ -181,20 +183,23 @@ mod tests {
|
|||||||
let hmac_key_clone = hmac_key.clone();
|
let hmac_key_clone = hmac_key.clone();
|
||||||
let data = b"hello world";
|
let data = b"hello world";
|
||||||
let data_clone = data.clone();
|
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 = TcpListener::bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
|
||||||
let listener_addr = listener.local_addr().unwrap();
|
let listener_addr = listener.local_addr().unwrap();
|
||||||
|
|
||||||
let server = listener.incoming().into_future().map_err(|(e, _)| e).map(
|
let nonce2 = nonce.clone();
|
||||||
move |(connec, _)| {
|
let server = listener.incoming()
|
||||||
let connec = Framed::new(connec.unwrap());
|
.into_future()
|
||||||
|
.map_err(|(e, _)| e)
|
||||||
|
.map(move |(connec, _)| {
|
||||||
full_codec(
|
full_codec(
|
||||||
connec,
|
Framed::new(connec.unwrap()),
|
||||||
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
|
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
|
||||||
Hmac::from_key(Digest::Sha256, &hmac_key),
|
Hmac::from_key(Digest::Sha256, &hmac_key),
|
||||||
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
|
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
|
||||||
Hmac::from_key(Digest::Sha256, &hmac_key),
|
Hmac::from_key(Digest::Sha256, &hmac_key),
|
||||||
|
nonce2
|
||||||
)
|
)
|
||||||
},
|
},
|
||||||
);
|
);
|
||||||
@ -202,14 +207,13 @@ mod tests {
|
|||||||
let client = TcpStream::connect(&listener_addr)
|
let client = TcpStream::connect(&listener_addr)
|
||||||
.map_err(|e| e.into())
|
.map_err(|e| e.into())
|
||||||
.map(move |stream| {
|
.map(move |stream| {
|
||||||
let stream = Framed::new(stream);
|
|
||||||
|
|
||||||
full_codec(
|
full_codec(
|
||||||
stream,
|
Framed::new(stream),
|
||||||
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
|
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
|
||||||
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
|
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
|
||||||
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
|
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
|
||||||
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
|
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
|
||||||
|
Vec::new()
|
||||||
)
|
)
|
||||||
});
|
});
|
||||||
|
|
||||||
@ -218,12 +222,11 @@ mod tests {
|
|||||||
.from_err::<SecioError>()
|
.from_err::<SecioError>()
|
||||||
.and_then(|(server, client)| {
|
.and_then(|(server, client)| {
|
||||||
client
|
client
|
||||||
.send(BytesMut::from(&data_clone[..]))
|
.send_all(stream::iter_ok::<_, IoError>(vec![nonce.into(), data_clone[..].into()]))
|
||||||
.map(move |_| server)
|
.map(move |_| server)
|
||||||
.from_err()
|
.from_err()
|
||||||
})
|
})
|
||||||
.and_then(|server| server.into_future().map_err(|(e, _)| e.into()))
|
.and_then(|server| server.concat2().from_err());
|
||||||
.map(|recved| recved.0.unwrap().to_vec());
|
|
||||||
|
|
||||||
let mut rt = Runtime::new().unwrap();
|
let mut rt = Runtime::new().unwrap();
|
||||||
let received = rt.block_on(fin).unwrap();
|
let received = rt.block_on(fin).unwrap();
|
||||||
|
@ -307,10 +307,8 @@ impl HandshakeContext<Ephemeral> {
|
|||||||
/// On success, returns an object that implements the `Sink` and `Stream` trait whose items are
|
/// 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
|
/// buffers of data, plus the public key of the remote, plus the ephemeral public key used during
|
||||||
/// negotiation.
|
/// negotiation.
|
||||||
pub fn handshake<'a, S: 'a>(
|
pub fn handshake<'a, S: 'a>(socket: S, config: SecioConfig)
|
||||||
socket: S,
|
-> impl Future<Item = (FullCodec<S>, PublicKey, Vec<u8>), Error = SecioError>
|
||||||
config: SecioConfig
|
|
||||||
) -> Box<Future<Item = (FullCodec<S>, PublicKey, Vec<u8>), Error = SecioError> + Send + 'a>
|
|
||||||
where
|
where
|
||||||
S: AsyncRead + AsyncWrite + Send,
|
S: AsyncRead + AsyncWrite + Send,
|
||||||
{
|
{
|
||||||
@ -320,7 +318,7 @@ where
|
|||||||
.length_field_length(4)
|
.length_field_length(4)
|
||||||
.new_framed(socket);
|
.new_framed(socket);
|
||||||
|
|
||||||
let future = future::ok::<_, SecioError>(HandshakeContext::new(config))
|
future::ok::<_, SecioError>(HandshakeContext::new(config))
|
||||||
.and_then(|context| {
|
.and_then(|context| {
|
||||||
// Generate our nonce.
|
// Generate our nonce.
|
||||||
let context = context.with_local()?;
|
let context = context.with_local()?;
|
||||||
@ -570,7 +568,14 @@ where
|
|||||||
(cipher, hmac)
|
(cipher, hmac)
|
||||||
};
|
};
|
||||||
|
|
||||||
let codec = full_codec(socket, encoding_cipher, encoding_hmac, decoding_cipher, decoding_hmac);
|
let codec = full_codec(
|
||||||
|
socket,
|
||||||
|
encoding_cipher,
|
||||||
|
encoding_hmac,
|
||||||
|
decoding_cipher,
|
||||||
|
decoding_hmac,
|
||||||
|
context.state.remote.local.nonce.to_vec()
|
||||||
|
);
|
||||||
Ok((codec, context))
|
Ok((codec, context))
|
||||||
})
|
})
|
||||||
// We send back their nonce to check if the connection works.
|
// We send back their nonce to check if the connection works.
|
||||||
@ -578,32 +583,9 @@ where
|
|||||||
let remote_nonce = context.state.remote.nonce.clone();
|
let remote_nonce = context.state.remote.nonce.clone();
|
||||||
trace!("checking encryption by sending back remote's nonce");
|
trace!("checking encryption by sending back remote's nonce");
|
||||||
codec.send(BytesMut::from(remote_nonce))
|
codec.send(BytesMut::from(remote_nonce))
|
||||||
.map(|s| (s, context))
|
.map(|s| (s, context.state.remote.public_key, context.state.local_tmp_pub_key))
|
||||||
.from_err()
|
.from_err()
|
||||||
})
|
})
|
||||||
// Check that the received nonce is correct.
|
|
||||||
.and_then(|(codec, context)| {
|
|
||||||
codec.into_future()
|
|
||||||
.map_err(|(e, _)| e)
|
|
||||||
.and_then(move |(nonce, rest)| {
|
|
||||||
match nonce {
|
|
||||||
Some(ref n) if n == &context.state.remote.local.nonce => {
|
|
||||||
trace!("secio handshake success");
|
|
||||||
Ok((rest, context.state.remote.public_key, context.state.local_tmp_pub_key))
|
|
||||||
},
|
|
||||||
None => {
|
|
||||||
debug!("unexpected eof during nonce check");
|
|
||||||
Err(IoError::new(IoErrorKind::BrokenPipe, "unexpected eof").into())
|
|
||||||
},
|
|
||||||
_ => {
|
|
||||||
debug!("failed nonce verification with remote");
|
|
||||||
Err(SecioError::NonceVerificationFailed)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
})
|
|
||||||
});
|
|
||||||
|
|
||||||
Box::new(future)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Custom algorithm translated from reference implementations. Needs to be the same algorithm
|
/// Custom algorithm translated from reference implementations. Needs to be the same algorithm
|
||||||
@ -647,15 +629,16 @@ where ::hmac::Hmac<D>: Clone {
|
|||||||
mod tests {
|
mod tests {
|
||||||
extern crate tokio;
|
extern crate tokio;
|
||||||
extern crate tokio_tcp;
|
extern crate tokio_tcp;
|
||||||
|
use bytes::BytesMut;
|
||||||
use self::tokio::runtime::current_thread::Runtime;
|
use self::tokio::runtime::current_thread::Runtime;
|
||||||
use self::tokio_tcp::TcpListener;
|
use self::tokio_tcp::TcpListener;
|
||||||
use self::tokio_tcp::TcpStream;
|
use self::tokio_tcp::TcpStream;
|
||||||
|
use crate::SecioError;
|
||||||
use super::handshake;
|
use super::handshake;
|
||||||
use super::stretch_key;
|
use super::stretch_key;
|
||||||
use algo_support::Digest;
|
use algo_support::Digest;
|
||||||
use codec::Hmac;
|
use codec::Hmac;
|
||||||
use futures::Future;
|
use futures::prelude::*;
|
||||||
use futures::Stream;
|
|
||||||
use {SecioConfig, SecioKeyPair};
|
use {SecioConfig, SecioKeyPair};
|
||||||
|
|
||||||
#[test]
|
#[test]
|
||||||
@ -707,11 +690,29 @@ mod tests {
|
|||||||
.incoming()
|
.incoming()
|
||||||
.into_future()
|
.into_future()
|
||||||
.map_err(|(e, _)| e.into())
|
.map_err(|(e, _)| e.into())
|
||||||
.and_then(move |(connec, _)| handshake(connec.unwrap(), key1));
|
.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))))
|
||||||
|
});
|
||||||
|
|
||||||
let client = TcpStream::connect(&listener_addr)
|
let client = TcpStream::connect(&listener_addr)
|
||||||
.map_err(|e| e.into())
|
.map_err(|e| e.into())
|
||||||
.and_then(move |stream| handshake(stream, key2));
|
.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 mut rt = Runtime::new().unwrap();
|
||||||
let _ = rt.block_on(server.join(client)).unwrap();
|
let _ = rt.block_on(server.join(client)).unwrap();
|
||||||
}
|
}
|
||||||
|
@ -427,19 +427,13 @@ where
|
|||||||
///
|
///
|
||||||
/// On success, produces a `SecioMiddleware` that can then be used to encode/decode
|
/// 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.
|
/// communications, plus the public key of the remote, plus the ephemeral public key.
|
||||||
pub fn handshake<'a>(
|
pub fn handshake(socket: S, config: SecioConfig)
|
||||||
socket: S,
|
-> impl Future<Item = (SecioMiddleware<S>, PublicKey, Vec<u8>), Error = SecioError>
|
||||||
config: SecioConfig,
|
|
||||||
) -> Box<Future<Item = (SecioMiddleware<S>, PublicKey, Vec<u8>), Error = SecioError> + Send + 'a>
|
|
||||||
where
|
|
||||||
S: 'a,
|
|
||||||
{
|
{
|
||||||
let fut = handshake::handshake(socket, config).map(|(inner, pubkey, ephemeral)| {
|
handshake::handshake(socket, config).map(|(inner, pubkey, ephemeral)| {
|
||||||
let inner = SecioMiddleware { inner };
|
let inner = SecioMiddleware { inner };
|
||||||
(inner, pubkey, ephemeral)
|
(inner, pubkey, ephemeral)
|
||||||
});
|
})
|
||||||
|
|
||||||
Box::new(fut)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
x
Reference in New Issue
Block a user