Make secio almost compile for asmjs/wasm (#519)

* Use the sha2 crate in the handshake

* Return a Digest in algo_support instead of a ring ref

* Switch to ed25519-dalek for keys

* Make ring more or less optional

* Switch to ed25519_dalek for the verification

* Extract the key exchange to its own module

* Remove the ring RNG from the handshake

* Some warning fixes and forgot file

* Move key exchange to own module

* Remove usage of ring::digest

* Remove ring from handshake entirely

* Implement ECDH for WebCrypto

* Remove the libp2p-secio feature

* Fix ring being included

* Address some concerns

* Provde some panics in WebCrypto

* Prove the Hmac panic

* Prove more panics
This commit is contained in:
Pierre Krieger
2018-10-01 15:42:40 +02:00
committed by GitHub
parent 7208bba92b
commit e3efc2dc9a
12 changed files with 570 additions and 214 deletions

View File

@ -5,8 +5,9 @@ authors = ["Parity Technologies <admin@parity.io>"]
license = "MIT"
[features]
default = ["libp2p-secio", "libp2p-secio-secp256k1"]
libp2p-secio-secp256k1 = ["libp2p-secio/secp256k1"]
default = ["secio-rsa", "secio-secp256k1"]
secio-rsa = ["libp2p-secio/rsa"]
secio-secp256k1 = ["libp2p-secio/secp256k1"]
[dependencies]
bytes = "0.4"
@ -21,6 +22,7 @@ libp2p-ping = { path = "./protocols/ping" }
libp2p-ratelimit = { path = "./transports/ratelimit" }
libp2p-relay = { path = "./transports/relay" }
libp2p-core = { path = "./core" }
libp2p-secio = { path = "./protocols/secio", default-features = false }
libp2p-transport-timeout = { path = "./transports/timeout" }
libp2p-uds = { path = "./transports/uds" }
libp2p-websocket = { path = "./transports/websocket" }
@ -30,7 +32,6 @@ tokio-io = "0.1"
[target.'cfg(not(target_os = "emscripten"))'.dependencies]
libp2p-dns = { path = "./transports/dns" }
libp2p-secio = { path = "./protocols/secio", optional = true, default-features = false }
libp2p-tcp-transport = { path = "./transports/tcp" }
tokio-current-thread = "0.1"
@ -48,23 +49,18 @@ tokio-stdin = "0.1"
[[example]]
name = "echo-dialer"
required-features = ["libp2p-secio"]
[[example]]
name = "echo-server"
required-features = ["libp2p-secio"]
[[example]]
name = "floodsub"
required-features = ["libp2p-secio"]
[[example]]
name = "kademlia"
required-features = ["libp2p-secio"]
[[example]]
name = "ping-client"
required-features = ["libp2p-secio"]
[[example]]
name = "random_peerid"

View File

@ -11,22 +11,32 @@ futures = "0.1"
libp2p-core = { path = "../../core" }
log = "0.4.1"
protobuf = "2.0.2"
rand = "0.3.17"
ring = { version = "0.12", features = ["rsa_signing"] }
rand = "0.5"
# TODO: use the paritytech repo after https://github.com/paritytech/rust-secp256k1/pull/14
eth-secp256k1 = { git = "https://github.com/tomaka/rust-secp256k1", branch = "pub-rand", optional = true }
aes-ctr = "0.1.0"
aesni = { version = "0.4.1", features = ["nocheck"], optional = true }
twofish = "0.1.0"
ctr = "0.1"
lazy_static = { version = "0.2.11", optional = true }
rw-stream-sink = { path = "../../misc/rw-stream-sink" }
eth-secp256k1 = { git = "https://github.com/paritytech/rust-secp256k1", optional = true }
tokio-io = "0.1.0"
untrusted = "0.5"
untrusted = { version = "0.5", optional = true }
sha2 = "0.7.1"
ed25519-dalek = "0.8.0"
hmac = "0.6.3"
[target.'cfg(not(target_os = "emscripten"))'.dependencies]
ring = { version = "0.12", default-features = false, optional = true }
[target.'cfg(target_os = "emscripten")'.dependencies]
stdweb = { version = "0.4.8", default-features = false }
[features]
default = ["secp256k1"]
default = ["rsa", "secp256k1"]
rsa = ["ring/rsa_signing", "untrusted"]
secp256k1 = ["eth-secp256k1"]
aes-all = ["aesni","lazy_static"]
aes-all = ["aesni", "lazy_static"]
[dev-dependencies]
libp2p-tcp-transport = { path = "../../transports/tcp" }

View File

@ -24,9 +24,11 @@
//! helps you with.
use error::SecioError;
use ring::{agreement, digest};
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
use ring::digest;
use std::cmp::Ordering;
use stream_cipher::Cipher;
use KeyAgreement;
const ECDH_P256: &str = "P-256";
const ECDH_P384: &str = "P-384";
@ -43,14 +45,6 @@ pub(crate) const DEFAULT_AGREEMENTS_PROPOSITION: &str = "P-256,P-384";
pub(crate) const DEFAULT_CIPHERS_PROPOSITION: &str = "AES-128,AES-256,TwofishCTR";
pub(crate) const DEFAULT_DIGESTS_PROPOSITION: &str = "SHA256,SHA512";
/// Possible key agreement algorithms.
#[derive(Copy, Clone, Debug, PartialEq, Eq)]
pub enum KeyAgreement {
EcdhP256,
EcdhP384
}
/// Return a proposition string from the given sequence of `KeyAgreement` values.
pub fn key_agreements_proposition<'a, I>(xchgs: I) -> String
where
@ -77,7 +71,7 @@ where
///
/// The `Ordering` parameter determines which argument is preferred. If `Less` or `Equal` we
/// try for each of `theirs` every one of `ours`, for `Greater` it's the other way around.
pub fn select_agreement<'a>(r: Ordering, ours: &str, theirs: &str) -> Result<&'a agreement::Algorithm, SecioError> {
pub fn select_agreement(r: Ordering, ours: &str, theirs: &str) -> Result<KeyAgreement, SecioError> {
let (a, b) = match r {
Ordering::Less | Ordering::Equal => (theirs, ours),
Ordering::Greater => (ours, theirs)
@ -85,8 +79,8 @@ pub fn select_agreement<'a>(r: Ordering, ours: &str, theirs: &str) -> Result<&'a
for x in a.split(',') {
if b.split(',').any(|y| x == y) {
match x {
ECDH_P256 => return Ok(&agreement::ECDH_P256),
ECDH_P384 => return Ok(&agreement::ECDH_P384),
ECDH_P256 => return Ok(KeyAgreement::EcdhP256),
ECDH_P384 => return Ok(KeyAgreement::EcdhP384),
_ => continue
}
}
@ -156,6 +150,17 @@ pub enum Digest {
Sha512
}
impl Digest {
/// Returns the size in bytes of a digest of this kind.
#[inline]
pub fn num_bytes(&self) -> usize {
match *self {
Digest::Sha256 => 256 / 8,
Digest::Sha512 => 512 / 8,
}
}
}
/// Return a proposition string from the given sequence of `Digest` values.
pub fn digests_proposition<'a, I>(digests: I) -> String
where
@ -182,7 +187,7 @@ where
///
/// The `Ordering` parameter determines which argument is preferred. If `Less` or `Equal` we
/// try for each of `theirs` every one of `ours`, for `Greater` it's the other way around.
pub fn select_digest<'a>(r: Ordering, ours: &str, theirs: &str) -> Result<&'a digest::Algorithm, SecioError> {
pub fn select_digest(r: Ordering, ours: &str, theirs: &str) -> Result<Digest, SecioError> {
let (a, b) = match r {
Ordering::Less | Ordering::Equal => (theirs, ours),
Ordering::Greater => (ours, theirs)
@ -190,8 +195,8 @@ pub fn select_digest<'a>(r: Ordering, ours: &str, theirs: &str) -> Result<&'a di
for x in a.split(',') {
if b.split(',').any(|y| x == y) {
match x {
SHA_256 => return Ok(&digest::SHA256),
SHA_512 => return Ok(&digest::SHA512),
SHA_256 => return Ok(Digest::Sha256),
SHA_512 => return Ok(Digest::Sha512),
_ => continue
}
}
@ -199,3 +204,13 @@ pub fn select_digest<'a>(r: Ordering, ours: &str, theirs: &str) -> Result<&'a di
Err(SecioError::NoSupportIntersection)
}
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
impl Into<&'static digest::Algorithm> for Digest {
#[inline]
fn into(self) -> &'static digest::Algorithm {
match self {
Digest::Sha256 => &digest::SHA256,
Digest::Sha512 => &digest::SHA512,
}
}
}

View File

@ -21,7 +21,7 @@
//! Individual messages decoding.
use bytes::BytesMut;
use super::StreamCipher;
use super::{Hmac, StreamCipher};
use error::SecioError;
use futures::sink::Sink;
@ -29,7 +29,6 @@ use futures::stream::Stream;
use futures::Async;
use futures::Poll;
use futures::StartSend;
use ring::hmac;
/// Wraps around a `Stream<Item = BytesMut>`. The buffers produced by the underlying stream
/// are decoded using the cipher and hmac.
@ -41,9 +40,7 @@ use ring::hmac;
/// Also implements `Sink` for convenience.
pub struct DecoderMiddleware<S> {
cipher_state: StreamCipher,
hmac_key: hmac::VerificationKey,
// TODO: when a new version of ring is released, we can use `hmac_key.digest_algorithm().output_len` instead
hmac_num_bytes: usize,
hmac: Hmac,
raw_stream: S,
}
@ -52,14 +49,12 @@ impl<S> DecoderMiddleware<S> {
pub fn new(
raw_stream: S,
cipher: StreamCipher,
hmac_key: hmac::VerificationKey,
hmac_num_bytes: usize, // TODO: remove this parameter
hmac: Hmac,
) -> DecoderMiddleware<S> {
DecoderMiddleware {
cipher_state: cipher,
hmac_key,
hmac,
raw_stream,
hmac_num_bytes,
}
}
}
@ -81,19 +76,16 @@ where
Err(err) => return Err(err.into()),
};
// TODO: when a new version of ring is released, we can use `hmac_key.digest_algorithm().output_len` instead
let hmac_num_bytes = self.hmac_num_bytes;
if frame.len() < hmac_num_bytes {
if frame.len() < self.hmac.num_bytes() {
debug!("frame too short when decoding secio frame");
return Err(SecioError::FrameTooShort);
}
let content_length = frame.len() - hmac_num_bytes;
let content_length = frame.len() - self.hmac.num_bytes();
{
let (crypted_data, expected_hash) = frame.split_at(content_length);
debug_assert_eq!(expected_hash.len(), hmac_num_bytes);
debug_assert_eq!(expected_hash.len(), self.hmac.num_bytes());
if hmac::verify(&self.hmac_key, crypted_data, expected_hash).is_err() {
if self.hmac.verify(crypted_data, expected_hash).is_err() {
debug!("hmac mismatch when decoding secio frame");
return Err(SecioError::HmacNotMatching);
}

View File

@ -21,9 +21,8 @@
//! Individual messages encoding.
use bytes::BytesMut;
use super::StreamCipher;
use super::{Hmac, StreamCipher};
use futures::prelude::*;
use ring::hmac;
/// Wraps around a `Sink`. Encodes the buffers passed to it and passes it to the underlying sink.
///
@ -34,16 +33,16 @@ use ring::hmac;
/// Also implements `Stream` for convenience.
pub struct EncoderMiddleware<S> {
cipher_state: StreamCipher,
hmac_key: hmac::SigningKey,
hmac: Hmac,
raw_sink: S,
pending: Option<BytesMut> // buffer encrypted data which can not be sent right away
}
impl<S> EncoderMiddleware<S> {
pub fn new(raw: S, cipher: StreamCipher, key: hmac::SigningKey) -> EncoderMiddleware<S> {
pub fn new(raw: S, cipher: StreamCipher, hmac: Hmac) -> EncoderMiddleware<S> {
EncoderMiddleware {
cipher_state: cipher,
hmac_key: key,
hmac,
raw_sink: raw,
pending: None
}
@ -67,7 +66,7 @@ where
debug_assert!(self.pending.is_none());
// TODO if SinkError gets refactor to SecioError, then use try_apply_keystream
self.cipher_state.apply_keystream(&mut data_buf[..]);
let signature = hmac::sign(&self.hmac_key, &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)

View File

@ -25,7 +25,9 @@ use self::decode::DecoderMiddleware;
use self::encode::EncoderMiddleware;
use aes_ctr::stream_cipher::StreamCipherCore;
use ring::hmac;
use algo_support::Digest;
use hmac::{self, Mac};
use sha2::{Sha256, Sha512};
use tokio_io::codec::length_delimited;
use tokio_io::{AsyncRead, AsyncWrite};
@ -37,6 +39,64 @@ pub type FullCodec<S> = DecoderMiddleware<EncoderMiddleware<length_delimited::Fr
pub type StreamCipher = Box<dyn StreamCipherCore + Send>;
#[derive(Debug, Clone)]
pub enum Hmac {
Sha256(hmac::Hmac<Sha256>),
Sha512(hmac::Hmac<Sha512>),
}
impl Hmac {
/// Returns the size of the hash in bytes.
#[inline]
pub fn num_bytes(&self) -> usize {
match *self {
Hmac::Sha256(_) => 32,
Hmac::Sha512(_) => 64,
}
}
/// Builds a `Hmac` from an algorithm and key.
pub fn from_key(algorithm: Digest, key: &[u8]) -> Self {
// TODO: it would be nice to tweak the hmac crate to add an equivalent to new_varkey that
// never errors
match algorithm {
Digest::Sha256 => Hmac::Sha256(Mac::new_varkey(key)
.expect("Hmac::new_varkey accepts any key length")),
Digest::Sha512 => Hmac::Sha512(Mac::new_varkey(key)
.expect("Hmac::new_varkey accepts any key length")),
}
}
/// Signs the data.
// TODO: better return type?
pub fn sign(&mut self, crypted_data: &[u8]) -> Vec<u8> {
match *self {
Hmac::Sha256(ref mut hmac) => {
hmac.input(crypted_data);
hmac.result().code().to_vec()
},
Hmac::Sha512(ref mut hmac) => {
hmac.input(crypted_data);
hmac.result().code().to_vec()
},
}
}
/// Verifies that the data matches the expected hash.
// TODO: better error?
pub fn verify(&mut self, crypted_data: &[u8], expected_hash: &[u8]) -> Result<(), ()> {
match *self {
Hmac::Sha256(ref mut hmac) => {
hmac.input(crypted_data);
hmac.verify(expected_hash).map_err(|_| ())
},
Hmac::Sha512(ref mut hmac) => {
hmac.input(crypted_data);
hmac.verify(expected_hash).map_err(|_| ())
},
}
}
}
/// Takes control of `socket`. Returns an object that implements `future::Sink` and
/// `future::Stream`. The `Stream` and `Sink` produce and accept `BytesMut` objects.
@ -46,16 +106,15 @@ pub type StreamCipher = Box<dyn StreamCipherCore + Send>;
pub fn full_codec<S>(
socket: length_delimited::Framed<S>,
cipher_encoding: StreamCipher,
encoding_hmac: hmac::SigningKey,
encoding_hmac: Hmac,
cipher_decoder: StreamCipher,
decoding_hmac: hmac::VerificationKey,
decoding_hmac: Hmac,
) -> FullCodec<S>
where
S: AsyncRead + AsyncWrite,
{
let hmac_num_bytes = encoding_hmac.digest_algorithm().output_len;
let encoder = EncoderMiddleware::new(socket, cipher_encoding, encoding_hmac);
DecoderMiddleware::new(encoder, cipher_decoder, decoding_hmac, hmac_num_bytes)
DecoderMiddleware::new(encoder, cipher_decoder, decoding_hmac)
}
#[cfg(test)]
@ -68,14 +127,13 @@ mod tests {
use super::full_codec;
use super::DecoderMiddleware;
use super::EncoderMiddleware;
use super::Hmac;
use algo_support::Digest;
use bytes::BytesMut;
use error::SecioError;
use futures::sync::mpsc::channel;
use futures::{Future, Sink, Stream};
use rand;
use ring::digest::SHA256;
use ring::hmac::SigningKey;
use ring::hmac::VerificationKey;
use std::io::Error as IoError;
use tokio_io::codec::length_delimited::Framed;
@ -94,13 +152,12 @@ mod tests {
let encoder = EncoderMiddleware::new(
data_tx,
ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]),
SigningKey::new(&SHA256, &hmac_key),
Hmac::from_key(Digest::Sha256, &hmac_key),
);
let decoder = DecoderMiddleware::new(
data_rx,
ctr(Cipher::Aes256, &cipher_key, &NULL_IV[..]),
VerificationKey::new(&SHA256, &hmac_key),
32,
Hmac::from_key(Digest::Sha256, &hmac_key),
);
let data = b"hello world";
@ -133,9 +190,9 @@ mod tests {
full_codec(
connec,
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
SigningKey::new(&SHA256, &hmac_key),
Hmac::from_key(Digest::Sha256, &hmac_key),
ctr(cipher, &cipher_key[..key_size], &NULL_IV[..]),
VerificationKey::new(&SHA256, &hmac_key),
Hmac::from_key(Digest::Sha256, &hmac_key),
)
},
);
@ -148,9 +205,9 @@ mod tests {
full_codec(
stream,
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
SigningKey::new(&SHA256, &hmac_key_clone),
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
ctr(cipher, &cipher_key_clone[..key_size], &NULL_IV[..]),
VerificationKey::new(&SHA256, &hmac_key_clone),
Hmac::from_key(Digest::Sha256, &hmac_key_clone),
)
});

View File

@ -0,0 +1,70 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
//! Implementation of the key agreement process using the `ring` library.
use futures::{future, prelude::*};
use ring::agreement as ring_agreement;
use ring::rand as ring_rand;
use untrusted::Input as UntrustedInput;
use {KeyAgreement, SecioError};
impl Into<&'static ring_agreement::Algorithm> for KeyAgreement {
#[inline]
fn into(self) -> &'static ring_agreement::Algorithm {
match self {
KeyAgreement::EcdhP256 => &ring_agreement::ECDH_P256,
KeyAgreement::EcdhP384 => &ring_agreement::ECDH_P384,
}
}
}
/// Opaque private key type.
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> {
let rng = ring_rand::SystemRandom::new();
match ring_agreement::EphemeralPrivateKey::generate(algorithm.into(), &rng) {
Ok(tmp_priv_key) => {
let mut tmp_pub_key: Vec<u8> = (0 .. tmp_priv_key.public_key_len()).map(|_| 0).collect();
tmp_priv_key.compute_public_key(&mut tmp_pub_key).unwrap();
future::ok((tmp_priv_key, tmp_pub_key))
},
Err(_) => {
debug!("failed to generate ECDH key");
future::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>
{
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()
}

View File

@ -0,0 +1,136 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
//! Implementation of the key agreement process using the WebCrypto API.
use futures::prelude::*;
use futures::sync::oneshot;
use stdweb::{self, Reference, web::ArrayBuffer, web::TypedArray};
use {KeyAgreement, SecioError};
/// Opaque private key type.
pub type AgreementPrivateKey = Reference;
/// 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> {
// Making sure we are initialized before we dial. Initialization is protected by a simple
// boolean static variable, so it's not a problem to call it multiple times and the cost
// is negligible.
stdweb::initialize();
let (tx, rx) = oneshot::channel();
let mut tx = Some(tx);
let curve = match algorithm {
KeyAgreement::EcdhP256 => "P-256",
KeyAgreement::EcdhP384 => "P-384",
};
let send = move |private, public| {
let _ = tx.take()
.expect("JavaScript promise has been resolved twice") // TODO: prove
.send((private, public));
};
js!{
var send = @{send};
let obj = {
name : "ECDH",
namedCurve: @{curve},
};
window.crypto.subtle
.generateKey("ECDH", true, ["deriveKey", "deriveBits"])
.then(function(key) {
window.crypto.subtle.exportKey("raw", key.publicKey)
.then(function(pubkey) { send(key.privateKey, pubkey) })
});
};
rx
.map(move |(private, public): (AgreementPrivateKey, Reference)| {
// TODO: is this actually true? the WebCrypto specs are blurry
let array = public.downcast::<ArrayBuffer>()
.expect("The output of crypto.subtle.exportKey is always an ArrayBuffer");
(private, Vec::<u8>::from(array))
})
.map_err(|_| unreachable!())
}
/// Finish the agreement. On success, returns the shared key that both remote agreed upon.
pub fn agree(algorithm: KeyAgreement, key: AgreementPrivateKey, other_public_key: &[u8], out_size: usize)
-> impl Future<Item = Vec<u8>, Error = SecioError>
{
let (tx, rx) = oneshot::channel();
let mut tx = Some(tx);
let curve = match algorithm {
KeyAgreement::EcdhP256 => "P-256",
KeyAgreement::EcdhP384 => "P-384",
};
let other_public_key = TypedArray::from(other_public_key).buffer();
let out_size = out_size as u32;
let send = move |out: Reference| {
let _ = tx.take()
.expect("JavaScript promise has been resolved twice") // TODO: prove
.send(out);
};
js!{
var key = @{key};
var other_public_key = @{other_public_key};
var send = @{send};
var curve = @{curve};
var out_size = @{out_size};
let import_params = {
name : "ECDH",
namedCurve: curve,
};
window.crypto.subtle.importKey("raw", other_public_key, import_params, false, ["deriveBits"])
.then(function(public_key) {
let derive_params = {
name : "ECDH",
namedCurve: curve,
public: public_key,
};
window.crypto.subtle.deriveBits(derive_params, key, out_size)
})
.then(function(bits) {
send(new Uint8Array(bits));
});
};
rx
.map(move |buffer| {
Vec::<u8>::from(buffer.downcast::<ArrayBuffer>().
expect("We put the bits into a Uint8Array, which can be casted into \
an ArrayBuffer"))
})
.map_err(|_| unreachable!())
}

View File

@ -0,0 +1,58 @@
// Copyright 2018 Parity Technologies (UK) Ltd.
//
// Permission is hereby granted, free of charge, to any person obtaining a
// copy of this software and associated documentation files (the "Software"),
// to deal in the Software without restriction, including without limitation
// the rights to use, copy, modify, merge, publish, distribute, sublicense,
// and/or sell copies of the Software, and to permit persons to whom the
// Software is furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
// OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
// DEALINGS IN THE SOFTWARE.
//! This module handles the key agreement process. Typically ECDH.
use futures::prelude::*;
use SecioError;
#[path = "impl_ring.rs"]
#[cfg(not(target_os = "emscripten"))]
mod platform;
#[path = "impl_webcrypto.rs"]
#[cfg(target_os = "emscripten")]
mod platform;
/// Possible key agreement algorithms.
#[derive(Copy, Clone, Debug, PartialEq, Eq)]
pub enum KeyAgreement {
EcdhP256,
EcdhP384
}
/// Opaque private key type.
pub struct AgreementPrivateKey(platform::AgreementPrivateKey);
/// Generates a new key pair as part of the exchange.
///
/// 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))
}
/// 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>
{
platform::agree(algorithm, my_private_key.0, other_public_key, out_size)
}

View File

@ -20,9 +20,11 @@
use algo_support;
use bytes::BytesMut;
use codec::{full_codec, FullCodec};
use codec::{full_codec, FullCodec, Hmac};
use stream_cipher::{Cipher, ctr};
use ed25519_dalek::{PublicKey as Ed25519PublicKey, Signature as Ed25519Signature};
use error::SecioError;
use exchange;
use futures::future;
use futures::sink::Sink;
use futures::stream::Stream;
@ -30,22 +32,23 @@ use futures::Future;
use libp2p_core::PublicKey;
use protobuf::parse_from_bytes as protobuf_parse_from_bytes;
use protobuf::Message as ProtobufMessage;
use ring::agreement::EphemeralPrivateKey;
use ring::hmac::{SigningContext, SigningKey, VerificationKey};
use ring::rand::SecureRandom;
use ring::signature::verify as signature_verify;
use ring::signature::{ED25519, RSASigningState, RSA_PKCS1_2048_8192_SHA256, RSA_PKCS1_SHA256};
use ring::{agreement, digest, rand};
use rand::{self, RngCore};
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
use ring::signature::{RSASigningState, RSA_PKCS1_2048_8192_SHA256, RSA_PKCS1_SHA256, verify as ring_verify};
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
use ring::rand::SystemRandom;
#[cfg(feature = "secp256k1")]
use secp256k1;
use sha2::{Digest as ShaDigestTrait, Sha256, Sha512};
use std::cmp::{self, Ordering};
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::mem;
use structs_proto::{Exchange, Propose};
use tokio_io::codec::length_delimited;
use tokio_io::{AsyncRead, AsyncWrite};
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
use untrusted::Input as UntrustedInput;
use {SecioConfig, SecioKeyPairInner};
use {KeyAgreement, SecioConfig, SecioKeyPairInner};
/// Performs a handshake on the given socket.
///
@ -71,7 +74,6 @@ where
// Filled with this function's parameter.
config: SecioConfig,
rng: rand::SystemRandom,
// Locally-generated random number. The array size can be changed without any repercussion.
local_nonce: [u8; 16],
@ -100,19 +102,18 @@ where
hashes_ordering: Ordering,
// Crypto algorithms chosen for the communication.
chosen_exchange: Option<&'static agreement::Algorithm>,
chosen_exchange: Option<KeyAgreement>,
// We only support AES for now, so store just a key size.
chosen_cipher: Option<Cipher>,
chosen_hash: Option<&'static digest::Algorithm>,
chosen_hash: Option<algo_support::Digest>,
// Ephemeral key generated for the handshake and then thrown away.
local_tmp_priv_key: Option<EphemeralPrivateKey>,
local_tmp_priv_key: Option<exchange::AgreementPrivateKey>,
local_tmp_pub_key: Vec<u8>,
}
let context = HandshakeContext {
config,
rng: rand::SystemRandom::new(),
local_nonce: Default::default(),
local_public_key_in_protobuf_bytes: Vec::new(),
local_proposition_bytes: Vec::new(),
@ -137,7 +138,7 @@ where
let future = future::ok::<_, SecioError>(context)
// Generate our nonce.
.and_then(|mut context| {
context.rng.fill(&mut context.local_nonce)
rand::thread_rng().try_fill_bytes(&mut context.local_nonce)
.map_err(|_| SecioError::NonceGenerationFailed)?;
trace!("starting handshake ; local nonce = {:?}", context.local_nonce);
Ok(context)
@ -175,7 +176,9 @@ where
proposition.set_hashes(algo_support::DEFAULT_DIGESTS_PROPOSITION.into())
}
let proposition_bytes = proposition.write_to_bytes().unwrap();
let proposition_bytes = proposition.write_to_bytes()
.expect("we fill all the elements of proposition, therefore writing can never \
fail ; qed");
context.local_proposition_bytes = proposition_bytes.clone();
trace!("sending proposition to remote");
@ -231,17 +234,17 @@ where
// based on which hash is larger.
context.hashes_ordering = {
let oh1 = {
let mut ctx = digest::Context::new(&digest::SHA256);
ctx.update(&context.remote_public_key_in_protobuf_bytes);
ctx.update(&context.local_nonce);
ctx.finish()
let mut ctx = Sha256::new();
ctx.input(&context.remote_public_key_in_protobuf_bytes);
ctx.input(&context.local_nonce);
ctx.result()
};
let oh2 = {
let mut ctx = digest::Context::new(&digest::SHA256);
ctx.update(&context.local_public_key_in_protobuf_bytes);
ctx.update(&context.remote_nonce);
ctx.finish()
let mut ctx = Sha256::new();
ctx.input(&context.local_public_key_in_protobuf_bytes);
ctx.input(&context.remote_nonce);
ctx.result()
};
oh1.as_ref().cmp(&oh2.as_ref())
@ -298,31 +301,28 @@ where
// Generate an ephemeral key for the negotiation.
.and_then(|(socket, context)| {
match EphemeralPrivateKey::generate(context.chosen_exchange.as_ref().unwrap(), &context.rng) {
Ok(tmp_priv_key) => Ok((socket, context, tmp_priv_key)),
Err(_) => {
debug!("failed to generate ECDH key");
Err(SecioError::EphemeralKeyGenerationFailed)
},
}
let exchange = context.chosen_exchange
.expect("chosen_exchange is set to Some earlier then never touched again ; qed");
exchange::generate_agreement(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, mut context, tmp_priv)| {
.and_then(|(socket, mut context, tmp_priv, tmp_pub_key)| {
let exchange = {
let mut local_tmp_pub_key: Vec<u8> = (0 .. tmp_priv.public_key_len()).map(|_| 0).collect();
tmp_priv.compute_public_key(&mut local_tmp_pub_key).unwrap();
context.local_tmp_priv_key = Some(tmp_priv);
context.local_tmp_pub_key = tmp_pub_key.clone();
let mut data_to_sign = context.local_proposition_bytes.clone();
data_to_sign.extend_from_slice(&context.remote_proposition_bytes);
data_to_sign.extend_from_slice(&local_tmp_pub_key);
data_to_sign.extend_from_slice(&tmp_pub_key);
let mut exchange = Exchange::new();
exchange.set_epubkey(local_tmp_pub_key.clone());
exchange.set_epubkey(tmp_pub_key);
exchange.set_signature({
match context.config.key.inner {
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
SecioKeyPairInner::Rsa { ref private, .. } => {
let mut state = match RSASigningState::new(private.clone()) {
Ok(s) => s,
@ -332,9 +332,8 @@ where
},
};
let mut signature = vec![0; private.public_modulus_len()];
match state.sign(&RSA_PKCS1_SHA256, &context.rng, &data_to_sign,
&mut signature)
{
let rng = SystemRandom::new();
match state.sign(&RSA_PKCS1_SHA256, &rng, &data_to_sign, &mut signature) {
Ok(_) => (),
Err(_) => {
debug!("failed to sign local exchange");
@ -345,12 +344,12 @@ where
signature
},
SecioKeyPairInner::Ed25519 { ref key_pair } => {
let signature = key_pair.sign(&data_to_sign);
signature.as_ref().to_owned()
let signature = key_pair.sign::<Sha512>(&data_to_sign);
signature.to_bytes().to_vec()
},
#[cfg(feature = "secp256k1")]
SecioKeyPairInner::Secp256k1 { ref private } => {
let data_to_sign = digest::digest(&digest::SHA256, &data_to_sign);
let data_to_sign = Sha256::digest(&data_to_sign);
let message = secp256k1::Message::from_slice(data_to_sign.as_ref())
.expect("digest output length doesn't match secp256k1 input length");
let secp256k1 = secp256k1::Secp256k1::with_caps(secp256k1::ContextFlag::SignOnly);
@ -414,14 +413,15 @@ where
data_to_verify.extend_from_slice(remote_exch.get_epubkey());
match context.remote_public_key {
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
Some(PublicKey::Rsa(ref remote_public_key)) => {
// TODO: The ring library doesn't like some stuff in our DER public key,
// therefore we scrap the first 24 bytes of the key. A proper fix would
// be to write a DER parser, but that's not trivial.
match signature_verify(&RSA_PKCS1_2048_8192_SHA256,
UntrustedInput::from(&remote_public_key[24..]),
UntrustedInput::from(&data_to_verify),
UntrustedInput::from(remote_exch.get_signature()))
match ring_verify(&RSA_PKCS1_2048_8192_SHA256,
UntrustedInput::from(&remote_public_key[24..]),
UntrustedInput::from(&data_to_verify),
UntrustedInput::from(remote_exch.get_signature()))
{
Ok(()) => (),
Err(_) => {
@ -431,21 +431,25 @@ where
}
},
Some(PublicKey::Ed25519(ref remote_public_key)) => {
match signature_verify(&ED25519,
UntrustedInput::from(remote_public_key),
UntrustedInput::from(&data_to_verify),
UntrustedInput::from(remote_exch.get_signature()))
{
Ok(()) => (),
Err(_) => {
debug!("failed to verify the remote's signature");
return Err(SecioError::SignatureVerificationFailed)
},
let signature = Ed25519Signature::from_bytes(remote_exch.get_signature());
let pubkey = Ed25519PublicKey::from_bytes(remote_public_key);
if let (Ok(signature), Ok(pubkey)) = (signature, pubkey) {
match pubkey.verify::<Sha512>(&data_to_verify, &signature) {
Ok(()) => (),
Err(_) => {
debug!("failed to verify the remote's signature");
return Err(SecioError::SignatureVerificationFailed)
}
}
} else {
debug!("the remote's signature or publickey are in the wrong format");
return Err(SecioError::SignatureVerificationFailed)
}
},
#[cfg(feature = "secp256k1")]
Some(PublicKey::Secp256k1(ref remote_public_key)) => {
let data_to_verify = digest::digest(&digest::SHA256, &data_to_verify);
let data_to_verify = Sha256::digest(&data_to_verify);
let message = secp256k1::Message::from_slice(data_to_verify.as_ref())
.expect("digest output length doesn't match secp256k1 input length");
let secp256k1 = secp256k1::Secp256k1::with_caps(secp256k1::ContextFlag::VerifyOnly);
@ -464,6 +468,11 @@ where
return Err(SecioError::SignatureVerificationFailed)
}
},
#[cfg(not(all(feature = "ring", not(target_os = "emscripten"))))]
Some(PublicKey::Rsa(_)) => {
debug!("support for RSA was disabled at compile-time");
return Err(SecioError::SignatureVerificationFailed);
},
#[cfg(not(feature = "secp256k1"))]
Some(PublicKey::Secp256k1(_)) => {
debug!("support for secp256k1 was disabled at compile-time");
@ -482,58 +491,57 @@ where
.and_then(|(remote_exch, socket, mut context)| {
let local_priv_key = context.local_tmp_priv_key.take()
.expect("we filled this Option earlier, and extract it now");
let codec = agreement::agree_ephemeral(local_priv_key,
&context.chosen_exchange.unwrap(),
UntrustedInput::from(remote_exch.get_epubkey()),
SecioError::SecretGenerationFailed,
|key_material| {
let key = SigningKey::new(context.chosen_hash.unwrap(), key_material);
let key_size = context.chosen_hash.as_ref()
.expect("chosen_hash is set to Some earlier never modified again ; qed")
.num_bytes();
exchange::agree(context.chosen_exchange.unwrap(), local_priv_key, remote_exch.get_epubkey(), key_size)
.map(move |key_material| (socket, context, key_material))
})
let chosen_cipher = context.chosen_cipher.unwrap();
let cipher_key_size = chosen_cipher.key_size();
let iv_size = chosen_cipher.iv_size();
// Generate a key from the local ephemeral private key and the remote ephemeral public key,
// derive from it a ciper key, an iv, and a hmac key, and build the encoder/decoder.
.and_then(|(socket, context, key_material)| {
let chosen_cipher = context.chosen_cipher
.expect("chosen_cipher is set to Some earlier never modified again ; qed");
let cipher_key_size = chosen_cipher.key_size();
let iv_size = chosen_cipher.iv_size();
let mut longer_key = vec![0u8; 2 * (iv_size + cipher_key_size + 20)];
stretch_key(&key, &mut longer_key);
let chosen_hash = context.chosen_hash
.expect("chosen_hash is set to Some earlier never modified again ; qed");
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 context.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 (local_infos, remote_infos) = {
let (first_half, second_half) = longer_key.split_at(longer_key.len() / 2);
match context.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 = SigningKey::new(&context.chosen_hash.unwrap(), 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.into(), 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 = VerificationKey::new(&context.chosen_hash.unwrap(), 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.into(), mac_key);
let cipher = ctr(chosen_cipher, cipher_key, iv);
(cipher, hmac)
};
Ok(full_codec(socket, encoding_cipher, encoding_hmac, decoding_cipher, decoding_hmac))
});
match codec {
Ok(c) => Ok((c, context)),
Err(err) => {
debug!("failed to generate shared secret with remote");
Err(err)
},
}
let codec = full_codec(socket, encoding_cipher, encoding_hmac, decoding_cipher, decoding_hmac);
Ok((codec, context))
})
// We send back their nonce to check if the connection works.
@ -570,21 +578,30 @@ where
Box::new(future)
}
// Custom algorithm translated from reference implementations. Needs to be the same algorithm
// amongst all implementations.
fn stretch_key(key: &SigningKey, result: &mut [u8]) {
/// Custom algorithm translated from reference implementations. Needs to be the same algorithm
/// amongst all implementations.
fn stretch_key(hmac: Hmac, result: &mut [u8]) {
match hmac {
Hmac::Sha256(hmac) => stretch_key_inner(hmac, result),
Hmac::Sha512(hmac) => stretch_key_inner(hmac, result),
}
}
fn stretch_key_inner<D: ::hmac::digest::Digest + Clone>(hmac: ::hmac::Hmac<D>, result: &mut [u8])
where ::hmac::Hmac<D>: Clone {
use ::hmac::Mac;
const SEED: &[u8] = b"key expansion";
let mut init_ctxt = SigningContext::with_key(key);
init_ctxt.update(SEED);
let mut a = init_ctxt.sign();
let mut init_ctxt = hmac.clone();
init_ctxt.input(SEED);
let mut a = init_ctxt.result().code();
let mut j = 0;
while j < result.len() {
let mut context = SigningContext::with_key(key);
context.update(a.as_ref());
context.update(SEED);
let b = context.sign();
let mut context = hmac.clone();
context.input(a.as_ref());
context.input(SEED);
let b = context.result().code();
let todo = cmp::min(b.as_ref().len(), result.len() - j);
@ -592,9 +609,9 @@ fn stretch_key(key: &SigningKey, result: &mut [u8]) {
j += todo;
let mut context = SigningContext::with_key(key);
context.update(a.as_ref());
a = context.sign();
let mut context = hmac.clone();
context.input(a.as_ref());
a = context.result().code();
}
}
@ -606,13 +623,14 @@ mod tests {
use self::tokio_tcp::TcpStream;
use super::handshake;
use super::stretch_key;
use algo_support::Digest;
use codec::Hmac;
use futures::Future;
use futures::Stream;
use ring::digest::SHA256;
use ring::hmac::SigningKey;
use {SecioConfig, SecioKeyPair};
#[test]
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
fn handshake_with_self_succeeds_rsa() {
let key1 = {
let private = include_bytes!("../tests/test-rsa-private-key.pk8");
@ -673,8 +691,8 @@ mod tests {
fn stretch() {
let mut output = [0u8; 32];
let key1 = SigningKey::new(&SHA256, &[]);
stretch_key(&key1, &mut output);
let key1 = Hmac::from_key(Digest::Sha256, &[]);
stretch_key(key1, &mut output);
assert_eq!(
&output,
&[
@ -683,8 +701,8 @@ mod tests {
]
);
let key2 = SigningKey::new(
&SHA256,
let key2 = Hmac::from_key(
Digest::Sha256,
&[
157, 166, 80, 144, 77, 193, 198, 6, 23, 220, 87, 220, 191, 72, 168, 197, 54, 33,
219, 225, 84, 156, 165, 37, 149, 224, 244, 32, 170, 79, 125, 35, 171, 26, 178, 176,
@ -692,7 +710,7 @@ mod tests {
89, 145, 5, 162, 108, 230, 55, 54, 9, 17,
],
);
stretch_key(&key2, &mut output);
stretch_key(key2, &mut output);
assert_eq!(
&output,
&[
@ -701,8 +719,8 @@ mod tests {
]
);
let key3 = SigningKey::new(
&SHA256,
let key3 = Hmac::from_key(
Digest::Sha256,
&[
98, 219, 94, 104, 97, 70, 139, 13, 185, 110, 56, 36, 66, 3, 80, 224, 32, 205, 102,
170, 59, 32, 140, 245, 86, 102, 231, 68, 85, 249, 227, 243, 57, 53, 171, 36, 62,
@ -710,7 +728,7 @@ mod tests {
19, 48, 127, 127, 55, 82, 117, 154, 124, 108,
],
);
stretch_key(&key3, &mut output);
stretch_key(key3, &mut output);
assert_eq!(
&output,
&[

View File

@ -77,23 +77,33 @@
//! `SecioMiddleware` that implements `Sink` and `Stream` and can be used to send packets of data.
//!
#![recursion_limit = "128"]
extern crate aes_ctr;
#[cfg(feature = "secp256k1")]
extern crate asn1_der;
extern crate bytes;
extern crate ctr;
extern crate ed25519_dalek;
extern crate futures;
extern crate hmac;
extern crate libp2p_core;
#[macro_use]
extern crate log;
extern crate protobuf;
extern crate rand;
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
extern crate ring;
extern crate rw_stream_sink;
#[cfg(feature = "secp256k1")]
extern crate secp256k1;
extern crate sha2;
#[cfg(target_os = "emscripten")]
#[macro_use]
extern crate stdweb;
extern crate tokio_io;
extern crate twofish;
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
extern crate untrusted;
#[cfg(feature = "aes-all")]
@ -104,27 +114,31 @@ pub use self::error::SecioError;
#[cfg(feature = "secp256k1")]
use asn1_der::{traits::FromDerEncoded, traits::FromDerObject, DerObject};
use bytes::{Bytes, BytesMut};
use ed25519_dalek::Keypair as Ed25519KeyPair;
use futures::stream::MapErr as StreamMapErr;
use futures::{Future, Poll, Sink, StartSend, Stream};
use libp2p_core::{PeerId, PublicKey};
use ring::rand::SystemRandom;
use ring::signature::{Ed25519KeyPair, RSAKeyPair};
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
use ring::signature::RSAKeyPair;
use rw_stream_sink::RwStreamSink;
use std::error::Error;
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
use std::iter;
use std::sync::Arc;
use tokio_io::{AsyncRead, AsyncWrite};
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
use untrusted::Input;
mod algo_support;
mod codec;
mod error;
mod exchange;
mod handshake;
mod structs_proto;
mod stream_cipher;
pub use algo_support::{Digest, KeyAgreement};
pub use algo_support::Digest;
pub use exchange::KeyAgreement;
pub use stream_cipher::Cipher;
/// Implementation of the `ConnectionUpgrade` trait of `libp2p_core`. Automatically applies
@ -206,6 +220,7 @@ pub struct SecioKeyPair {
impl SecioKeyPair {
/// Builds a `SecioKeyPair` from a PKCS8 private key and public key.
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
pub fn rsa_from_pkcs8<P>(
private: &[u8],
public: P,
@ -223,33 +238,22 @@ impl SecioKeyPair {
})
}
/// Builds a `SecioKeyPair` from a PKCS8 ED25519 private key.
pub fn ed25519_from_pkcs8<K>(key: K) -> Result<SecioKeyPair, Box<Error + Send + Sync>>
where
K: AsRef<[u8]>,
{
let key_pair = Ed25519KeyPair::from_pkcs8(Input::from(key.as_ref())).map_err(Box::new)?;
Ok(SecioKeyPair {
inner: SecioKeyPairInner::Ed25519 {
key_pair: Arc::new(key_pair),
},
})
}
/// Generates a new Ed25519 key pair and uses it.
pub fn ed25519_generated() -> Result<SecioKeyPair, Box<Error + Send + Sync>> {
let rng = SystemRandom::new();
let gen = Ed25519KeyPair::generate_pkcs8(&rng).map_err(Box::new)?;
Ok(SecioKeyPair::ed25519_from_pkcs8(&gen[..])
.expect("failed to parse generated Ed25519 key"))
let mut csprng = rand::OsRng::new()?;
let keypair: Ed25519KeyPair = Ed25519KeyPair::generate::<sha2::Sha512, _>(&mut csprng);
Ok(SecioKeyPair {
inner: SecioKeyPairInner::Ed25519 {
key_pair: Arc::new(keypair),
}
})
}
/// Generates a new random sec256k1 key pair.
#[cfg(feature = "secp256k1")]
pub fn secp256k1_generated() -> Result<SecioKeyPair, Box<Error + Send + Sync>> {
let secp = secp256k1::Secp256k1::with_caps(secp256k1::ContextFlag::Full);
let (private, _) = secp.generate_keypair(&mut ::rand::thread_rng())
let (private, _) = secp.generate_keypair(&mut secp256k1::rand::thread_rng())
.expect("failed to generate secp256k1 key");
Ok(SecioKeyPair {
@ -291,9 +295,10 @@ impl SecioKeyPair {
/// Returns the public key corresponding to this key pair.
pub fn to_public_key(&self) -> PublicKey {
match self.inner {
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
SecioKeyPairInner::Rsa { ref public, .. } => PublicKey::Rsa(public.clone()),
SecioKeyPairInner::Ed25519 { ref key_pair } => {
PublicKey::Ed25519(key_pair.public_key_bytes().to_vec())
PublicKey::Ed25519(key_pair.public.as_bytes().to_vec())
}
#[cfg(feature = "secp256k1")]
SecioKeyPairInner::Secp256k1 { ref private } => {
@ -317,6 +322,7 @@ impl SecioKeyPair {
// Inner content of `SecioKeyPair`.
#[derive(Clone)]
enum SecioKeyPairInner {
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
Rsa {
public: Vec<u8>,
// We use an `Arc` so that we can clone the enum.

View File

@ -149,7 +149,6 @@ pub extern crate libp2p_peerstore as peerstore;
pub extern crate libp2p_ping as ping;
pub extern crate libp2p_ratelimit as ratelimit;
pub extern crate libp2p_relay as relay;
#[cfg(all(not(target_os = "emscripten"), feature = "libp2p-secio"))]
pub extern crate libp2p_secio as secio;
#[cfg(not(target_os = "emscripten"))]
pub extern crate libp2p_tcp_transport as tcp;