2017-10-30 10:22:38 +01:00
|
|
|
// Copyright 2017 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.
|
|
|
|
|
|
|
|
use algo_support;
|
|
|
|
use bytes::BytesMut;
|
2018-10-01 15:42:40 +02:00
|
|
|
use codec::{full_codec, FullCodec, Hmac};
|
2018-09-05 02:15:16 +02:00
|
|
|
use stream_cipher::{Cipher, ctr};
|
2018-10-01 15:42:40 +02:00
|
|
|
use ed25519_dalek::{PublicKey as Ed25519PublicKey, Signature as Ed25519Signature};
|
2017-10-30 10:22:38 +01:00
|
|
|
use error::SecioError;
|
2018-10-01 15:42:40 +02:00
|
|
|
use exchange;
|
2017-10-30 10:22:38 +01:00
|
|
|
use futures::future;
|
|
|
|
use futures::sink::Sink;
|
|
|
|
use futures::stream::Stream;
|
2018-07-11 11:14:40 +02:00
|
|
|
use futures::Future;
|
2018-06-25 14:54:55 +02:00
|
|
|
use libp2p_core::PublicKey;
|
2018-05-21 17:32:59 +02:00
|
|
|
use protobuf::parse_from_bytes as protobuf_parse_from_bytes;
|
2018-07-11 11:14:40 +02:00
|
|
|
use protobuf::Message as ProtobufMessage;
|
2018-10-01 15:42:40 +02:00
|
|
|
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;
|
2018-06-22 13:07:57 +02:00
|
|
|
#[cfg(feature = "secp256k1")]
|
2018-06-20 09:47:43 +02:00
|
|
|
use secp256k1;
|
2018-10-01 15:42:40 +02:00
|
|
|
use sha2::{Digest as ShaDigestTrait, Sha256, Sha512};
|
2017-10-30 10:22:38 +01:00
|
|
|
use std::cmp::{self, Ordering};
|
|
|
|
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
|
2018-03-07 16:20:55 +01:00
|
|
|
use structs_proto::{Exchange, Propose};
|
2017-10-30 10:22:38 +01:00
|
|
|
use tokio_io::codec::length_delimited;
|
2018-05-14 15:55:16 +02:00
|
|
|
use tokio_io::{AsyncRead, AsyncWrite};
|
2018-10-01 15:42:40 +02:00
|
|
|
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
|
2017-10-30 10:22:38 +01:00
|
|
|
use untrusted::Input as UntrustedInput;
|
2018-10-01 15:42:40 +02:00
|
|
|
use {KeyAgreement, SecioConfig, SecioKeyPairInner};
|
2017-10-30 10:22:38 +01:00
|
|
|
|
2018-10-08 14:37:36 +02:00
|
|
|
// 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 local public key protobuf structure encoded in bytes:
|
|
|
|
public_key_in_protobuf_bytes: 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_in_protobuf_bytes =
|
|
|
|
self.config.key.to_public_key().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());
|
|
|
|
|
|
|
|
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_in_protobuf_bytes,
|
|
|
|
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_in_protobuf_bytes = prop.take_pubkey();
|
|
|
|
let nonce = prop.take_rand();
|
|
|
|
|
|
|
|
let pubkey = match PublicKey::from_protobuf_encoding(&public_key_in_protobuf_bytes) {
|
|
|
|
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_in_protobuf_bytes);
|
|
|
|
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(&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)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-30 10:22:38 +01:00
|
|
|
/// Performs a handshake on the given socket.
|
|
|
|
///
|
|
|
|
/// This function expects that the remote is identified with `remote_public_key`, and the remote
|
2018-05-31 14:50:24 +02:00
|
|
|
/// will expect that we are identified with `local_key`.Any mismatch somewhere will produce a
|
|
|
|
/// `SecioError`.
|
2017-10-30 10:22:38 +01:00
|
|
|
///
|
|
|
|
/// On success, returns an object that implements the `Sink` and `Stream` trait whose items are
|
2018-07-04 17:07:38 +02:00
|
|
|
/// buffers of data, plus the public key of the remote, plus the ephemeral public key used during
|
|
|
|
/// negotiation.
|
2017-10-30 10:22:38 +01:00
|
|
|
pub fn handshake<'a, S: 'a>(
|
2018-03-07 16:20:55 +01:00
|
|
|
socket: S,
|
2018-09-12 09:10:05 +02:00
|
|
|
config: SecioConfig
|
2018-09-06 09:54:35 +02:00
|
|
|
) -> Box<Future<Item = (FullCodec<S>, PublicKey, Vec<u8>), Error = SecioError> + Send + 'a>
|
2018-03-07 16:20:55 +01:00
|
|
|
where
|
2018-09-06 09:54:35 +02:00
|
|
|
S: AsyncRead + AsyncWrite + Send,
|
2017-10-30 10:22:38 +01:00
|
|
|
{
|
2018-03-07 16:20:55 +01:00
|
|
|
// 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);
|
|
|
|
|
2018-10-08 14:37:36 +02:00
|
|
|
let future = 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);
|
2018-03-07 16:20:55 +01:00
|
|
|
Ok(context)
|
|
|
|
})
|
2018-10-08 14:37:36 +02:00
|
|
|
.and_then(|context| {
|
2018-05-17 15:14:13 +02:00
|
|
|
trace!("sending proposition to remote");
|
2018-10-08 14:37:36 +02:00
|
|
|
socket.send(BytesMut::from(context.state.proposition_bytes.clone()))
|
2018-03-07 16:20:55 +01:00
|
|
|
.from_err()
|
|
|
|
.map(|s| (s, context))
|
|
|
|
})
|
|
|
|
// Receive the remote's proposition.
|
2018-10-08 14:37:36 +02:00
|
|
|
.and_then(move |(socket, context)| {
|
2018-03-07 16:20:55 +01:00
|
|
|
socket.into_future()
|
|
|
|
.map_err(|(e, _)| e.into())
|
|
|
|
.and_then(move |(prop_raw, socket)| {
|
2018-10-08 14:37:36 +02:00
|
|
|
let context = match prop_raw {
|
|
|
|
Some(p) => context.with_remote(p)?,
|
2018-03-07 16:20:55 +01:00
|
|
|
None => {
|
|
|
|
let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof");
|
2018-05-17 15:14:13 +02:00
|
|
|
debug!("unexpected eof while waiting for remote's proposition");
|
2018-03-07 16:20:55 +01:00
|
|
|
return Err(err.into())
|
|
|
|
},
|
|
|
|
};
|
2018-05-17 15:14:13 +02:00
|
|
|
trace!("received proposition from remote ; pubkey = {:?} ; nonce = {:?}",
|
2018-10-08 14:37:36 +02:00
|
|
|
context.state.public_key, context.state.nonce);
|
|
|
|
Ok((socket, context))
|
2018-03-07 16:20:55 +01:00
|
|
|
})
|
|
|
|
})
|
|
|
|
// Generate an ephemeral key for the negotiation.
|
|
|
|
.and_then(|(socket, context)| {
|
2018-10-08 14:37:36 +02:00
|
|
|
exchange::generate_agreement(context.state.chosen_exchange)
|
2018-10-01 15:42:40 +02:00
|
|
|
.map(move |(tmp_priv_key, tmp_pub_key)| (socket, context, tmp_priv_key, tmp_pub_key))
|
2018-03-07 16:20:55 +01:00
|
|
|
})
|
|
|
|
// 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.
|
2018-10-08 14:37:36 +02:00
|
|
|
.and_then(|(socket, context, tmp_priv, tmp_pub_key)| {
|
|
|
|
let context = context.with_ephemeral(tmp_priv, tmp_pub_key.clone());
|
2018-03-07 16:20:55 +01:00
|
|
|
let exchange = {
|
2018-10-08 14:37:36 +02:00
|
|
|
let mut data_to_sign = context.state.remote.local.proposition_bytes.clone();
|
|
|
|
data_to_sign.extend_from_slice(&context.state.remote.proposition_bytes);
|
2018-10-01 15:42:40 +02:00
|
|
|
data_to_sign.extend_from_slice(&tmp_pub_key);
|
2018-03-07 16:20:55 +01:00
|
|
|
|
|
|
|
let mut exchange = Exchange::new();
|
2018-10-01 15:42:40 +02:00
|
|
|
exchange.set_epubkey(tmp_pub_key);
|
2018-03-07 16:20:55 +01:00
|
|
|
exchange.set_signature({
|
2018-09-12 09:10:05 +02:00
|
|
|
match context.config.key.inner {
|
2018-10-01 15:42:40 +02:00
|
|
|
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
|
2018-05-31 14:50:24 +02:00
|
|
|
SecioKeyPairInner::Rsa { ref private, .. } => {
|
|
|
|
let mut state = match RSASigningState::new(private.clone()) {
|
|
|
|
Ok(s) => s,
|
|
|
|
Err(_) => {
|
|
|
|
debug!("failed to sign local exchange");
|
|
|
|
return Err(SecioError::SigningFailure);
|
|
|
|
},
|
|
|
|
};
|
|
|
|
let mut signature = vec![0; private.public_modulus_len()];
|
2018-10-01 15:42:40 +02:00
|
|
|
let rng = SystemRandom::new();
|
|
|
|
match state.sign(&RSA_PKCS1_SHA256, &rng, &data_to_sign, &mut signature) {
|
2018-05-31 14:50:24 +02:00
|
|
|
Ok(_) => (),
|
|
|
|
Err(_) => {
|
|
|
|
debug!("failed to sign local exchange");
|
|
|
|
return Err(SecioError::SigningFailure);
|
|
|
|
},
|
|
|
|
};
|
|
|
|
|
|
|
|
signature
|
2018-03-07 16:20:55 +01:00
|
|
|
},
|
2018-05-31 14:50:24 +02:00
|
|
|
SecioKeyPairInner::Ed25519 { ref key_pair } => {
|
2018-10-01 15:42:40 +02:00
|
|
|
let signature = key_pair.sign::<Sha512>(&data_to_sign);
|
|
|
|
signature.to_bytes().to_vec()
|
2018-03-07 16:20:55 +01:00
|
|
|
},
|
2018-06-22 13:07:57 +02:00
|
|
|
#[cfg(feature = "secp256k1")]
|
2018-06-20 09:47:43 +02:00
|
|
|
SecioKeyPairInner::Secp256k1 { ref private } => {
|
2018-10-01 15:42:40 +02:00
|
|
|
let data_to_sign = Sha256::digest(&data_to_sign);
|
2018-06-20 09:47:43 +02:00
|
|
|
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);
|
|
|
|
secp256k1
|
|
|
|
.sign(&message, private)
|
|
|
|
.expect("failed to sign message")
|
|
|
|
.serialize_der(&secp256k1)
|
|
|
|
},
|
2018-05-31 14:50:24 +02:00
|
|
|
}
|
2018-03-07 16:20:55 +01:00
|
|
|
});
|
|
|
|
exchange
|
|
|
|
};
|
2018-10-08 14:37:36 +02:00
|
|
|
let local_exch = exchange.write_to_bytes()?;
|
2018-03-07 16:20:55 +01:00
|
|
|
Ok((BytesMut::from(local_exch), socket, context))
|
|
|
|
})
|
|
|
|
// Send our local `Exchange`.
|
|
|
|
.and_then(|(local_exch, socket, context)| {
|
2018-05-17 15:14:13 +02:00
|
|
|
trace!("sending exchange to remote");
|
2018-03-07 16:20:55 +01:00
|
|
|
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");
|
2018-05-17 15:14:13 +02:00
|
|
|
debug!("unexpected eof while waiting for remote's exchange");
|
2018-03-07 16:20:55 +01:00
|
|
|
return Err(err.into())
|
|
|
|
},
|
|
|
|
};
|
|
|
|
|
|
|
|
let remote_exch = match protobuf_parse_from_bytes::<Exchange>(&raw) {
|
|
|
|
Ok(e) => e,
|
|
|
|
Err(err) => {
|
2018-05-17 15:14:13 +02:00
|
|
|
debug!("failed to parse remote's exchange protobuf ; {:?}", err);
|
2018-03-07 16:20:55 +01:00
|
|
|
return Err(SecioError::HandshakeParsingFailure);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2018-05-17 15:14:13 +02:00
|
|
|
trace!("received and decoded the remote's exchange");
|
2018-03-07 16:20:55 +01:00
|
|
|
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)| {
|
2018-10-08 14:37:36 +02:00
|
|
|
let mut data_to_verify = context.state.remote.proposition_bytes.clone();
|
|
|
|
data_to_verify.extend_from_slice(&context.state.remote.local.proposition_bytes);
|
2018-03-07 16:20:55 +01:00
|
|
|
data_to_verify.extend_from_slice(remote_exch.get_epubkey());
|
|
|
|
|
2018-10-08 14:37:36 +02:00
|
|
|
match context.state.remote.public_key {
|
2018-10-01 15:42:40 +02:00
|
|
|
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
|
2018-10-08 14:37:36 +02:00
|
|
|
PublicKey::Rsa(ref remote_public_key) => {
|
2018-05-31 14:50:24 +02:00
|
|
|
// 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.
|
2018-10-01 15:42:40 +02:00
|
|
|
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()))
|
2018-05-31 14:50:24 +02:00
|
|
|
{
|
|
|
|
Ok(()) => (),
|
|
|
|
Err(_) => {
|
|
|
|
debug!("failed to verify the remote's signature");
|
|
|
|
return Err(SecioError::SignatureVerificationFailed)
|
|
|
|
},
|
|
|
|
}
|
2018-03-07 16:20:55 +01:00
|
|
|
},
|
2018-10-08 14:37:36 +02:00
|
|
|
PublicKey::Ed25519(ref remote_public_key) => {
|
2018-10-01 15:42:40 +02:00
|
|
|
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)
|
2018-05-31 14:50:24 +02:00
|
|
|
}
|
|
|
|
},
|
2018-06-22 13:07:57 +02:00
|
|
|
#[cfg(feature = "secp256k1")]
|
2018-10-08 14:37:36 +02:00
|
|
|
PublicKey::Secp256k1(ref remote_public_key) => {
|
2018-10-01 15:42:40 +02:00
|
|
|
let data_to_verify = Sha256::digest(&data_to_verify);
|
2018-06-20 09:47:43 +02:00
|
|
|
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);
|
|
|
|
let signature = secp256k1::Signature::from_der(&secp256k1, remote_exch.get_signature());
|
|
|
|
let remote_public_key = secp256k1::key::PublicKey::from_slice(&secp256k1, 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)
|
|
|
|
}
|
|
|
|
},
|
2018-10-01 15:42:40 +02:00
|
|
|
#[cfg(not(all(feature = "ring", not(target_os = "emscripten"))))]
|
2018-10-08 14:37:36 +02:00
|
|
|
PublicKey::Rsa(_) => {
|
2018-10-01 15:42:40 +02:00
|
|
|
debug!("support for RSA was disabled at compile-time");
|
|
|
|
return Err(SecioError::SignatureVerificationFailed);
|
|
|
|
},
|
2018-06-22 13:07:57 +02:00
|
|
|
#[cfg(not(feature = "secp256k1"))]
|
2018-10-08 14:37:36 +02:00
|
|
|
PublicKey::Secp256k1(_) => {
|
2018-06-22 13:07:57 +02:00
|
|
|
debug!("support for secp256k1 was disabled at compile-time");
|
|
|
|
return Err(SecioError::SignatureVerificationFailed);
|
2018-10-08 14:37:36 +02:00
|
|
|
}
|
2018-05-31 14:50:24 +02:00
|
|
|
};
|
2018-03-07 16:20:55 +01:00
|
|
|
|
2018-05-17 15:14:13 +02:00
|
|
|
trace!("successfully verified the remote's signature");
|
2018-03-07 16:20:55 +01:00
|
|
|
Ok((remote_exch, socket, context))
|
|
|
|
})
|
|
|
|
// 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.
|
2018-10-08 14:37:36 +02:00
|
|
|
.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)
|
2018-10-01 15:42:40 +02:00
|
|
|
.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 ciper key, an iv, and a hmac key, and build the encoder/decoder.
|
|
|
|
.and_then(|(socket, context, key_material)| {
|
2018-10-08 14:37:36 +02:00
|
|
|
let chosen_cipher = context.state.remote.chosen_cipher;
|
2018-10-01 15:42:40 +02:00
|
|
|
let cipher_key_size = chosen_cipher.key_size();
|
|
|
|
let iv_size = chosen_cipher.iv_size();
|
|
|
|
|
2018-10-08 14:37:36 +02:00
|
|
|
let key = Hmac::from_key(context.state.remote.chosen_hash, &key_material);
|
2018-10-01 15:42:40 +02:00
|
|
|
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);
|
2018-10-08 14:37:36 +02:00
|
|
|
match context.state.remote.hashes_ordering {
|
2018-10-01 15:42:40 +02:00
|
|
|
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),
|
|
|
|
}
|
|
|
|
};
|
2018-03-07 16:20:55 +01:00
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
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);
|
2018-10-08 14:37:36 +02:00
|
|
|
let hmac = Hmac::from_key(context.state.remote.chosen_hash.into(), mac_key);
|
2018-10-01 15:42:40 +02:00
|
|
|
let cipher = ctr(chosen_cipher, cipher_key, iv);
|
|
|
|
(cipher, hmac)
|
|
|
|
};
|
2018-03-07 16:20:55 +01:00
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
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);
|
2018-10-08 14:37:36 +02:00
|
|
|
let hmac = Hmac::from_key(context.state.remote.chosen_hash.into(), mac_key);
|
2018-10-01 15:42:40 +02:00
|
|
|
let cipher = ctr(chosen_cipher, cipher_key, iv);
|
|
|
|
(cipher, hmac)
|
|
|
|
};
|
2018-03-07 16:20:55 +01:00
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
let codec = full_codec(socket, encoding_cipher, encoding_hmac, decoding_cipher, decoding_hmac);
|
|
|
|
Ok((codec, context))
|
2018-03-07 16:20:55 +01:00
|
|
|
})
|
|
|
|
// We send back their nonce to check if the connection works.
|
2018-10-08 14:37:36 +02:00
|
|
|
.and_then(|(codec, context)| {
|
|
|
|
let remote_nonce = context.state.remote.nonce.clone();
|
2018-05-17 15:14:13 +02:00
|
|
|
trace!("checking encryption by sending back remote's nonce");
|
2018-03-07 16:20:55 +01:00
|
|
|
codec.send(BytesMut::from(remote_nonce))
|
|
|
|
.map(|s| (s, context))
|
|
|
|
.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 {
|
2018-10-08 14:37:36 +02:00
|
|
|
Some(ref n) if n == &context.state.remote.local.nonce => {
|
2018-05-17 15:14:13 +02:00
|
|
|
trace!("secio handshake success");
|
2018-10-08 14:37:36 +02:00
|
|
|
Ok((rest, context.state.remote.public_key, context.state.local_tmp_pub_key))
|
2018-03-07 16:20:55 +01:00
|
|
|
},
|
|
|
|
None => {
|
2018-05-17 15:14:13 +02:00
|
|
|
debug!("unexpected eof during nonce check");
|
2018-03-07 16:20:55 +01:00
|
|
|
Err(IoError::new(IoErrorKind::BrokenPipe, "unexpected eof").into())
|
|
|
|
},
|
|
|
|
_ => {
|
2018-05-17 15:14:13 +02:00
|
|
|
debug!("failed nonce verification with remote");
|
2018-03-07 16:20:55 +01:00
|
|
|
Err(SecioError::NonceVerificationFailed)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
})
|
|
|
|
});
|
|
|
|
|
|
|
|
Box::new(future)
|
2017-10-30 10:22:38 +01:00
|
|
|
}
|
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
/// 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;
|
2018-07-17 11:55:18 +02:00
|
|
|
const SEED: &[u8] = b"key expansion";
|
2017-10-30 10:22:38 +01:00
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
let mut init_ctxt = hmac.clone();
|
|
|
|
init_ctxt.input(SEED);
|
|
|
|
let mut a = init_ctxt.result().code();
|
2017-10-30 10:22:38 +01:00
|
|
|
|
2018-03-07 16:20:55 +01:00
|
|
|
let mut j = 0;
|
|
|
|
while j < result.len() {
|
2018-10-01 15:42:40 +02:00
|
|
|
let mut context = hmac.clone();
|
|
|
|
context.input(a.as_ref());
|
|
|
|
context.input(SEED);
|
|
|
|
let b = context.result().code();
|
2017-10-30 10:22:38 +01:00
|
|
|
|
2018-03-07 16:20:55 +01:00
|
|
|
let todo = cmp::min(b.as_ref().len(), result.len() - j);
|
2017-10-30 10:22:38 +01:00
|
|
|
|
2018-03-07 16:20:55 +01:00
|
|
|
result[j..j + todo].copy_from_slice(&b.as_ref()[..todo]);
|
2017-10-30 10:22:38 +01:00
|
|
|
|
2018-03-07 16:20:55 +01:00
|
|
|
j += todo;
|
2017-10-30 10:22:38 +01:00
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
let mut context = hmac.clone();
|
|
|
|
context.input(a.as_ref());
|
|
|
|
a = context.result().code();
|
2018-03-07 16:20:55 +01:00
|
|
|
}
|
2017-10-30 10:22:38 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
#[cfg(test)]
|
|
|
|
mod tests {
|
2018-10-25 05:26:37 -04:00
|
|
|
extern crate tokio;
|
2018-07-16 12:15:27 +02:00
|
|
|
extern crate tokio_tcp;
|
2018-10-25 05:26:37 -04:00
|
|
|
use self::tokio::runtime::current_thread::Runtime;
|
2018-07-16 12:15:27 +02:00
|
|
|
use self::tokio_tcp::TcpListener;
|
|
|
|
use self::tokio_tcp::TcpStream;
|
2018-03-07 16:20:55 +01:00
|
|
|
use super::handshake;
|
|
|
|
use super::stretch_key;
|
2018-10-01 15:42:40 +02:00
|
|
|
use algo_support::Digest;
|
|
|
|
use codec::Hmac;
|
2018-03-07 16:20:55 +01:00
|
|
|
use futures::Future;
|
|
|
|
use futures::Stream;
|
2018-09-12 09:10:05 +02:00
|
|
|
use {SecioConfig, SecioKeyPair};
|
2018-03-07 16:20:55 +01:00
|
|
|
|
|
|
|
#[test]
|
2018-10-01 15:42:40 +02:00
|
|
|
#[cfg(all(feature = "ring", not(target_os = "emscripten")))]
|
2018-05-31 14:50:24 +02:00
|
|
|
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()
|
2018-03-07 16:20:55 +01:00
|
|
|
};
|
|
|
|
|
2018-05-31 14:50:24 +02:00
|
|
|
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()
|
2018-03-07 16:20:55 +01:00
|
|
|
};
|
2018-07-11 11:14:40 +02:00
|
|
|
|
2018-09-12 09:10:05 +02:00
|
|
|
handshake_with_self_succeeds(SecioConfig::new(key1), SecioConfig::new(key2));
|
2018-06-20 09:47:43 +02:00
|
|
|
}
|
2018-03-07 16:20:55 +01:00
|
|
|
|
2018-06-20 09:47:43 +02:00
|
|
|
#[test]
|
|
|
|
fn handshake_with_self_succeeds_ed25519() {
|
|
|
|
let key1 = SecioKeyPair::ed25519_generated().unwrap();
|
|
|
|
let key2 = SecioKeyPair::ed25519_generated().unwrap();
|
2018-09-12 09:10:05 +02:00
|
|
|
handshake_with_self_succeeds(SecioConfig::new(key1), SecioConfig::new(key2));
|
2018-06-20 09:47:43 +02:00
|
|
|
}
|
2018-03-07 16:20:55 +01:00
|
|
|
|
2018-06-20 09:47:43 +02:00
|
|
|
#[test]
|
2018-06-22 13:07:57 +02:00
|
|
|
#[cfg(feature = "secp256k1")]
|
2018-06-20 09:47:43 +02:00
|
|
|
fn handshake_with_self_succeeds_secp256k1() {
|
|
|
|
let key1 = {
|
|
|
|
let key = include_bytes!("../tests/test-secp256k1-private-key.der");
|
|
|
|
SecioKeyPair::secp256k1_from_der(&key[..]).unwrap()
|
|
|
|
};
|
2018-05-31 14:50:24 +02:00
|
|
|
|
2018-06-20 09:47:43 +02:00
|
|
|
let key2 = {
|
|
|
|
let key = include_bytes!("../tests/test-secp256k1-private-key-2.der");
|
|
|
|
SecioKeyPair::secp256k1_from_der(&key[..]).unwrap()
|
|
|
|
};
|
2018-05-31 14:50:24 +02:00
|
|
|
|
2018-09-12 09:10:05 +02:00
|
|
|
handshake_with_self_succeeds(SecioConfig::new(key1), SecioConfig::new(key2));
|
2018-05-31 14:50:24 +02:00
|
|
|
}
|
|
|
|
|
2018-09-12 09:10:05 +02:00
|
|
|
fn handshake_with_self_succeeds(key1: SecioConfig, key2: SecioConfig) {
|
2018-07-16 12:15:27 +02:00
|
|
|
let listener = TcpListener::bind(&"127.0.0.1:0".parse().unwrap()).unwrap();
|
2018-05-31 14:50:24 +02:00
|
|
|
let listener_addr = listener.local_addr().unwrap();
|
|
|
|
|
|
|
|
let server = listener
|
|
|
|
.incoming()
|
|
|
|
.into_future()
|
|
|
|
.map_err(|(e, _)| e.into())
|
2018-07-16 12:15:27 +02:00
|
|
|
.and_then(move |(connec, _)| handshake(connec.unwrap(), key1));
|
2018-03-07 16:20:55 +01:00
|
|
|
|
2018-07-16 12:15:27 +02:00
|
|
|
let client = TcpStream::connect(&listener_addr)
|
2018-03-07 16:20:55 +01:00
|
|
|
.map_err(|e| e.into())
|
2018-05-31 14:50:24 +02:00
|
|
|
.and_then(move |stream| handshake(stream, key2));
|
2018-10-25 05:26:37 -04:00
|
|
|
let mut rt = Runtime::new().unwrap();
|
|
|
|
let _ = rt.block_on(server.join(client)).unwrap();
|
2018-03-07 16:20:55 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
#[test]
|
|
|
|
fn stretch() {
|
|
|
|
let mut output = [0u8; 32];
|
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
let key1 = Hmac::from_key(Digest::Sha256, &[]);
|
|
|
|
stretch_key(key1, &mut output);
|
2018-03-07 16:20:55 +01:00
|
|
|
assert_eq!(
|
|
|
|
&output,
|
|
|
|
&[
|
|
|
|
103, 144, 60, 199, 85, 145, 239, 71, 79, 198, 85, 164, 32, 53, 143, 205, 50, 48,
|
|
|
|
153, 10, 37, 32, 85, 1, 226, 61, 193, 1, 154, 120, 207, 80,
|
|
|
|
]
|
|
|
|
);
|
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
let key2 = Hmac::from_key(
|
|
|
|
Digest::Sha256,
|
2018-03-07 16:20:55 +01:00
|
|
|
&[
|
|
|
|
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,
|
|
|
|
92, 168, 22, 27, 205, 44, 229, 61, 152, 21, 222, 81, 241, 81, 116, 236, 74, 166,
|
|
|
|
89, 145, 5, 162, 108, 230, 55, 54, 9, 17,
|
|
|
|
],
|
|
|
|
);
|
2018-10-01 15:42:40 +02:00
|
|
|
stretch_key(key2, &mut output);
|
2018-03-07 16:20:55 +01:00
|
|
|
assert_eq!(
|
|
|
|
&output,
|
|
|
|
&[
|
|
|
|
39, 151, 182, 63, 180, 175, 224, 139, 42, 131, 130, 116, 55, 146, 62, 31, 157, 95,
|
|
|
|
217, 15, 73, 81, 10, 83, 243, 141, 64, 227, 103, 144, 99, 121,
|
|
|
|
]
|
|
|
|
);
|
|
|
|
|
2018-10-01 15:42:40 +02:00
|
|
|
let key3 = Hmac::from_key(
|
|
|
|
Digest::Sha256,
|
2018-03-07 16:20:55 +01:00
|
|
|
&[
|
|
|
|
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,
|
|
|
|
225, 178, 74, 89, 142, 151, 94, 183, 231, 208, 166, 244, 130, 130, 209, 248, 65,
|
|
|
|
19, 48, 127, 127, 55, 82, 117, 154, 124, 108,
|
|
|
|
],
|
|
|
|
);
|
2018-10-01 15:42:40 +02:00
|
|
|
stretch_key(key3, &mut output);
|
2018-03-07 16:20:55 +01:00
|
|
|
assert_eq!(
|
|
|
|
&output,
|
|
|
|
&[
|
|
|
|
28, 39, 158, 206, 164, 16, 211, 194, 99, 43, 208, 36, 24, 141, 90, 93, 157, 236,
|
|
|
|
238, 111, 170, 0, 60, 11, 49, 174, 177, 121, 30, 12, 182, 25,
|
|
|
|
]
|
|
|
|
);
|
|
|
|
}
|
2017-10-30 10:22:38 +01:00
|
|
|
}
|