Consolidate keypairs in core. (#972)

* Consolidate keypairs in core.

Introduce the concept of a node's identity keypair in libp2p-core,
instead of only the public key:

  * New module: libp2p_core::identity with submodules for the currently
    supported key types. An identity::Keypair and identity::PublicKey
    support the creation and verification of signatures. The public key
    supports encoding/decoding according to the libp2p specs.

  * The secio protocol is simplified as a result of moving code to libp2p-core.

  * The noise protocol is slightly simplified by consolidating ed25519
    keypairs in libp2p-core and using x25519-dalek for DH. Furthermore,
    Ed25519 to X25519 keypair conversion is now complete and tested.

Generalise over the DH keys in the noise protocol.

Generalise over the DH keys and thus DH parameter in handshake patterns
of the Noise protocol, such that it is easy to support other DH schemes
in the future, e.g. X448.

* Address new review comments.
This commit is contained in:
Roman Borschel
2019-03-11 13:42:53 +01:00
committed by GitHub
parent 26df15641c
commit 2c66f82b11
37 changed files with 1742 additions and 1020 deletions

View File

@ -33,7 +33,7 @@ pub enum SecioError {
IoError(IoError),
/// Protocol buffer error.
Protobuf(ProtobufError),
ProtobufError(ProtobufError),
/// Failed to parse one of the handshake protobuf messages.
HandshakeParsingFailure,
@ -79,7 +79,7 @@ impl error::Error for SecioError {
fn cause(&self) -> Option<&dyn error::Error> {
match *self {
SecioError::IoError(ref err) => Some(err),
SecioError::Protobuf(ref err) => Some(err),
SecioError::ProtobufError(ref err) => Some(err),
// TODO: The type doesn't implement `Error`
/*SecioError::CipherError(ref err) => {
Some(err)
@ -95,8 +95,8 @@ impl fmt::Display for SecioError {
match self {
SecioError::IoError(e) =>
write!(f, "I/O error: {}", e),
SecioError::Protobuf(e) =>
write!(f, "protobuf error: {}", e),
SecioError::ProtobufError(e) =>
write!(f, "Protobuf error: {}", e),
SecioError::HandshakeParsingFailure =>
f.write_str("Failed to parse one of the handshake protobuf messages"),
SecioError::NoSupportIntersection =>
@ -144,6 +144,6 @@ impl From<IoError> for SecioError {
impl From<ProtobufError> for SecioError {
#[inline]
fn from(err: ProtobufError) -> SecioError {
SecioError::Protobuf(err)
SecioError::ProtobufError(err)
}
}

View File

@ -22,7 +22,6 @@ use crate::algo_support;
use bytes::BytesMut;
use crate::codec::{full_codec, FullCodec, Hmac};
use crate::stream_cipher::{Cipher, ctr};
use ed25519_dalek::{PublicKey as Ed25519PublicKey, Signature as Ed25519Signature};
use crate::error::SecioError;
use crate::exchange;
use futures::future;
@ -34,23 +33,13 @@ use log::{debug, trace};
use protobuf::parse_from_bytes as protobuf_parse_from_bytes;
use protobuf::Message as ProtobufMessage;
use rand::{self, RngCore};
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
use ring::signature::{RSA_PKCS1_2048_8192_SHA256, RSA_PKCS1_SHA256, verify as ring_verify};
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
use ring::rand::SystemRandom;
#[cfg(feature = "secp256k1")]
use secp256k1;
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};
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
use untrusted::Input as UntrustedInput;
use crate::{KeyAgreement, SecioConfig, SecioKeyPairInner};
#[cfg(feature = "secp256k1")]
use crate::SECP256K1;
use crate::{KeyAgreement, SecioConfig};
// This struct contains the whole context of a handshake, and is filled progressively
// throughout the various parts of the handshake.
@ -63,8 +52,8 @@ struct HandshakeContext<T> {
struct Local {
// Locally-generated random number. The array size can be changed without any repercussion.
nonce: [u8; 16],
// Our local public key protobuf structure encoded in bytes:
public_key_in_protobuf_bytes: Vec<u8>,
// Our encoded local public key
public_key_encoded: Vec<u8>,
// Our local proposition's raw bytes:
proposition_bytes: Vec<u8>
}
@ -124,13 +113,12 @@ impl HandshakeContext<()> {
.try_fill_bytes(&mut nonce)
.map_err(|_| SecioError::NonceGenerationFailed)?;
let public_key_in_protobuf_bytes =
self.config.key.to_public_key().into_protobuf_encoding();
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_in_protobuf_bytes.clone());
proposition.set_pubkey(public_key_encoded.clone());
if let Some(ref p) = self.config.agreements_prop {
trace!("agreements proposition: {}", p);
@ -162,7 +150,7 @@ impl HandshakeContext<()> {
config: self.config,
state: Local {
nonce,
public_key_in_protobuf_bytes,
public_key_encoded,
proposition_bytes
}
})
@ -180,10 +168,10 @@ impl HandshakeContext<Local> {
}
};
let public_key_in_protobuf_bytes = prop.take_pubkey();
let public_key_encoded = prop.take_pubkey();
let nonce = prop.take_rand();
let pubkey = match PublicKey::from_protobuf_encoding(&public_key_in_protobuf_bytes) {
let pubkey = match PublicKey::from_protobuf_encoding(&public_key_encoded) {
Ok(p) => p,
Err(_) => {
debug!("failed to parse remote's proposition's pubkey protobuf");
@ -196,14 +184,14 @@ impl HandshakeContext<Local> {
let hashes_ordering = {
let oh1 = {
let mut ctx = Sha256::new();
ctx.input(&public_key_in_protobuf_bytes);
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_in_protobuf_bytes);
ctx.input(&self.state.public_key_encoded);
ctx.input(&nonce);
ctx.result()
};
@ -368,37 +356,10 @@ where
let mut exchange = Exchange::new();
exchange.set_epubkey(tmp_pub_key);
exchange.set_signature({
match context.config.key.inner {
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
SecioKeyPairInner::Rsa { ref private, .. } => {
let mut signature = vec![0; private.public_modulus_len()];
let rng = SystemRandom::new();
match private.sign(&RSA_PKCS1_SHA256, &rng, &data_to_sign, &mut signature) {
Ok(_) => (),
Err(_) => {
debug!("failed to sign local exchange");
return Err(SecioError::SigningFailure);
},
};
signature
},
SecioKeyPairInner::Ed25519 { ref key_pair } => {
let signature = key_pair.sign(&data_to_sign);
signature.to_bytes().to_vec()
},
#[cfg(feature = "secp256k1")]
SecioKeyPairInner::Secp256k1 { ref private } => {
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");
SECP256K1
.sign(&message, private)
.serialize_der()
},
}
});
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()?;
@ -445,72 +406,9 @@ where
data_to_verify.extend_from_slice(&context.state.remote.local.proposition_bytes);
data_to_verify.extend_from_slice(remote_exch.get_epubkey());
match context.state.remote.public_key {
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
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 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(_) => {
debug!("failed to verify the remote's signature");
return Err(SecioError::SignatureVerificationFailed)
},
}
},
PublicKey::Ed25519(ref remote_public_key) => {
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(&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")]
PublicKey::Secp256k1(ref remote_public_key) => {
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 signature = secp256k1::Signature::from_der(remote_exch.get_signature());
let remote_public_key = secp256k1::key::PublicKey::from_slice(remote_public_key);
if let (Ok(signature), Ok(remote_public_key)) = (signature, remote_public_key) {
match SECP256K1.verify(&message, &signature, &remote_public_key) {
Ok(()) => (),
Err(_) => {
debug!("failed to verify the remote's signature");
return Err(SecioError::SignatureVerificationFailed)
},
}
} else {
debug!("remote's secp256k1 signature has wrong format");
return Err(SecioError::SignatureVerificationFailed)
}
},
#[cfg(any(target_os = "emscripten", target_os = "unknown"))]
PublicKey::Rsa(_) => {
debug!("support for RSA was disabled at compile-time");
return Err(SecioError::SignatureVerificationFailed);
},
#[cfg(not(feature = "secp256k1"))]
PublicKey::Secp256k1(_) => {
debug!("support for secp256k1 was disabled at compile-time");
return Err(SecioError::SignatureVerificationFailed);
}
};
if !context.state.remote.public_key.verify(&data_to_verify, remote_exch.get_signature()) {
return Err(SecioError::SignatureVerificationFailed)
}
trace!("successfully verified the remote's signature");
Ok((remote_exch, socket, context))
@ -625,29 +523,27 @@ where D: ::hmac::digest::Input + ::hmac::digest::BlockInput +
#[cfg(test)]
mod tests {
use bytes::BytesMut;
use libp2p_core::identity;
use tokio::runtime::current_thread::Runtime;
use tokio_tcp::{TcpListener, TcpStream};
use crate::SecioError;
use crate::{SecioConfig, SecioError};
use super::handshake;
use super::stretch_key;
use crate::algo_support::Digest;
use crate::codec::Hmac;
use futures::prelude::*;
use crate::{SecioConfig, SecioKeyPair};
#[test]
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
fn handshake_with_self_succeeds_rsa() {
let key1 = {
let private = include_bytes!("../tests/test-rsa-private-key.pk8");
let public = include_bytes!("../tests/test-rsa-public-key.der").to_vec();
SecioKeyPair::rsa_from_pkcs8(private, public).unwrap()
let mut private = include_bytes!("../tests/test-rsa-private-key.pk8").to_vec();
identity::Keypair::rsa_from_pkcs8(&mut private).unwrap()
};
let key2 = {
let private = include_bytes!("../tests/test-rsa-private-key-2.pk8");
let public = include_bytes!("../tests/test-rsa-public-key-2.der").to_vec();
SecioKeyPair::rsa_from_pkcs8(private, public).unwrap()
let mut private = include_bytes!("../tests/test-rsa-private-key-2.pk8").to_vec();
identity::Keypair::rsa_from_pkcs8(&mut private).unwrap()
};
handshake_with_self_succeeds(SecioConfig::new(key1), SecioConfig::new(key2));
@ -655,8 +551,8 @@ mod tests {
#[test]
fn handshake_with_self_succeeds_ed25519() {
let key1 = SecioKeyPair::ed25519_generated().unwrap();
let key2 = SecioKeyPair::ed25519_generated().unwrap();
let key1 = identity::Keypair::generate_ed25519();
let key2 = identity::Keypair::generate_ed25519();
handshake_with_self_succeeds(SecioConfig::new(key1), SecioConfig::new(key2));
}
@ -664,13 +560,13 @@ mod tests {
#[cfg(feature = "secp256k1")]
fn handshake_with_self_succeeds_secp256k1() {
let key1 = {
let key = include_bytes!("../tests/test-secp256k1-private-key.der");
SecioKeyPair::secp256k1_from_der(&key[..]).unwrap()
let mut key = include_bytes!("../tests/test-secp256k1-private-key.der").to_vec();
identity::Keypair::secp256k1_from_der(&mut key).unwrap()
};
let key2 = {
let key = include_bytes!("../tests/test-secp256k1-private-key-2.der");
SecioKeyPair::secp256k1_from_der(&key[..]).unwrap()
let mut key = include_bytes!("../tests/test-secp256k1-private-key-2.der").to_vec();
identity::Keypair::secp256k1_from_der(&mut key).unwrap()
};
handshake_with_self_succeeds(SecioConfig::new(key1), SecioConfig::new(key2));

View File

@ -31,8 +31,8 @@
//! ```no_run
//! # fn main() {
//! use futures::Future;
//! use libp2p_secio::{SecioConfig, SecioKeyPair, SecioOutput};
//! use libp2p_core::{Multiaddr, upgrade::apply_inbound};
//! use libp2p_secio::{SecioConfig, SecioOutput};
//! use libp2p_core::{Multiaddr, identity, upgrade::apply_inbound};
//! use libp2p_core::transport::Transport;
//! use libp2p_tcp::TcpConfig;
//! use tokio_io::io::write_all;
@ -40,12 +40,9 @@
//!
//! let dialer = TcpConfig::new()
//! .with_upgrade({
//! # let private_key = b"";
//! //let private_key = include_bytes!("test-rsa-private-key.pk8");
//! # let public_key = vec![];
//! //let public_key = include_bytes!("test-rsa-public-key.der").to_vec();
//! // See the documentation of `SecioKeyPair`.
//! let keypair = SecioKeyPair::rsa_from_pkcs8(private_key, public_key).unwrap();
//! # let private_key = &mut [];
//! // See the documentation of `identity::Keypair`.
//! let keypair = identity::Keypair::rsa_from_pkcs8(private_key).unwrap();
//! SecioConfig::new(keypair)
//! })
//! .map(|out: SecioOutput<_>, _| out.stream);
@ -84,25 +81,15 @@ extern crate stdweb;
pub use self::error::SecioError;
#[cfg(feature = "secp256k1")]
use asn1_der::{FromDerObject, DerObject};
use bytes::BytesMut;
use ed25519_dalek::Keypair as Ed25519KeyPair;
use futures::stream::MapErr as StreamMapErr;
use futures::{Future, Poll, Sink, StartSend, Stream};
use lazy_static::lazy_static;
use libp2p_core::{PeerId, PublicKey, upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade}};
use libp2p_core::{PublicKey, identity, upgrade::{UpgradeInfo, InboundUpgrade, OutboundUpgrade}};
use log::debug;
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
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(not(any(target_os = "emscripten", target_os = "unknown")))]
use untrusted::Input;
mod algo_support;
mod codec;
@ -116,18 +103,12 @@ pub use crate::algo_support::Digest;
pub use crate::exchange::KeyAgreement;
pub use crate::stream_cipher::Cipher;
// Cached `Secp256k1` context, to avoid recreating it every time.
#[cfg(feature = "secp256k1")]
lazy_static! {
static ref SECP256K1: secp256k1::Secp256k1<secp256k1::All> = secp256k1::Secp256k1::new();
}
/// Implementation of the `ConnectionUpgrade` trait of `libp2p_core`. Automatically applies
/// secio on any connection.
#[derive(Clone)]
pub struct SecioConfig {
/// Private and public keys of the local node.
pub(crate) key: SecioKeyPair,
pub(crate) key: identity::Keypair,
pub(crate) agreements_prop: Option<String>,
pub(crate) ciphers_prop: Option<String>,
pub(crate) digests_prop: Option<String>
@ -135,7 +116,7 @@ pub struct SecioConfig {
impl SecioConfig {
/// Create a new `SecioConfig` with the given keypair.
pub fn new(kp: SecioKeyPair) -> Self {
pub fn new(kp: identity::Keypair) -> Self {
SecioConfig {
key: kp,
agreements_prop: None,
@ -188,163 +169,6 @@ impl SecioConfig {
}
}
/// Private and public keys of the local node.
///
/// # Generating offline keys with OpenSSL
///
/// ## RSA
///
/// Generating the keys:
///
/// ```text
/// openssl genrsa -out private.pem 2048
/// openssl rsa -in private.pem -outform DER -pubout -out public.der
/// openssl pkcs8 -in private.pem -topk8 -nocrypt -out private.pk8
/// rm private.pem # optional
/// ```
///
/// Loading the keys:
///
/// ```ignore
/// let key_pair = SecioKeyPair::rsa_from_pkcs8(include_bytes!("private.pk8"),
/// include_bytes!("public.der"));
/// ```
///
#[derive(Clone)]
pub struct SecioKeyPair {
inner: SecioKeyPairInner,
}
impl SecioKeyPair {
/// Builds a `SecioKeyPair` from a PKCS8 private key and public key.
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
pub fn rsa_from_pkcs8<P>(
private: &[u8],
public: P,
) -> Result<SecioKeyPair, Box<dyn Error + Send + Sync>>
where
P: Into<Vec<u8>>,
{
let private = RsaKeyPair::from_pkcs8(Input::from(&private[..])).map_err(Box::new)?;
Ok(SecioKeyPair {
inner: SecioKeyPairInner::Rsa {
public: public.into(),
private: Arc::new(private),
},
})
}
/// Generates a new Ed25519 key pair and uses it.
pub fn ed25519_generated() -> Result<SecioKeyPair, Box<dyn Error + Send + Sync>> {
let mut csprng = rand::thread_rng();
let keypair: Ed25519KeyPair = Ed25519KeyPair::generate::<_>(&mut csprng);
Ok(SecioKeyPair {
inner: SecioKeyPairInner::Ed25519 {
key_pair: Arc::new(keypair),
}
})
}
/// Builds a `SecioKeyPair` from a raw ed25519 32 bytes private key.
///
/// Returns an error if the slice doesn't have the correct length.
pub fn ed25519_raw_key(key: impl AsRef<[u8]>) -> Result<SecioKeyPair, Box<dyn Error + Send + Sync>> {
let secret = ed25519_dalek::SecretKey::from_bytes(key.as_ref())
.map_err(|err| err.to_string())?;
let public = ed25519_dalek::PublicKey::from(&secret);
Ok(SecioKeyPair {
inner: SecioKeyPairInner::Ed25519 {
key_pair: Arc::new(Ed25519KeyPair {
secret,
public,
}),
}
})
}
/// Generates a new random sec256k1 key pair.
#[cfg(feature = "secp256k1")]
pub fn secp256k1_generated() -> Result<SecioKeyPair, Box<dyn Error + Send + Sync>> {
let private = secp256k1::key::SecretKey::new(&mut secp256k1::rand::thread_rng());
Ok(SecioKeyPair {
inner: SecioKeyPairInner::Secp256k1 { private },
})
}
/// Builds a `SecioKeyPair` from a raw secp256k1 32 bytes private key.
#[cfg(feature = "secp256k1")]
pub fn secp256k1_raw_key<K>(key: K) -> Result<SecioKeyPair, Box<dyn Error + Send + Sync>>
where
K: AsRef<[u8]>,
{
let private = secp256k1::key::SecretKey::from_slice(key.as_ref())?;
Ok(SecioKeyPair {
inner: SecioKeyPairInner::Secp256k1 { private },
})
}
/// Builds a `SecioKeyPair` from a secp256k1 private key in DER format.
#[cfg(feature = "secp256k1")]
pub fn secp256k1_from_der<K>(key: K) -> Result<SecioKeyPair, Box<dyn Error + Send + Sync>>
where
K: AsRef<[u8]>,
{
// See ECPrivateKey in https://tools.ietf.org/html/rfc5915
let obj: Vec<DerObject> =
FromDerObject::deserialize(key.as_ref().iter()).map_err(|err| err.to_string())?;
let priv_key_obj = obj.into_iter()
.nth(1)
.ok_or_else(|| "Not enough elements in DER".to_string())?;
let private_key: Vec<u8> =
FromDerObject::from_der_object(priv_key_obj).map_err(|err| err.to_string())?;
SecioKeyPair::secp256k1_raw_key(&private_key)
}
/// Returns the public key corresponding to this key pair.
pub fn to_public_key(&self) -> PublicKey {
match self.inner {
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
SecioKeyPairInner::Rsa { ref public, .. } => PublicKey::Rsa(public.clone()),
SecioKeyPairInner::Ed25519 { ref key_pair } => {
PublicKey::Ed25519(key_pair.public.as_bytes().to_vec())
}
#[cfg(feature = "secp256k1")]
SecioKeyPairInner::Secp256k1 { ref private } => {
let pubkey = secp256k1::key::PublicKey::from_secret_key(&SECP256K1, private);
PublicKey::Secp256k1(pubkey.serialize().to_vec())
}
}
}
/// Builds a `PeerId` corresponding to the public key of this key pair.
#[inline]
pub fn to_peer_id(&self) -> PeerId {
self.to_public_key().into_peer_id()
}
// TODO: method to save generated key on disk?
}
// Inner content of `SecioKeyPair`.
#[derive(Clone)]
enum SecioKeyPairInner {
#[cfg(not(any(target_os = "emscripten", target_os = "unknown")))]
Rsa {
public: Vec<u8>,
// We use an `Arc` so that we can clone the enum.
private: Arc<RsaKeyPair>,
},
Ed25519 {
// We use an `Arc` so that we can clone the enum.
key_pair: Arc<Ed25519KeyPair>,
},
#[cfg(feature = "secp256k1")]
Secp256k1 { private: secp256k1::key::SecretKey },
}
/// Output of the secio protocol.
pub struct SecioOutput<S>
where