mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-12 01:21:21 +00:00
Implement secio
This commit is contained in:
@ -5,6 +5,11 @@ members = [
|
||||
"datastore",
|
||||
"libp2p-host",
|
||||
"libp2p-peerstore",
|
||||
"libp2p-secio",
|
||||
"libp2p-transport",
|
||||
"libp2p-tcp-transport",
|
||||
]
|
||||
|
||||
[replace]
|
||||
# Ring has a feature merged in master that hasn't been published yet ; remove this after 0.12.2
|
||||
"ring:0.12.1" = { git = "https://github.com/briansmith/ring" }
|
||||
|
15
libp2p-secio/Cargo.toml
Normal file
15
libp2p-secio/Cargo.toml
Normal file
@ -0,0 +1,15 @@
|
||||
[package]
|
||||
name = "libp2p-secio"
|
||||
version = "0.1.0"
|
||||
authors = ["Parity Technologies <admin@parity.io>"]
|
||||
|
||||
[dependencies]
|
||||
bytes = "0.4"
|
||||
futures = "0.1"
|
||||
protobuf = "1.4.2"
|
||||
rand = "0.3.17"
|
||||
ring = { version = "0.12.1", features = ["rsa_signing"] }
|
||||
rust-crypto = "^0.2"
|
||||
tokio-core = "0.1.6"
|
||||
tokio-io = "0.1.0"
|
||||
untrusted = "0.6.0"
|
15
libp2p-secio/keys.proto
Normal file
15
libp2p-secio/keys.proto
Normal file
@ -0,0 +1,15 @@
|
||||
enum KeyType {
|
||||
RSA = 0;
|
||||
Ed25519 = 1;
|
||||
Secp256k1 = 2;
|
||||
}
|
||||
|
||||
message PublicKey {
|
||||
required KeyType Type = 1;
|
||||
required bytes Data = 2;
|
||||
}
|
||||
|
||||
message PrivateKey {
|
||||
required KeyType Type = 1;
|
||||
required bytes Data = 2;
|
||||
}
|
14
libp2p-secio/regen_structs_proto.sh
Executable file
14
libp2p-secio/regen_structs_proto.sh
Executable file
@ -0,0 +1,14 @@
|
||||
#!/bin/sh
|
||||
|
||||
# This script regenerates the `src/structs_proto.rs` and `src/keys_proto.rs` files from
|
||||
# `structs.proto` and `keys.proto`.
|
||||
|
||||
sudo docker run --rm -v `pwd`:/usr/code:z -w /usr/code rust /bin/bash -c " \
|
||||
apt-get update; \
|
||||
apt-get install -y protobuf-compiler; \
|
||||
cargo install protobuf; \
|
||||
protoc --rust_out . structs.proto; \
|
||||
protoc --rust_out . keys.proto"
|
||||
|
||||
mv -f structs.rs ./src/structs_proto.rs
|
||||
mv -f keys.rs ./src/keys_proto.rs
|
95
libp2p-secio/src/algo_support.rs
Normal file
95
libp2p-secio/src/algo_support.rs
Normal file
@ -0,0 +1,95 @@
|
||||
// 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.
|
||||
|
||||
//! This module contains some utilities for algorithm support exchange.
|
||||
//!
|
||||
//! One important part of the SECIO handshake is negotiating algorithms. This is what this module
|
||||
//! helps you with.
|
||||
|
||||
macro_rules! supported_impl {
|
||||
($mod_name:ident: $ty:ty, $($name:expr => $val:expr),*,) => (
|
||||
pub mod $mod_name {
|
||||
use std::cmp::Ordering;
|
||||
#[allow(unused_imports)]
|
||||
use crypto::aes::KeySize;
|
||||
#[allow(unused_imports)]
|
||||
use ring::{agreement, digest};
|
||||
use error::SecioError;
|
||||
|
||||
/// String to advertise to the remote.
|
||||
pub const PROPOSITION_STRING: &'static str = concat_comma!($($name),*);
|
||||
|
||||
/// Choose which algorithm to use based on the remote's advertised list.
|
||||
pub fn select_best(hashes_ordering: Ordering, input: &str) -> Result<$ty, SecioError> {
|
||||
match hashes_ordering {
|
||||
Ordering::Less | Ordering::Equal => {
|
||||
for second_elem in input.split(',') {
|
||||
$(
|
||||
if $name == second_elem {
|
||||
return Ok($val);
|
||||
}
|
||||
)+
|
||||
}
|
||||
},
|
||||
Ordering::Greater => {
|
||||
$(
|
||||
for second_elem in input.split(',') {
|
||||
if $name == second_elem {
|
||||
return Ok($val);
|
||||
}
|
||||
}
|
||||
)+
|
||||
},
|
||||
};
|
||||
|
||||
Err(SecioError::NoSupportIntersection(PROPOSITION_STRING, input.to_owned()))
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
// Concatenates several strings with commas.
|
||||
macro_rules! concat_comma {
|
||||
($first:expr, $($rest:expr),*) => (
|
||||
concat!($first $(, ',', $rest)*)
|
||||
);
|
||||
}
|
||||
|
||||
// TODO: there's no library in the Rust ecosystem that supports P-521, but the Go & JS
|
||||
// implementations advertise it
|
||||
supported_impl!(
|
||||
exchanges: &'static agreement::Algorithm,
|
||||
"P-256" => &agreement::ECDH_P256,
|
||||
"P-384" => &agreement::ECDH_P384,
|
||||
);
|
||||
|
||||
// TODO: the Go & JS implementations advertise Blowfish ; however doing so in Rust leads to
|
||||
// runtime errors
|
||||
supported_impl!(
|
||||
ciphers: KeySize,
|
||||
"AES-128" => KeySize::KeySize128,
|
||||
"AES-256" => KeySize::KeySize256,
|
||||
);
|
||||
|
||||
supported_impl!(
|
||||
hashes: &'static digest::Algorithm,
|
||||
"SHA256" => &digest::SHA256,
|
||||
"SHA512" => &digest::SHA512,
|
||||
);
|
115
libp2p-secio/src/codec/decode.rs
Normal file
115
libp2p-secio/src/codec/decode.rs
Normal file
@ -0,0 +1,115 @@
|
||||
// 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.
|
||||
|
||||
//! Individual messages decoding.
|
||||
|
||||
use bytes::BytesMut;
|
||||
use crypto::symmetriccipher::SynchronousStreamCipher;
|
||||
|
||||
use error::SecioError;
|
||||
use futures::Async;
|
||||
use futures::Poll;
|
||||
use futures::StartSend;
|
||||
use futures::sink::Sink;
|
||||
use futures::stream::Stream;
|
||||
use ring::hmac;
|
||||
|
||||
/// Wraps around a `Stream<Item = BytesMut>`. The buffers produced by the underlying stream
|
||||
/// are decoded using the cipher and hmac.
|
||||
///
|
||||
/// This struct implements `Stream`, whose stream item are frames of data without the length
|
||||
/// prefix. The mechanism for removing the length prefix and splitting the incoming data into
|
||||
/// frames isn't handled by this module.
|
||||
///
|
||||
/// Also implements `Sink` for convenience.
|
||||
pub struct DecoderMiddleware<S> {
|
||||
cipher_state: Box<SynchronousStreamCipher>,
|
||||
hmac_key: hmac::VerificationKey,
|
||||
raw_stream: S,
|
||||
}
|
||||
|
||||
impl<S> DecoderMiddleware<S> {
|
||||
#[inline]
|
||||
pub fn new(
|
||||
raw_stream: S,
|
||||
cipher: Box<SynchronousStreamCipher>,
|
||||
hmac_key: hmac::VerificationKey,
|
||||
) -> DecoderMiddleware<S> {
|
||||
DecoderMiddleware {
|
||||
cipher_state: cipher,
|
||||
hmac_key: hmac_key,
|
||||
raw_stream: raw_stream,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Stream for DecoderMiddleware<S>
|
||||
where S: Stream<Item = BytesMut>,
|
||||
S::Error: Into<SecioError>
|
||||
{
|
||||
type Item = Vec<u8>;
|
||||
type Error = SecioError;
|
||||
|
||||
#[inline]
|
||||
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
|
||||
let frame = match self.raw_stream.poll() {
|
||||
Ok(Async::Ready(Some(t))) => t,
|
||||
Ok(Async::Ready(None)) => return Ok(Async::Ready(None)),
|
||||
Ok(Async::NotReady) => return Ok(Async::NotReady),
|
||||
Err(err) => return Err(err.into()),
|
||||
};
|
||||
|
||||
let hmac_num_bytes = self.hmac_key.digest_algorithm().output_len;
|
||||
|
||||
if frame.len() < hmac_num_bytes {
|
||||
return Err(SecioError::FrameTooShort);
|
||||
}
|
||||
|
||||
let (crypted_data, expected_hash) = frame.split_at(frame.len() - hmac_num_bytes);
|
||||
debug_assert_eq!(expected_hash.len(), hmac_num_bytes);
|
||||
|
||||
if let Err(_) = hmac::verify(&self.hmac_key, crypted_data, expected_hash) {
|
||||
return Err(SecioError::HmacNotMatching);
|
||||
}
|
||||
|
||||
// Note that there is no way to decipher in place with rust-crypto right now.
|
||||
let mut decrypted_data = crypted_data.to_vec();
|
||||
self.cipher_state.process(&crypted_data, &mut decrypted_data);
|
||||
|
||||
Ok(Async::Ready(Some(decrypted_data)))
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Sink for DecoderMiddleware<S>
|
||||
where S: Sink
|
||||
{
|
||||
type SinkItem = S::SinkItem;
|
||||
type SinkError = S::SinkError;
|
||||
|
||||
#[inline]
|
||||
fn start_send(&mut self, item: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> {
|
||||
self.raw_stream.start_send(item)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
|
||||
self.raw_stream.poll_complete()
|
||||
}
|
||||
}
|
95
libp2p-secio/src/codec/encode.rs
Normal file
95
libp2p-secio/src/codec/encode.rs
Normal file
@ -0,0 +1,95 @@
|
||||
// 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.
|
||||
|
||||
//! Individual messages encoding.
|
||||
|
||||
use bytes::BytesMut;
|
||||
use bytes::buf::BufMut;
|
||||
use crypto::symmetriccipher::SynchronousStreamCipher;
|
||||
use futures::Poll;
|
||||
use futures::StartSend;
|
||||
use futures::sink::Sink;
|
||||
use futures::stream::Stream;
|
||||
use ring::hmac;
|
||||
|
||||
/// Wraps around a `Sink`. Encodes the buffers passed to it and passes it to the underlying sink.
|
||||
///
|
||||
/// This struct implements `Sink`. It expects individual frames of data, and outputs individual
|
||||
/// frames as well, most notably without the length prefix. The mechanism for adding the length
|
||||
/// prefix is not covered by this module.
|
||||
///
|
||||
/// Also implements `Stream` for convenience.
|
||||
pub struct EncoderMiddleware<S> {
|
||||
cipher_state: Box<SynchronousStreamCipher>,
|
||||
hmac_key: hmac::SigningKey,
|
||||
raw_sink: S,
|
||||
}
|
||||
|
||||
impl<S> EncoderMiddleware<S> {
|
||||
pub fn new(
|
||||
raw_sink: S,
|
||||
cipher: Box<SynchronousStreamCipher>,
|
||||
hmac_key: hmac::SigningKey,
|
||||
) -> EncoderMiddleware<S> {
|
||||
EncoderMiddleware {
|
||||
cipher_state: cipher,
|
||||
hmac_key: hmac_key,
|
||||
raw_sink: raw_sink,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Sink for EncoderMiddleware<S>
|
||||
where S: Sink<SinkItem = BytesMut>
|
||||
{
|
||||
type SinkItem = BytesMut;
|
||||
type SinkError = S::SinkError;
|
||||
|
||||
fn start_send(&mut self, item: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> {
|
||||
let mut out_buffer = BytesMut::new();
|
||||
// Note: Alternatively to `extend`, we could also call `advance_mut()`, which will add
|
||||
// uninitialized bytes to the buffer. But that's unsafe.
|
||||
out_buffer.extend((0..item.len()).map(|_| 0));
|
||||
self.cipher_state.process(&item, &mut out_buffer);
|
||||
|
||||
let signature = hmac::sign(&self.hmac_key, &out_buffer);
|
||||
out_buffer.reserve(signature.as_ref().len());
|
||||
out_buffer.put_slice(signature.as_ref());
|
||||
|
||||
self.raw_sink.start_send(out_buffer)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
|
||||
self.raw_sink.poll_complete()
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Stream for EncoderMiddleware<S>
|
||||
where S: Stream
|
||||
{
|
||||
type Item = S::Item;
|
||||
type Error = S::Error;
|
||||
|
||||
#[inline]
|
||||
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
|
||||
self.raw_sink.poll()
|
||||
}
|
||||
}
|
166
libp2p-secio/src/codec/mod.rs
Normal file
166
libp2p-secio/src/codec/mod.rs
Normal file
@ -0,0 +1,166 @@
|
||||
// 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.
|
||||
|
||||
//! Individual messages encoding and decoding. Use this after the algorithms have been
|
||||
//! successfully negotiated.
|
||||
|
||||
use self::decode::DecoderMiddleware;
|
||||
use self::encode::EncoderMiddleware;
|
||||
|
||||
use crypto::symmetriccipher::SynchronousStreamCipher;
|
||||
use ring::hmac;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use tokio_io::codec::length_delimited;
|
||||
|
||||
mod decode;
|
||||
mod encode;
|
||||
|
||||
/// Type returned by `full_codec`.
|
||||
pub type FullCodec<S> = DecoderMiddleware<EncoderMiddleware<length_delimited::Framed<S>>>;
|
||||
|
||||
/// Takes control of `socket`. Returns an object that implements `future::Sink` and
|
||||
/// `future::Stream`. The `Stream` and `Sink` produce and accept `BytesMut` objects.
|
||||
///
|
||||
/// The conversion between the stream/sink items and the socket is done with the given cipher and
|
||||
/// hash algorithm (which are generally decided during the handshake).
|
||||
///
|
||||
/// > **Note**: The encoding block size could theoretically be determined from `cipher_encoding`,
|
||||
/// > but the underlying library doesn't allow doing that.
|
||||
pub fn full_codec<S>(
|
||||
socket: length_delimited::Framed<S>,
|
||||
cipher_encoding: Box<SynchronousStreamCipher>,
|
||||
encoding_hmac: hmac::SigningKey,
|
||||
cipher_decoder: Box<SynchronousStreamCipher>,
|
||||
decoding_hmac: hmac::VerificationKey,
|
||||
) -> FullCodec<S>
|
||||
where S: AsyncRead + AsyncWrite
|
||||
{
|
||||
let encoder = EncoderMiddleware::new(socket, cipher_encoding, encoding_hmac);
|
||||
let codec = DecoderMiddleware::new(encoder, cipher_decoder, decoding_hmac);
|
||||
|
||||
codec
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::DecoderMiddleware;
|
||||
use super::EncoderMiddleware;
|
||||
use super::full_codec;
|
||||
use bytes::BytesMut;
|
||||
use crypto::aessafe::AesSafe256Encryptor;
|
||||
use crypto::blockmodes::CtrMode;
|
||||
use error::SecioError;
|
||||
use futures::{Future, Sink, Stream};
|
||||
use futures::sync::mpsc::channel;
|
||||
use rand;
|
||||
use ring::digest::SHA256;
|
||||
use ring::hmac::SigningKey;
|
||||
use ring::hmac::VerificationKey;
|
||||
use std::io::Error as IoError;
|
||||
use tokio_core::net::TcpListener;
|
||||
use tokio_core::net::TcpStream;
|
||||
use tokio_core::reactor::Core;
|
||||
use tokio_io::codec::length_delimited::Framed;
|
||||
|
||||
#[test]
|
||||
fn raw_encode_then_decode() {
|
||||
let (data_tx, data_rx) = channel::<BytesMut>(256);
|
||||
let data_tx = data_tx.sink_map_err::<_, IoError>(|_| panic!());
|
||||
let data_rx = data_rx.map_err::<IoError, _>(|_| panic!());
|
||||
|
||||
let cipher_key: [u8; 32] = rand::random();
|
||||
let hmac_key: [u8; 32] = rand::random();
|
||||
|
||||
let encoder =
|
||||
EncoderMiddleware::new(
|
||||
data_tx,
|
||||
Box::new(CtrMode::new(AesSafe256Encryptor::new(&cipher_key), vec![0; 16])),
|
||||
SigningKey::new(&SHA256, &hmac_key),
|
||||
);
|
||||
let decoder =
|
||||
DecoderMiddleware::new(
|
||||
data_rx,
|
||||
Box::new(CtrMode::new(AesSafe256Encryptor::new(&cipher_key), vec![0; 16])),
|
||||
VerificationKey::new(&SHA256, &hmac_key),
|
||||
);
|
||||
|
||||
let data = b"hello world";
|
||||
|
||||
let data_sent = encoder.send(BytesMut::from(data.to_vec())).from_err();
|
||||
let data_received = decoder.into_future().map(|(n, _)| n).map_err(|(e, _)| e);
|
||||
|
||||
let mut core = Core::new().unwrap();
|
||||
let (_, decoded) = core.run(data_sent.join(data_received)).map_err(|_| ()).unwrap();
|
||||
assert_eq!(decoded.unwrap(), data);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn full_codec_encode_then_decode() {
|
||||
let mut core = Core::new().unwrap();
|
||||
|
||||
let cipher_key: [u8; 32] = rand::random();
|
||||
let cipher_key_clone = cipher_key.clone();
|
||||
let hmac_key: [u8; 32] = rand::random();
|
||||
let hmac_key_clone = hmac_key.clone();
|
||||
let data = b"hello world";
|
||||
let data_clone = data.clone();
|
||||
|
||||
let listener = TcpListener::bind(&"127.0.0.1:0".parse().unwrap(), &core.handle()).unwrap();
|
||||
let listener_addr = listener.local_addr().unwrap();
|
||||
|
||||
let server =
|
||||
listener.incoming().into_future().map_err(|(e, _)| e).map(move |(connec, _)| {
|
||||
let connec = Framed::new(connec.unwrap().0);
|
||||
|
||||
full_codec(
|
||||
connec,
|
||||
Box::new(CtrMode::new(AesSafe256Encryptor::new(&cipher_key), vec![0; 16])),
|
||||
SigningKey::new(&SHA256, &hmac_key),
|
||||
Box::new(CtrMode::new(AesSafe256Encryptor::new(&cipher_key), vec![0; 16])),
|
||||
VerificationKey::new(&SHA256, &hmac_key),
|
||||
)
|
||||
});
|
||||
|
||||
let client = TcpStream::connect(&listener_addr, &core.handle())
|
||||
.map_err(|e| e.into())
|
||||
.map(move |stream| {
|
||||
let stream = Framed::new(stream);
|
||||
|
||||
full_codec(
|
||||
stream,
|
||||
Box::new(CtrMode::new(AesSafe256Encryptor::new(&cipher_key_clone), vec![0; 16])),
|
||||
SigningKey::new(&SHA256, &hmac_key_clone),
|
||||
Box::new(CtrMode::new(AesSafe256Encryptor::new(&cipher_key_clone), vec![0; 16])),
|
||||
VerificationKey::new(&SHA256, &hmac_key_clone),
|
||||
)
|
||||
});
|
||||
|
||||
let fin = server.join(client)
|
||||
.from_err::<SecioError>()
|
||||
.and_then(|(server, client)| {
|
||||
client.send(BytesMut::from(&data_clone[..])).map(move |_| server).from_err()
|
||||
})
|
||||
.and_then(|server| server.into_future().map_err(|(e, _)| e.into()))
|
||||
.map(|recved| recved.0.unwrap().to_vec());
|
||||
|
||||
let received = core.run(fin).unwrap();
|
||||
assert_eq!(received, data);
|
||||
}
|
||||
}
|
144
libp2p-secio/src/error.rs
Normal file
144
libp2p-secio/src/error.rs
Normal file
@ -0,0 +1,144 @@
|
||||
// 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.
|
||||
|
||||
//! Defines the `SecioError` enum that groups all possible errors in SECIO.
|
||||
|
||||
use crypto::symmetriccipher::SymmetricCipherError;
|
||||
use std::error;
|
||||
use std::fmt;
|
||||
use std::io::Error as IoError;
|
||||
|
||||
/// Error at the SECIO layer communication.
|
||||
#[derive(Debug)]
|
||||
pub enum SecioError {
|
||||
/// I/O error.
|
||||
IoError(IoError),
|
||||
|
||||
/// Failed to parse one of the handshake protobuf messages.
|
||||
HandshakeParsingFailure,
|
||||
|
||||
/// There is no protocol supported by both the local and remote hosts.
|
||||
NoSupportIntersection(&'static str, String),
|
||||
|
||||
/// Failed to generate nonce.
|
||||
NonceGenerationFailed,
|
||||
|
||||
/// Failed to generate ephemeral key.
|
||||
EphemeralKeyGenerationFailed,
|
||||
|
||||
/// Failed to sign a message with our local private key.
|
||||
SigningFailure,
|
||||
|
||||
/// The signature of the exchange packet doesn't verify the remote public key.
|
||||
SignatureVerificationFailed,
|
||||
|
||||
/// Failed to generate the secret shared key from the ephemeral key.
|
||||
SecretGenerationFailed,
|
||||
|
||||
/// The final check of the handshake failed.
|
||||
NonceVerificationFailed,
|
||||
|
||||
/// Error while decoding/encoding data.
|
||||
CipherError(SymmetricCipherError),
|
||||
|
||||
/// The received frame was of invalid length.
|
||||
FrameTooShort,
|
||||
|
||||
/// The hashes of the message didn't match.
|
||||
HmacNotMatching,
|
||||
}
|
||||
|
||||
impl error::Error for SecioError {
|
||||
#[inline]
|
||||
fn description(&self) -> &str {
|
||||
match *self {
|
||||
SecioError::IoError(_) => {
|
||||
"I/O error"
|
||||
}
|
||||
SecioError::HandshakeParsingFailure => {
|
||||
"Failed to parse one of the handshake protobuf messages"
|
||||
}
|
||||
SecioError::NoSupportIntersection(_, _) => {
|
||||
"There is no protocol supported by both the local and remote hosts"
|
||||
}
|
||||
SecioError::NonceGenerationFailed => {
|
||||
"Failed to generate nonce"
|
||||
}
|
||||
SecioError::EphemeralKeyGenerationFailed => {
|
||||
"Failed to generate ephemeral key"
|
||||
}
|
||||
SecioError::SigningFailure => {
|
||||
"Failed to sign a message with our local private key"
|
||||
}
|
||||
SecioError::SignatureVerificationFailed => {
|
||||
"The signature of the exchange packet doesn't verify the remote public key"
|
||||
}
|
||||
SecioError::SecretGenerationFailed => {
|
||||
"Failed to generate the secret shared key from the ephemeral key"
|
||||
}
|
||||
SecioError::NonceVerificationFailed => {
|
||||
"The final check of the handshake failed"
|
||||
}
|
||||
SecioError::CipherError(_) => {
|
||||
"Error while decoding/encoding data"
|
||||
}
|
||||
SecioError::FrameTooShort => {
|
||||
"The received frame was of invalid length"
|
||||
}
|
||||
SecioError::HmacNotMatching => {
|
||||
"The hashes of the message didn't match"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn cause(&self) -> Option<&error::Error> {
|
||||
match *self {
|
||||
SecioError::IoError(ref err) => {
|
||||
Some(err)
|
||||
}
|
||||
// TODO: The type doesn't implement `Error`
|
||||
/*SecioError::CipherError(ref err) => {
|
||||
Some(err)
|
||||
},*/
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl fmt::Display for SecioError {
|
||||
#[inline]
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter) -> Result<(), fmt::Error> {
|
||||
write!(fmt, "{}", error::Error::description(self))
|
||||
}
|
||||
}
|
||||
|
||||
impl From<SymmetricCipherError> for SecioError {
|
||||
#[inline]
|
||||
fn from(err: SymmetricCipherError) -> SecioError {
|
||||
SecioError::CipherError(err)
|
||||
}
|
||||
}
|
||||
|
||||
impl From<IoError> for SecioError {
|
||||
#[inline]
|
||||
fn from(err: IoError) -> SecioError {
|
||||
SecioError::IoError(err)
|
||||
}
|
||||
}
|
488
libp2p-secio/src/handshake.rs
Normal file
488
libp2p-secio/src/handshake.rs
Normal file
@ -0,0 +1,488 @@
|
||||
// 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;
|
||||
use codec::{full_codec, FullCodec};
|
||||
use crypto::aes::{ctr, KeySize};
|
||||
use error::SecioError;
|
||||
use futures::Future;
|
||||
use futures::future;
|
||||
use futures::sink::Sink;
|
||||
use futures::stream::Stream;
|
||||
use keys_proto::{PublicKey as PublicKeyProtobuf, KeyType as KeyTypeProtobuf};
|
||||
use protobuf::Message as ProtobufMessage;
|
||||
use protobuf::core::parse_from_bytes as protobuf_parse_from_bytes;
|
||||
use ring::{agreement, digest, rand};
|
||||
use ring::agreement::EphemeralPrivateKey;
|
||||
use ring::hmac::{SigningKey, SigningContext, VerificationKey};
|
||||
use ring::rand::SecureRandom;
|
||||
use ring::signature::{RSAKeyPair, RSASigningState, RSA_PKCS1_SHA256, RSA_PKCS1_2048_8192_SHA256};
|
||||
use ring::signature::verify as signature_verify;
|
||||
use std::cmp::{self, Ordering};
|
||||
use std::io::{Error as IoError, ErrorKind as IoErrorKind};
|
||||
use std::mem;
|
||||
use std::sync::Arc;
|
||||
use structs_proto::{Propose, Exchange};
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
use tokio_io::codec::length_delimited;
|
||||
use untrusted::Input as UntrustedInput;
|
||||
|
||||
/// Performs a handshake on the given socket.
|
||||
///
|
||||
/// This function expects that the remote is identified with `remote_public_key`, and the remote
|
||||
/// will expect that we are identified with `local_public_key`. Obviously `local_private_key` must
|
||||
/// be paired with `local_public_key`. Any mismatch somewhere will produce a `SecioError`.
|
||||
///
|
||||
/// On success, returns an object that implements the `Sink` and `Stream` trait whose items are
|
||||
/// buffers of data, plus the public key of the remote.
|
||||
pub fn handshake<'a, S: 'a>(
|
||||
socket: S,
|
||||
local_public_key: Vec<u8>,
|
||||
local_private_key: Arc<RSAKeyPair>,
|
||||
) -> Box<Future<Item = (FullCodec<S>, Vec<u8>), Error = SecioError> + 'a>
|
||||
where S: AsyncRead + AsyncWrite
|
||||
{
|
||||
// TODO: could be rewritten as a coroutine once coroutines land in stable Rust
|
||||
|
||||
// This struct contains the whole context of a handshake, and is filled progressively
|
||||
// throughout the various parts of the handshake.
|
||||
struct HandshakeContext {
|
||||
// Filled with this function's parameters.
|
||||
local_public_key: Vec<u8>,
|
||||
local_private_key: Arc<RSAKeyPair>,
|
||||
|
||||
rng: rand::SystemRandom,
|
||||
// Locally-generated random number. The array size can be changed without any repercussion.
|
||||
local_nonce: [u8; 16],
|
||||
|
||||
// Our local proposition's raw bytes.
|
||||
local_public_key_in_protobuf_bytes: Vec<u8>,
|
||||
local_proposition_bytes: Vec<u8>,
|
||||
|
||||
// The remote proposition's raw bytes.
|
||||
remote_proposition_bytes: BytesMut,
|
||||
remote_public_key_in_protobuf_bytes: Vec<u8>,
|
||||
remote_public_key: Vec<u8>,
|
||||
|
||||
// The remote peer's version of `local_nonce`.
|
||||
// If the NONCE size is actually part of the protocol, we can change this to a fixed-size
|
||||
// array instead of a `Vec`.
|
||||
remote_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: Option<&'static agreement::Algorithm>,
|
||||
// We only support AES for now, so store just a key size.
|
||||
chosen_cipher: Option<KeySize>,
|
||||
chosen_hash: Option<&'static digest::Algorithm>,
|
||||
|
||||
// Ephemeral key generated for the handshake and then thrown away.
|
||||
local_tmp_priv_key: Option<EphemeralPrivateKey>,
|
||||
local_tmp_pub_key: [u8; agreement::PUBLIC_KEY_MAX_LEN],
|
||||
}
|
||||
|
||||
let context = HandshakeContext {
|
||||
local_public_key: local_public_key,
|
||||
local_private_key: local_private_key,
|
||||
rng: rand::SystemRandom::new(),
|
||||
local_nonce: Default::default(),
|
||||
local_public_key_in_protobuf_bytes: Vec::new(),
|
||||
local_proposition_bytes: Vec::new(),
|
||||
remote_proposition_bytes: BytesMut::new(),
|
||||
remote_public_key_in_protobuf_bytes: Vec::new(),
|
||||
remote_public_key: Vec::new(),
|
||||
remote_nonce: Vec::new(),
|
||||
hashes_ordering: Ordering::Equal,
|
||||
chosen_exchange: None,
|
||||
chosen_cipher: None,
|
||||
chosen_hash: None,
|
||||
local_tmp_priv_key: None,
|
||||
local_tmp_pub_key: [0; agreement::PUBLIC_KEY_MAX_LEN],
|
||||
};
|
||||
|
||||
// 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);
|
||||
|
||||
let future = future::ok::<_, SecioError>(context)
|
||||
// Generate our nonce.
|
||||
.and_then(|mut context| {
|
||||
context.rng.fill(&mut context.local_nonce)
|
||||
.map_err(|_| SecioError::NonceGenerationFailed)?;
|
||||
Ok(context)
|
||||
})
|
||||
|
||||
// Send our proposition with our nonce, public key and supported protocols.
|
||||
.and_then(|mut context| {
|
||||
let mut public_key = PublicKeyProtobuf::new();
|
||||
public_key.set_Type(KeyTypeProtobuf::RSA);
|
||||
public_key.set_Data(context.local_public_key.clone());
|
||||
context.local_public_key_in_protobuf_bytes = public_key.write_to_bytes().unwrap();
|
||||
|
||||
let mut proposition = Propose::new();
|
||||
proposition.set_rand(context.local_nonce.clone().to_vec());
|
||||
proposition.set_pubkey(context.local_public_key_in_protobuf_bytes.clone());
|
||||
proposition.set_exchanges(algo_support::exchanges::PROPOSITION_STRING.into());
|
||||
proposition.set_ciphers(algo_support::ciphers::PROPOSITION_STRING.into());
|
||||
proposition.set_hashes(algo_support::hashes::PROPOSITION_STRING.into());
|
||||
let proposition_bytes = proposition.write_to_bytes().unwrap();
|
||||
context.local_proposition_bytes = proposition_bytes.clone();
|
||||
|
||||
socket.send(BytesMut::from(proposition_bytes.clone()))
|
||||
.from_err()
|
||||
.map(|s| (s, context))
|
||||
})
|
||||
|
||||
// Receive the remote's proposition.
|
||||
.and_then(move |(socket, mut context)| {
|
||||
socket.into_future()
|
||||
.map_err(|(e, _)| e.into())
|
||||
.and_then(move |(prop_raw, socket)| {
|
||||
match prop_raw {
|
||||
Some(p) => context.remote_proposition_bytes = p,
|
||||
None => {
|
||||
let err = IoError::new(IoErrorKind::BrokenPipe, "unexpected eof");
|
||||
return Err(err.into())
|
||||
},
|
||||
};
|
||||
|
||||
let mut prop = {
|
||||
protobuf_parse_from_bytes::<Propose>(&context.remote_proposition_bytes)
|
||||
.map_err(|_| SecioError::HandshakeParsingFailure)?
|
||||
};
|
||||
context.remote_public_key_in_protobuf_bytes = prop.take_pubkey();
|
||||
let mut pubkey = {
|
||||
let bytes = &context.remote_public_key_in_protobuf_bytes;
|
||||
protobuf_parse_from_bytes::<PublicKeyProtobuf>(bytes)
|
||||
.map_err(|_| SecioError::HandshakeParsingFailure)?
|
||||
};
|
||||
|
||||
// TODO: For now we suppose that the key is in the RSA format because that's
|
||||
// the only thing the Go and JS implementations support.
|
||||
match pubkey.get_Type() {
|
||||
KeyTypeProtobuf::RSA => (),
|
||||
_ => {
|
||||
let err = IoError::new(IoErrorKind::Other, "unsupported protocol");
|
||||
return Err(err.into());
|
||||
},
|
||||
};
|
||||
context.remote_public_key = pubkey.take_Data();
|
||||
context.remote_nonce = prop.take_rand();
|
||||
Ok((prop, socket, context))
|
||||
})
|
||||
})
|
||||
|
||||
// Decide which algorithms to use (thanks to the remote's proposition).
|
||||
.and_then(move |(remote_prop, socket, mut context)| {
|
||||
// In order to determine which protocols to use, we compute two hashes and choose
|
||||
// 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 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()
|
||||
};
|
||||
|
||||
oh1.as_ref().cmp(&oh2.as_ref())
|
||||
};
|
||||
|
||||
context.chosen_exchange = {
|
||||
let list = &remote_prop.get_exchanges();
|
||||
Some(algo_support::exchanges::select_best(context.hashes_ordering, list)?)
|
||||
};
|
||||
context.chosen_cipher = {
|
||||
let list = &remote_prop.get_ciphers();
|
||||
Some(algo_support::ciphers::select_best(context.hashes_ordering, list)?)
|
||||
};
|
||||
context.chosen_hash = {
|
||||
let list = &remote_prop.get_hashes();
|
||||
Some(algo_support::hashes::select_best(context.hashes_ordering, list)?)
|
||||
};
|
||||
|
||||
Ok((socket, context))
|
||||
})
|
||||
|
||||
// Generate an ephemeral key for the negotiation.
|
||||
.and_then(|(socket, context)| {
|
||||
let tmp_priv_key = EphemeralPrivateKey::generate(&agreement::ECDH_P256, &context.rng)
|
||||
.map_err(|_| SecioError::EphemeralKeyGenerationFailed)?;
|
||||
Ok((socket, context, tmp_priv_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)| {
|
||||
let exchange = {
|
||||
let local_tmp_pub_key = &mut context.local_tmp_pub_key[..tmp_priv.public_key_len()];
|
||||
tmp_priv.compute_public_key(local_tmp_pub_key).unwrap();
|
||||
context.local_tmp_priv_key = Some(tmp_priv);
|
||||
|
||||
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);
|
||||
|
||||
let mut exchange = Exchange::new();
|
||||
exchange.set_epubkey(local_tmp_pub_key.to_vec());
|
||||
exchange.set_signature({
|
||||
let mut state = match RSASigningState::new(context.local_private_key.clone()) {
|
||||
Ok(s) => s,
|
||||
Err(_) => return Err(SecioError::SigningFailure),
|
||||
};
|
||||
let mut signature = vec![0; context.local_private_key.public_modulus_len()];
|
||||
match state.sign(&RSA_PKCS1_SHA256, &context.rng, &data_to_sign,
|
||||
&mut signature)
|
||||
{
|
||||
Ok(_) => (),
|
||||
Err(_) => return Err(SecioError::SigningFailure),
|
||||
};
|
||||
|
||||
signature
|
||||
});
|
||||
exchange
|
||||
};
|
||||
|
||||
let local_exch = exchange.write_to_bytes()
|
||||
.expect("can only fail if the protobuf msg is malformed, which can't happen for \
|
||||
this message in particular");
|
||||
Ok((BytesMut::from(local_exch), socket, context))
|
||||
})
|
||||
|
||||
// Send our local `Exchange`.
|
||||
.and_then(|(local_exch, socket, context)| {
|
||||
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");
|
||||
return Err(err.into())
|
||||
},
|
||||
};
|
||||
|
||||
let remote_exch = protobuf_parse_from_bytes::<Exchange>(&raw)
|
||||
.map_err(|_| SecioError::HandshakeParsingFailure)?;
|
||||
Ok((remote_exch, socket, context))
|
||||
})
|
||||
})
|
||||
|
||||
// Check the validity of the remote's `Exchange`. This verifies that the remote was really
|
||||
// the sender of its proposition, and that it is the owner of both its global and ephemeral
|
||||
// keys.
|
||||
.and_then(|(remote_exch, socket, context)| {
|
||||
let mut data_to_verify = context.remote_proposition_bytes.clone();
|
||||
data_to_verify.extend_from_slice(&context.local_proposition_bytes);
|
||||
data_to_verify.extend_from_slice(remote_exch.get_epubkey());
|
||||
|
||||
// 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(&context.remote_public_key[24..]),
|
||||
UntrustedInput::from(&data_to_verify),
|
||||
UntrustedInput::from(remote_exch.get_signature()))
|
||||
{
|
||||
Ok(()) => (),
|
||||
Err(_) => return Err(SecioError::SignatureVerificationFailed),
|
||||
}
|
||||
|
||||
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.
|
||||
.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.clone().unwrap(),
|
||||
UntrustedInput::from(remote_exch.get_epubkey()),
|
||||
SecioError::SecretGenerationFailed,
|
||||
|key_material| {
|
||||
let key = SigningKey::new(context.chosen_hash.unwrap(), key_material);
|
||||
|
||||
let chosen_cipher = context.chosen_cipher.unwrap();
|
||||
let (cipher_key_size, iv_size) = match chosen_cipher {
|
||||
KeySize::KeySize128 => (16, 16),
|
||||
KeySize::KeySize256 => (32, 16),
|
||||
_ => panic!()
|
||||
};
|
||||
|
||||
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 => panic!(),
|
||||
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.clone().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 = VerificationKey::new(&context.chosen_hash.clone().unwrap(), mac_key);
|
||||
let cipher = ctr(chosen_cipher, cipher_key, iv);
|
||||
(cipher, hmac)
|
||||
};
|
||||
|
||||
Ok(full_codec(socket, Box::new(encoding_cipher), encoding_hmac,
|
||||
Box::new(decoding_cipher), decoding_hmac))
|
||||
})?;
|
||||
|
||||
Ok((codec, context))
|
||||
})
|
||||
|
||||
// We send back their nonce to check if the connection works.
|
||||
.and_then(|(codec, mut context)| {
|
||||
let remote_nonce = mem::replace(&mut context.remote_nonce, Vec::new());
|
||||
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 {
|
||||
Some(ref n) if n == &context.local_nonce => {
|
||||
Ok((rest, context.remote_public_key))
|
||||
},
|
||||
None => {
|
||||
Err(IoError::new(IoErrorKind::BrokenPipe, "unexpected eof").into())
|
||||
},
|
||||
_ => Err(SecioError::NonceVerificationFailed)
|
||||
}
|
||||
})
|
||||
});
|
||||
|
||||
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]) {
|
||||
const SEED: &'static [u8] = b"key expansion";
|
||||
|
||||
let mut init_ctxt = SigningContext::with_key(key);
|
||||
init_ctxt.update(SEED);
|
||||
let mut a = init_ctxt.sign();
|
||||
|
||||
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 todo = cmp::min(b.as_ref().len(), result.len() - j);
|
||||
|
||||
result[j..j + todo].copy_from_slice(&b.as_ref()[..todo]);
|
||||
|
||||
j += todo;
|
||||
|
||||
let mut context = SigningContext::with_key(key);
|
||||
context.update(a.as_ref());
|
||||
a = context.sign();
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::handshake;
|
||||
use futures::Future;
|
||||
use futures::Stream;
|
||||
use ring::signature::RSAKeyPair;
|
||||
use std::sync::Arc;
|
||||
use tokio_core::net::TcpListener;
|
||||
use tokio_core::net::TcpStream;
|
||||
use tokio_core::reactor::Core;
|
||||
use untrusted::Input;
|
||||
|
||||
#[test]
|
||||
fn handshake_with_self_succeeds() {
|
||||
let mut core = Core::new().unwrap();
|
||||
|
||||
let private_key1 = {
|
||||
let pkcs8 = include_bytes!("../tests/test-private-key.pk8");
|
||||
Arc::new(RSAKeyPair::from_pkcs8(Input::from(&pkcs8[..])).unwrap())
|
||||
};
|
||||
let public_key1 = include_bytes!("../tests/test-public-key.der").to_vec();
|
||||
|
||||
let private_key2 = {
|
||||
let pkcs8 = include_bytes!("../tests/test-private-key-2.pk8");
|
||||
Arc::new(RSAKeyPair::from_pkcs8(Input::from(&pkcs8[..])).unwrap())
|
||||
};
|
||||
let public_key2 = include_bytes!("../tests/test-public-key-2.der").to_vec();
|
||||
|
||||
let listener = TcpListener::bind(&"127.0.0.1:0".parse().unwrap(), &core.handle()).unwrap();
|
||||
let listener_addr = listener.local_addr().unwrap();
|
||||
|
||||
let server = listener.incoming()
|
||||
.into_future()
|
||||
.map_err(|(e, _)| e.into())
|
||||
.and_then(move |(connec, _)| {
|
||||
handshake(connec.unwrap().0, public_key1, private_key1)
|
||||
});
|
||||
|
||||
let client = TcpStream::connect(&listener_addr, &core.handle())
|
||||
.map_err(|e| e.into())
|
||||
.and_then(move |stream| handshake(stream, public_key2, private_key2));
|
||||
|
||||
core.run(server.join(client)).unwrap();
|
||||
}
|
||||
}
|
601
libp2p-secio/src/keys_proto.rs
Normal file
601
libp2p-secio/src/keys_proto.rs
Normal file
@ -0,0 +1,601 @@
|
||||
// This file is generated. Do not edit
|
||||
// @generated
|
||||
|
||||
// https://github.com/Manishearth/rust-clippy/issues/702
|
||||
#![allow(unknown_lints)]
|
||||
#![allow(clippy)]
|
||||
|
||||
#![cfg_attr(rustfmt, rustfmt_skip)]
|
||||
|
||||
#![allow(box_pointers)]
|
||||
#![allow(dead_code)]
|
||||
#![allow(missing_docs)]
|
||||
#![allow(non_camel_case_types)]
|
||||
#![allow(non_snake_case)]
|
||||
#![allow(non_upper_case_globals)]
|
||||
#![allow(trivial_casts)]
|
||||
#![allow(unsafe_code)]
|
||||
#![allow(unused_imports)]
|
||||
#![allow(unused_results)]
|
||||
|
||||
use protobuf::Message as Message_imported_for_functions;
|
||||
use protobuf::ProtobufEnum as ProtobufEnum_imported_for_functions;
|
||||
|
||||
#[derive(PartialEq,Clone,Default)]
|
||||
pub struct PublicKey {
|
||||
// message fields
|
||||
Type: ::std::option::Option<KeyType>,
|
||||
Data: ::protobuf::SingularField<::std::vec::Vec<u8>>,
|
||||
// special fields
|
||||
unknown_fields: ::protobuf::UnknownFields,
|
||||
cached_size: ::protobuf::CachedSize,
|
||||
}
|
||||
|
||||
// see codegen.rs for the explanation why impl Sync explicitly
|
||||
unsafe impl ::std::marker::Sync for PublicKey {}
|
||||
|
||||
impl PublicKey {
|
||||
pub fn new() -> PublicKey {
|
||||
::std::default::Default::default()
|
||||
}
|
||||
|
||||
pub fn default_instance() -> &'static PublicKey {
|
||||
static mut instance: ::protobuf::lazy::Lazy<PublicKey> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const PublicKey,
|
||||
};
|
||||
unsafe {
|
||||
instance.get(PublicKey::new)
|
||||
}
|
||||
}
|
||||
|
||||
// required .KeyType Type = 1;
|
||||
|
||||
pub fn clear_Type(&mut self) {
|
||||
self.Type = ::std::option::Option::None;
|
||||
}
|
||||
|
||||
pub fn has_Type(&self) -> bool {
|
||||
self.Type.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_Type(&mut self, v: KeyType) {
|
||||
self.Type = ::std::option::Option::Some(v);
|
||||
}
|
||||
|
||||
pub fn get_Type(&self) -> KeyType {
|
||||
self.Type.unwrap_or(KeyType::RSA)
|
||||
}
|
||||
|
||||
fn get_Type_for_reflect(&self) -> &::std::option::Option<KeyType> {
|
||||
&self.Type
|
||||
}
|
||||
|
||||
fn mut_Type_for_reflect(&mut self) -> &mut ::std::option::Option<KeyType> {
|
||||
&mut self.Type
|
||||
}
|
||||
|
||||
// required bytes Data = 2;
|
||||
|
||||
pub fn clear_Data(&mut self) {
|
||||
self.Data.clear();
|
||||
}
|
||||
|
||||
pub fn has_Data(&self) -> bool {
|
||||
self.Data.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_Data(&mut self, v: ::std::vec::Vec<u8>) {
|
||||
self.Data = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_Data(&mut self) -> &mut ::std::vec::Vec<u8> {
|
||||
if self.Data.is_none() {
|
||||
self.Data.set_default();
|
||||
}
|
||||
self.Data.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_Data(&mut self) -> ::std::vec::Vec<u8> {
|
||||
self.Data.take().unwrap_or_else(|| ::std::vec::Vec::new())
|
||||
}
|
||||
|
||||
pub fn get_Data(&self) -> &[u8] {
|
||||
match self.Data.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => &[],
|
||||
}
|
||||
}
|
||||
|
||||
fn get_Data_for_reflect(&self) -> &::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&self.Data
|
||||
}
|
||||
|
||||
fn mut_Data_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&mut self.Data
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Message for PublicKey {
|
||||
fn is_initialized(&self) -> bool {
|
||||
if self.Type.is_none() {
|
||||
return false;
|
||||
}
|
||||
if self.Data.is_none() {
|
||||
return false;
|
||||
}
|
||||
true
|
||||
}
|
||||
|
||||
fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
while !is.eof()? {
|
||||
let (field_number, wire_type) = is.read_tag_unpack()?;
|
||||
match field_number {
|
||||
1 => {
|
||||
if wire_type != ::protobuf::wire_format::WireTypeVarint {
|
||||
return ::std::result::Result::Err(::protobuf::rt::unexpected_wire_type(wire_type));
|
||||
}
|
||||
let tmp = is.read_enum()?;
|
||||
self.Type = ::std::option::Option::Some(tmp);
|
||||
},
|
||||
2 => {
|
||||
::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.Data)?;
|
||||
},
|
||||
_ => {
|
||||
::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?;
|
||||
},
|
||||
};
|
||||
}
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
// Compute sizes of nested messages
|
||||
#[allow(unused_variables)]
|
||||
fn compute_size(&self) -> u32 {
|
||||
let mut my_size = 0;
|
||||
if let Some(v) = self.Type {
|
||||
my_size += ::protobuf::rt::enum_size(1, v);
|
||||
}
|
||||
if let Some(ref v) = self.Data.as_ref() {
|
||||
my_size += ::protobuf::rt::bytes_size(2, &v);
|
||||
}
|
||||
my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields());
|
||||
self.cached_size.set(my_size);
|
||||
my_size
|
||||
}
|
||||
|
||||
fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
if let Some(v) = self.Type {
|
||||
os.write_enum(1, v.value())?;
|
||||
}
|
||||
if let Some(ref v) = self.Data.as_ref() {
|
||||
os.write_bytes(2, &v)?;
|
||||
}
|
||||
os.write_unknown_fields(self.get_unknown_fields())?;
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
fn get_cached_size(&self) -> u32 {
|
||||
self.cached_size.get()
|
||||
}
|
||||
|
||||
fn get_unknown_fields(&self) -> &::protobuf::UnknownFields {
|
||||
&self.unknown_fields
|
||||
}
|
||||
|
||||
fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields {
|
||||
&mut self.unknown_fields
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &::std::any::Any {
|
||||
self as &::std::any::Any
|
||||
}
|
||||
fn as_any_mut(&mut self) -> &mut ::std::any::Any {
|
||||
self as &mut ::std::any::Any
|
||||
}
|
||||
fn into_any(self: Box<Self>) -> ::std::boxed::Box<::std::any::Any> {
|
||||
self
|
||||
}
|
||||
|
||||
fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
::protobuf::MessageStatic::descriptor_static(None::<Self>)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::MessageStatic for PublicKey {
|
||||
fn new() -> PublicKey {
|
||||
PublicKey::new()
|
||||
}
|
||||
|
||||
fn descriptor_static(_: ::std::option::Option<PublicKey>) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const ::protobuf::reflect::MessageDescriptor,
|
||||
};
|
||||
unsafe {
|
||||
descriptor.get(|| {
|
||||
let mut fields = ::std::vec::Vec::new();
|
||||
fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeEnum<KeyType>>(
|
||||
"Type",
|
||||
PublicKey::get_Type_for_reflect,
|
||||
PublicKey::mut_Type_for_reflect,
|
||||
));
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>(
|
||||
"Data",
|
||||
PublicKey::get_Data_for_reflect,
|
||||
PublicKey::mut_Data_for_reflect,
|
||||
));
|
||||
::protobuf::reflect::MessageDescriptor::new::<PublicKey>(
|
||||
"PublicKey",
|
||||
fields,
|
||||
file_descriptor_proto()
|
||||
)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Clear for PublicKey {
|
||||
fn clear(&mut self) {
|
||||
self.clear_Type();
|
||||
self.clear_Data();
|
||||
self.unknown_fields.clear();
|
||||
}
|
||||
}
|
||||
|
||||
impl ::std::fmt::Debug for PublicKey {
|
||||
fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result {
|
||||
::protobuf::text_format::fmt(self, f)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::reflect::ProtobufValue for PublicKey {
|
||||
fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef {
|
||||
::protobuf::reflect::ProtobufValueRef::Message(self)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(PartialEq,Clone,Default)]
|
||||
pub struct PrivateKey {
|
||||
// message fields
|
||||
Type: ::std::option::Option<KeyType>,
|
||||
Data: ::protobuf::SingularField<::std::vec::Vec<u8>>,
|
||||
// special fields
|
||||
unknown_fields: ::protobuf::UnknownFields,
|
||||
cached_size: ::protobuf::CachedSize,
|
||||
}
|
||||
|
||||
// see codegen.rs for the explanation why impl Sync explicitly
|
||||
unsafe impl ::std::marker::Sync for PrivateKey {}
|
||||
|
||||
impl PrivateKey {
|
||||
pub fn new() -> PrivateKey {
|
||||
::std::default::Default::default()
|
||||
}
|
||||
|
||||
pub fn default_instance() -> &'static PrivateKey {
|
||||
static mut instance: ::protobuf::lazy::Lazy<PrivateKey> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const PrivateKey,
|
||||
};
|
||||
unsafe {
|
||||
instance.get(PrivateKey::new)
|
||||
}
|
||||
}
|
||||
|
||||
// required .KeyType Type = 1;
|
||||
|
||||
pub fn clear_Type(&mut self) {
|
||||
self.Type = ::std::option::Option::None;
|
||||
}
|
||||
|
||||
pub fn has_Type(&self) -> bool {
|
||||
self.Type.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_Type(&mut self, v: KeyType) {
|
||||
self.Type = ::std::option::Option::Some(v);
|
||||
}
|
||||
|
||||
pub fn get_Type(&self) -> KeyType {
|
||||
self.Type.unwrap_or(KeyType::RSA)
|
||||
}
|
||||
|
||||
fn get_Type_for_reflect(&self) -> &::std::option::Option<KeyType> {
|
||||
&self.Type
|
||||
}
|
||||
|
||||
fn mut_Type_for_reflect(&mut self) -> &mut ::std::option::Option<KeyType> {
|
||||
&mut self.Type
|
||||
}
|
||||
|
||||
// required bytes Data = 2;
|
||||
|
||||
pub fn clear_Data(&mut self) {
|
||||
self.Data.clear();
|
||||
}
|
||||
|
||||
pub fn has_Data(&self) -> bool {
|
||||
self.Data.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_Data(&mut self, v: ::std::vec::Vec<u8>) {
|
||||
self.Data = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_Data(&mut self) -> &mut ::std::vec::Vec<u8> {
|
||||
if self.Data.is_none() {
|
||||
self.Data.set_default();
|
||||
}
|
||||
self.Data.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_Data(&mut self) -> ::std::vec::Vec<u8> {
|
||||
self.Data.take().unwrap_or_else(|| ::std::vec::Vec::new())
|
||||
}
|
||||
|
||||
pub fn get_Data(&self) -> &[u8] {
|
||||
match self.Data.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => &[],
|
||||
}
|
||||
}
|
||||
|
||||
fn get_Data_for_reflect(&self) -> &::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&self.Data
|
||||
}
|
||||
|
||||
fn mut_Data_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&mut self.Data
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Message for PrivateKey {
|
||||
fn is_initialized(&self) -> bool {
|
||||
if self.Type.is_none() {
|
||||
return false;
|
||||
}
|
||||
if self.Data.is_none() {
|
||||
return false;
|
||||
}
|
||||
true
|
||||
}
|
||||
|
||||
fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
while !is.eof()? {
|
||||
let (field_number, wire_type) = is.read_tag_unpack()?;
|
||||
match field_number {
|
||||
1 => {
|
||||
if wire_type != ::protobuf::wire_format::WireTypeVarint {
|
||||
return ::std::result::Result::Err(::protobuf::rt::unexpected_wire_type(wire_type));
|
||||
}
|
||||
let tmp = is.read_enum()?;
|
||||
self.Type = ::std::option::Option::Some(tmp);
|
||||
},
|
||||
2 => {
|
||||
::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.Data)?;
|
||||
},
|
||||
_ => {
|
||||
::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?;
|
||||
},
|
||||
};
|
||||
}
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
// Compute sizes of nested messages
|
||||
#[allow(unused_variables)]
|
||||
fn compute_size(&self) -> u32 {
|
||||
let mut my_size = 0;
|
||||
if let Some(v) = self.Type {
|
||||
my_size += ::protobuf::rt::enum_size(1, v);
|
||||
}
|
||||
if let Some(ref v) = self.Data.as_ref() {
|
||||
my_size += ::protobuf::rt::bytes_size(2, &v);
|
||||
}
|
||||
my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields());
|
||||
self.cached_size.set(my_size);
|
||||
my_size
|
||||
}
|
||||
|
||||
fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
if let Some(v) = self.Type {
|
||||
os.write_enum(1, v.value())?;
|
||||
}
|
||||
if let Some(ref v) = self.Data.as_ref() {
|
||||
os.write_bytes(2, &v)?;
|
||||
}
|
||||
os.write_unknown_fields(self.get_unknown_fields())?;
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
fn get_cached_size(&self) -> u32 {
|
||||
self.cached_size.get()
|
||||
}
|
||||
|
||||
fn get_unknown_fields(&self) -> &::protobuf::UnknownFields {
|
||||
&self.unknown_fields
|
||||
}
|
||||
|
||||
fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields {
|
||||
&mut self.unknown_fields
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &::std::any::Any {
|
||||
self as &::std::any::Any
|
||||
}
|
||||
fn as_any_mut(&mut self) -> &mut ::std::any::Any {
|
||||
self as &mut ::std::any::Any
|
||||
}
|
||||
fn into_any(self: Box<Self>) -> ::std::boxed::Box<::std::any::Any> {
|
||||
self
|
||||
}
|
||||
|
||||
fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
::protobuf::MessageStatic::descriptor_static(None::<Self>)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::MessageStatic for PrivateKey {
|
||||
fn new() -> PrivateKey {
|
||||
PrivateKey::new()
|
||||
}
|
||||
|
||||
fn descriptor_static(_: ::std::option::Option<PrivateKey>) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const ::protobuf::reflect::MessageDescriptor,
|
||||
};
|
||||
unsafe {
|
||||
descriptor.get(|| {
|
||||
let mut fields = ::std::vec::Vec::new();
|
||||
fields.push(::protobuf::reflect::accessor::make_option_accessor::<_, ::protobuf::types::ProtobufTypeEnum<KeyType>>(
|
||||
"Type",
|
||||
PrivateKey::get_Type_for_reflect,
|
||||
PrivateKey::mut_Type_for_reflect,
|
||||
));
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>(
|
||||
"Data",
|
||||
PrivateKey::get_Data_for_reflect,
|
||||
PrivateKey::mut_Data_for_reflect,
|
||||
));
|
||||
::protobuf::reflect::MessageDescriptor::new::<PrivateKey>(
|
||||
"PrivateKey",
|
||||
fields,
|
||||
file_descriptor_proto()
|
||||
)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Clear for PrivateKey {
|
||||
fn clear(&mut self) {
|
||||
self.clear_Type();
|
||||
self.clear_Data();
|
||||
self.unknown_fields.clear();
|
||||
}
|
||||
}
|
||||
|
||||
impl ::std::fmt::Debug for PrivateKey {
|
||||
fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result {
|
||||
::protobuf::text_format::fmt(self, f)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::reflect::ProtobufValue for PrivateKey {
|
||||
fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef {
|
||||
::protobuf::reflect::ProtobufValueRef::Message(self)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone,PartialEq,Eq,Debug,Hash)]
|
||||
pub enum KeyType {
|
||||
RSA = 0,
|
||||
Ed25519 = 1,
|
||||
Secp256k1 = 2,
|
||||
}
|
||||
|
||||
impl ::protobuf::ProtobufEnum for KeyType {
|
||||
fn value(&self) -> i32 {
|
||||
*self as i32
|
||||
}
|
||||
|
||||
fn from_i32(value: i32) -> ::std::option::Option<KeyType> {
|
||||
match value {
|
||||
0 => ::std::option::Option::Some(KeyType::RSA),
|
||||
1 => ::std::option::Option::Some(KeyType::Ed25519),
|
||||
2 => ::std::option::Option::Some(KeyType::Secp256k1),
|
||||
_ => ::std::option::Option::None
|
||||
}
|
||||
}
|
||||
|
||||
fn values() -> &'static [Self] {
|
||||
static values: &'static [KeyType] = &[
|
||||
KeyType::RSA,
|
||||
KeyType::Ed25519,
|
||||
KeyType::Secp256k1,
|
||||
];
|
||||
values
|
||||
}
|
||||
|
||||
fn enum_descriptor_static(_: ::std::option::Option<KeyType>) -> &'static ::protobuf::reflect::EnumDescriptor {
|
||||
static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::EnumDescriptor> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const ::protobuf::reflect::EnumDescriptor,
|
||||
};
|
||||
unsafe {
|
||||
descriptor.get(|| {
|
||||
::protobuf::reflect::EnumDescriptor::new("KeyType", file_descriptor_proto())
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl ::std::marker::Copy for KeyType {
|
||||
}
|
||||
|
||||
impl ::protobuf::reflect::ProtobufValue for KeyType {
|
||||
fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef {
|
||||
::protobuf::reflect::ProtobufValueRef::Enum(self.descriptor())
|
||||
}
|
||||
}
|
||||
|
||||
static file_descriptor_proto_data: &'static [u8] = b"\
|
||||
\n\nkeys.proto\"=\n\tPublicKey\x12\x1c\n\x04Type\x18\x01\x20\x02(\x0e2\
|
||||
\x08.KeyTypeR\x04type\x12\x12\n\x04Data\x18\x02\x20\x02(\x0cR\x04data\">\
|
||||
\n\nPrivateKey\x12\x1c\n\x04Type\x18\x01\x20\x02(\x0e2\x08.KeyTypeR\x04t\
|
||||
ype\x12\x12\n\x04Data\x18\x02\x20\x02(\x0cR\x04data*.\n\x07KeyType\x12\
|
||||
\x07\n\x03RSA\x10\0\x12\x0b\n\x07Ed25519\x10\x01\x12\r\n\tSecp256k1\x10\
|
||||
\x02J\xdf\x03\n\x06\x12\x04\0\0\x0e\x01\n\n\n\x02\x05\0\x12\x04\0\0\x04\
|
||||
\x01\n\n\n\x03\x05\0\x01\x12\x03\0\x05\x0c\n\x0b\n\x04\x05\0\x02\0\x12\
|
||||
\x03\x01\x02\n\n\x0c\n\x05\x05\0\x02\0\x01\x12\x03\x01\x02\x05\n\x0c\n\
|
||||
\x05\x05\0\x02\0\x02\x12\x03\x01\x08\t\n\x0b\n\x04\x05\0\x02\x01\x12\x03\
|
||||
\x02\x02\x0e\n\x0c\n\x05\x05\0\x02\x01\x01\x12\x03\x02\x02\t\n\x0c\n\x05\
|
||||
\x05\0\x02\x01\x02\x12\x03\x02\x0c\r\n\x0b\n\x04\x05\0\x02\x02\x12\x03\
|
||||
\x03\x02\x10\n\x0c\n\x05\x05\0\x02\x02\x01\x12\x03\x03\x02\x0b\n\x0c\n\
|
||||
\x05\x05\0\x02\x02\x02\x12\x03\x03\x0e\x0f\n\n\n\x02\x04\0\x12\x04\x06\0\
|
||||
\t\x01\n\n\n\x03\x04\0\x01\x12\x03\x06\x08\x11\n\x0b\n\x04\x04\0\x02\0\
|
||||
\x12\x03\x07\x02\x1c\n\x0c\n\x05\x04\0\x02\0\x04\x12\x03\x07\x02\n\n\x0c\
|
||||
\n\x05\x04\0\x02\0\x06\x12\x03\x07\x0b\x12\n\x0c\n\x05\x04\0\x02\0\x01\
|
||||
\x12\x03\x07\x13\x17\n\x0c\n\x05\x04\0\x02\0\x03\x12\x03\x07\x1a\x1b\n\
|
||||
\x0b\n\x04\x04\0\x02\x01\x12\x03\x08\x02\x1a\n\x0c\n\x05\x04\0\x02\x01\
|
||||
\x04\x12\x03\x08\x02\n\n\x0c\n\x05\x04\0\x02\x01\x05\x12\x03\x08\x0b\x10\
|
||||
\n\x0c\n\x05\x04\0\x02\x01\x01\x12\x03\x08\x11\x15\n\x0c\n\x05\x04\0\x02\
|
||||
\x01\x03\x12\x03\x08\x18\x19\n\n\n\x02\x04\x01\x12\x04\x0b\0\x0e\x01\n\n\
|
||||
\n\x03\x04\x01\x01\x12\x03\x0b\x08\x12\n\x0b\n\x04\x04\x01\x02\0\x12\x03\
|
||||
\x0c\x02\x1c\n\x0c\n\x05\x04\x01\x02\0\x04\x12\x03\x0c\x02\n\n\x0c\n\x05\
|
||||
\x04\x01\x02\0\x06\x12\x03\x0c\x0b\x12\n\x0c\n\x05\x04\x01\x02\0\x01\x12\
|
||||
\x03\x0c\x13\x17\n\x0c\n\x05\x04\x01\x02\0\x03\x12\x03\x0c\x1a\x1b\n\x0b\
|
||||
\n\x04\x04\x01\x02\x01\x12\x03\r\x02\x1a\n\x0c\n\x05\x04\x01\x02\x01\x04\
|
||||
\x12\x03\r\x02\n\n\x0c\n\x05\x04\x01\x02\x01\x05\x12\x03\r\x0b\x10\n\x0c\
|
||||
\n\x05\x04\x01\x02\x01\x01\x12\x03\r\x11\x15\n\x0c\n\x05\x04\x01\x02\x01\
|
||||
\x03\x12\x03\r\x18\x19\
|
||||
";
|
||||
|
||||
static mut file_descriptor_proto_lazy: ::protobuf::lazy::Lazy<::protobuf::descriptor::FileDescriptorProto> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const ::protobuf::descriptor::FileDescriptorProto,
|
||||
};
|
||||
|
||||
fn parse_descriptor_proto() -> ::protobuf::descriptor::FileDescriptorProto {
|
||||
::protobuf::parse_from_bytes(file_descriptor_proto_data).unwrap()
|
||||
}
|
||||
|
||||
pub fn file_descriptor_proto() -> &'static ::protobuf::descriptor::FileDescriptorProto {
|
||||
unsafe {
|
||||
file_descriptor_proto_lazy.get(|| {
|
||||
parse_descriptor_proto()
|
||||
})
|
||||
}
|
||||
}
|
127
libp2p-secio/src/lib.rs
Normal file
127
libp2p-secio/src/lib.rs
Normal file
@ -0,0 +1,127 @@
|
||||
// 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.
|
||||
|
||||
//! # Implementation of the `secio` protocol.
|
||||
//!
|
||||
//! The `secio` protocol is a middleware that will encrypt and decrypt communications going
|
||||
//! through a socket (or anything that implements `AsyncRead + AsyncWrite`).
|
||||
//!
|
||||
//! You can add the `secio` layer over a socket by calling `SecioMiddleware::handshake()`. This
|
||||
//! method will perform a handshake with the host, and return a future that corresponds to the
|
||||
//! moment when the handshake succeeds or errored. On success, the future produces a
|
||||
//! `SecioMiddleware` that implements `Sink` and `Stream` and can be used to send packets of data.
|
||||
|
||||
extern crate bytes;
|
||||
extern crate crypto;
|
||||
extern crate futures;
|
||||
extern crate protobuf;
|
||||
extern crate rand;
|
||||
extern crate ring;
|
||||
extern crate tokio_core;
|
||||
extern crate tokio_io;
|
||||
extern crate untrusted;
|
||||
|
||||
pub use self::error::SecioError;
|
||||
|
||||
use bytes::BytesMut;
|
||||
use futures::{Future, Poll, StartSend, Sink, Stream};
|
||||
use ring::signature::RSAKeyPair;
|
||||
use std::io::Error as IoError;
|
||||
use std::sync::Arc;
|
||||
use tokio_io::{AsyncRead, AsyncWrite};
|
||||
|
||||
mod algo_support;
|
||||
mod codec;
|
||||
mod error;
|
||||
mod keys_proto;
|
||||
mod handshake;
|
||||
mod structs_proto;
|
||||
|
||||
/// Wraps around an object that implements `AsyncRead` and `AsyncWrite`.
|
||||
///
|
||||
/// Implements `Sink` and `Stream` whose items are frames of data. Each frame is encoded
|
||||
/// individually, so you are encouraged to group data in few frames if possible.
|
||||
pub struct SecioMiddleware<S> {
|
||||
inner: codec::FullCodec<S>,
|
||||
remote_pubkey_der: Vec<u8>,
|
||||
}
|
||||
|
||||
impl<S> SecioMiddleware<S>
|
||||
where S: AsyncRead + AsyncWrite
|
||||
{
|
||||
/// Attempts to perform a handshake on the given socket.
|
||||
///
|
||||
/// `local_public_key` and `local_private_key` must match. `local_public_key` must be in the
|
||||
/// DER format.
|
||||
///
|
||||
/// On success, produces a `SecioMiddleware` that can then be used to encode/decode
|
||||
/// communications.
|
||||
pub fn handshake<'a>(
|
||||
socket: S,
|
||||
local_public_key: Vec<u8>,
|
||||
local_private_key: Arc<RSAKeyPair>,
|
||||
) -> Box<Future<Item = SecioMiddleware<S>, Error = SecioError> + 'a>
|
||||
where S: 'a
|
||||
{
|
||||
let fut = handshake::handshake(socket, local_public_key, local_private_key)
|
||||
.map(|(inner, pubkey)| {
|
||||
SecioMiddleware {
|
||||
inner: inner,
|
||||
remote_pubkey_der: pubkey,
|
||||
}
|
||||
});
|
||||
Box::new(fut)
|
||||
}
|
||||
|
||||
/// Returns the public key of the remote in the `DER` format.
|
||||
#[inline]
|
||||
pub fn remote_public_key_der(&self) -> &[u8] {
|
||||
&self.remote_pubkey_der
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Sink for SecioMiddleware<S>
|
||||
where S: AsyncRead + AsyncWrite
|
||||
{
|
||||
type SinkItem = BytesMut;
|
||||
type SinkError = IoError;
|
||||
|
||||
#[inline]
|
||||
fn start_send(&mut self, item: Self::SinkItem) -> StartSend<Self::SinkItem, Self::SinkError> {
|
||||
self.inner.start_send(item)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn poll_complete(&mut self) -> Poll<(), Self::SinkError> {
|
||||
self.inner.poll_complete()
|
||||
}
|
||||
}
|
||||
|
||||
impl<S> Stream for SecioMiddleware<S>
|
||||
where S: AsyncRead + AsyncWrite
|
||||
{
|
||||
type Item = Vec<u8>;
|
||||
type Error = SecioError;
|
||||
|
||||
#[inline]
|
||||
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
|
||||
self.inner.poll()
|
||||
}
|
||||
}
|
747
libp2p-secio/src/structs_proto.rs
Normal file
747
libp2p-secio/src/structs_proto.rs
Normal file
@ -0,0 +1,747 @@
|
||||
// This file is generated. Do not edit
|
||||
// @generated
|
||||
|
||||
// https://github.com/Manishearth/rust-clippy/issues/702
|
||||
#![allow(unknown_lints)]
|
||||
#![allow(clippy)]
|
||||
|
||||
#![cfg_attr(rustfmt, rustfmt_skip)]
|
||||
|
||||
#![allow(box_pointers)]
|
||||
#![allow(dead_code)]
|
||||
#![allow(missing_docs)]
|
||||
#![allow(non_camel_case_types)]
|
||||
#![allow(non_snake_case)]
|
||||
#![allow(non_upper_case_globals)]
|
||||
#![allow(trivial_casts)]
|
||||
#![allow(unsafe_code)]
|
||||
#![allow(unused_imports)]
|
||||
#![allow(unused_results)]
|
||||
|
||||
use protobuf::Message as Message_imported_for_functions;
|
||||
use protobuf::ProtobufEnum as ProtobufEnum_imported_for_functions;
|
||||
|
||||
#[derive(PartialEq,Clone,Default)]
|
||||
pub struct Propose {
|
||||
// message fields
|
||||
rand: ::protobuf::SingularField<::std::vec::Vec<u8>>,
|
||||
pubkey: ::protobuf::SingularField<::std::vec::Vec<u8>>,
|
||||
exchanges: ::protobuf::SingularField<::std::string::String>,
|
||||
ciphers: ::protobuf::SingularField<::std::string::String>,
|
||||
hashes: ::protobuf::SingularField<::std::string::String>,
|
||||
// special fields
|
||||
unknown_fields: ::protobuf::UnknownFields,
|
||||
cached_size: ::protobuf::CachedSize,
|
||||
}
|
||||
|
||||
// see codegen.rs for the explanation why impl Sync explicitly
|
||||
unsafe impl ::std::marker::Sync for Propose {}
|
||||
|
||||
impl Propose {
|
||||
pub fn new() -> Propose {
|
||||
::std::default::Default::default()
|
||||
}
|
||||
|
||||
pub fn default_instance() -> &'static Propose {
|
||||
static mut instance: ::protobuf::lazy::Lazy<Propose> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const Propose,
|
||||
};
|
||||
unsafe {
|
||||
instance.get(Propose::new)
|
||||
}
|
||||
}
|
||||
|
||||
// optional bytes rand = 1;
|
||||
|
||||
pub fn clear_rand(&mut self) {
|
||||
self.rand.clear();
|
||||
}
|
||||
|
||||
pub fn has_rand(&self) -> bool {
|
||||
self.rand.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_rand(&mut self, v: ::std::vec::Vec<u8>) {
|
||||
self.rand = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_rand(&mut self) -> &mut ::std::vec::Vec<u8> {
|
||||
if self.rand.is_none() {
|
||||
self.rand.set_default();
|
||||
}
|
||||
self.rand.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_rand(&mut self) -> ::std::vec::Vec<u8> {
|
||||
self.rand.take().unwrap_or_else(|| ::std::vec::Vec::new())
|
||||
}
|
||||
|
||||
pub fn get_rand(&self) -> &[u8] {
|
||||
match self.rand.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => &[],
|
||||
}
|
||||
}
|
||||
|
||||
fn get_rand_for_reflect(&self) -> &::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&self.rand
|
||||
}
|
||||
|
||||
fn mut_rand_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&mut self.rand
|
||||
}
|
||||
|
||||
// optional bytes pubkey = 2;
|
||||
|
||||
pub fn clear_pubkey(&mut self) {
|
||||
self.pubkey.clear();
|
||||
}
|
||||
|
||||
pub fn has_pubkey(&self) -> bool {
|
||||
self.pubkey.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_pubkey(&mut self, v: ::std::vec::Vec<u8>) {
|
||||
self.pubkey = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_pubkey(&mut self) -> &mut ::std::vec::Vec<u8> {
|
||||
if self.pubkey.is_none() {
|
||||
self.pubkey.set_default();
|
||||
}
|
||||
self.pubkey.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_pubkey(&mut self) -> ::std::vec::Vec<u8> {
|
||||
self.pubkey.take().unwrap_or_else(|| ::std::vec::Vec::new())
|
||||
}
|
||||
|
||||
pub fn get_pubkey(&self) -> &[u8] {
|
||||
match self.pubkey.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => &[],
|
||||
}
|
||||
}
|
||||
|
||||
fn get_pubkey_for_reflect(&self) -> &::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&self.pubkey
|
||||
}
|
||||
|
||||
fn mut_pubkey_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&mut self.pubkey
|
||||
}
|
||||
|
||||
// optional string exchanges = 3;
|
||||
|
||||
pub fn clear_exchanges(&mut self) {
|
||||
self.exchanges.clear();
|
||||
}
|
||||
|
||||
pub fn has_exchanges(&self) -> bool {
|
||||
self.exchanges.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_exchanges(&mut self, v: ::std::string::String) {
|
||||
self.exchanges = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_exchanges(&mut self) -> &mut ::std::string::String {
|
||||
if self.exchanges.is_none() {
|
||||
self.exchanges.set_default();
|
||||
}
|
||||
self.exchanges.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_exchanges(&mut self) -> ::std::string::String {
|
||||
self.exchanges.take().unwrap_or_else(|| ::std::string::String::new())
|
||||
}
|
||||
|
||||
pub fn get_exchanges(&self) -> &str {
|
||||
match self.exchanges.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => "",
|
||||
}
|
||||
}
|
||||
|
||||
fn get_exchanges_for_reflect(&self) -> &::protobuf::SingularField<::std::string::String> {
|
||||
&self.exchanges
|
||||
}
|
||||
|
||||
fn mut_exchanges_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::string::String> {
|
||||
&mut self.exchanges
|
||||
}
|
||||
|
||||
// optional string ciphers = 4;
|
||||
|
||||
pub fn clear_ciphers(&mut self) {
|
||||
self.ciphers.clear();
|
||||
}
|
||||
|
||||
pub fn has_ciphers(&self) -> bool {
|
||||
self.ciphers.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_ciphers(&mut self, v: ::std::string::String) {
|
||||
self.ciphers = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_ciphers(&mut self) -> &mut ::std::string::String {
|
||||
if self.ciphers.is_none() {
|
||||
self.ciphers.set_default();
|
||||
}
|
||||
self.ciphers.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_ciphers(&mut self) -> ::std::string::String {
|
||||
self.ciphers.take().unwrap_or_else(|| ::std::string::String::new())
|
||||
}
|
||||
|
||||
pub fn get_ciphers(&self) -> &str {
|
||||
match self.ciphers.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => "",
|
||||
}
|
||||
}
|
||||
|
||||
fn get_ciphers_for_reflect(&self) -> &::protobuf::SingularField<::std::string::String> {
|
||||
&self.ciphers
|
||||
}
|
||||
|
||||
fn mut_ciphers_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::string::String> {
|
||||
&mut self.ciphers
|
||||
}
|
||||
|
||||
// optional string hashes = 5;
|
||||
|
||||
pub fn clear_hashes(&mut self) {
|
||||
self.hashes.clear();
|
||||
}
|
||||
|
||||
pub fn has_hashes(&self) -> bool {
|
||||
self.hashes.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_hashes(&mut self, v: ::std::string::String) {
|
||||
self.hashes = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_hashes(&mut self) -> &mut ::std::string::String {
|
||||
if self.hashes.is_none() {
|
||||
self.hashes.set_default();
|
||||
}
|
||||
self.hashes.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_hashes(&mut self) -> ::std::string::String {
|
||||
self.hashes.take().unwrap_or_else(|| ::std::string::String::new())
|
||||
}
|
||||
|
||||
pub fn get_hashes(&self) -> &str {
|
||||
match self.hashes.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => "",
|
||||
}
|
||||
}
|
||||
|
||||
fn get_hashes_for_reflect(&self) -> &::protobuf::SingularField<::std::string::String> {
|
||||
&self.hashes
|
||||
}
|
||||
|
||||
fn mut_hashes_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::string::String> {
|
||||
&mut self.hashes
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Message for Propose {
|
||||
fn is_initialized(&self) -> bool {
|
||||
true
|
||||
}
|
||||
|
||||
fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
while !is.eof()? {
|
||||
let (field_number, wire_type) = is.read_tag_unpack()?;
|
||||
match field_number {
|
||||
1 => {
|
||||
::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.rand)?;
|
||||
},
|
||||
2 => {
|
||||
::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.pubkey)?;
|
||||
},
|
||||
3 => {
|
||||
::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.exchanges)?;
|
||||
},
|
||||
4 => {
|
||||
::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.ciphers)?;
|
||||
},
|
||||
5 => {
|
||||
::protobuf::rt::read_singular_string_into(wire_type, is, &mut self.hashes)?;
|
||||
},
|
||||
_ => {
|
||||
::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?;
|
||||
},
|
||||
};
|
||||
}
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
// Compute sizes of nested messages
|
||||
#[allow(unused_variables)]
|
||||
fn compute_size(&self) -> u32 {
|
||||
let mut my_size = 0;
|
||||
if let Some(ref v) = self.rand.as_ref() {
|
||||
my_size += ::protobuf::rt::bytes_size(1, &v);
|
||||
}
|
||||
if let Some(ref v) = self.pubkey.as_ref() {
|
||||
my_size += ::protobuf::rt::bytes_size(2, &v);
|
||||
}
|
||||
if let Some(ref v) = self.exchanges.as_ref() {
|
||||
my_size += ::protobuf::rt::string_size(3, &v);
|
||||
}
|
||||
if let Some(ref v) = self.ciphers.as_ref() {
|
||||
my_size += ::protobuf::rt::string_size(4, &v);
|
||||
}
|
||||
if let Some(ref v) = self.hashes.as_ref() {
|
||||
my_size += ::protobuf::rt::string_size(5, &v);
|
||||
}
|
||||
my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields());
|
||||
self.cached_size.set(my_size);
|
||||
my_size
|
||||
}
|
||||
|
||||
fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
if let Some(ref v) = self.rand.as_ref() {
|
||||
os.write_bytes(1, &v)?;
|
||||
}
|
||||
if let Some(ref v) = self.pubkey.as_ref() {
|
||||
os.write_bytes(2, &v)?;
|
||||
}
|
||||
if let Some(ref v) = self.exchanges.as_ref() {
|
||||
os.write_string(3, &v)?;
|
||||
}
|
||||
if let Some(ref v) = self.ciphers.as_ref() {
|
||||
os.write_string(4, &v)?;
|
||||
}
|
||||
if let Some(ref v) = self.hashes.as_ref() {
|
||||
os.write_string(5, &v)?;
|
||||
}
|
||||
os.write_unknown_fields(self.get_unknown_fields())?;
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
fn get_cached_size(&self) -> u32 {
|
||||
self.cached_size.get()
|
||||
}
|
||||
|
||||
fn get_unknown_fields(&self) -> &::protobuf::UnknownFields {
|
||||
&self.unknown_fields
|
||||
}
|
||||
|
||||
fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields {
|
||||
&mut self.unknown_fields
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &::std::any::Any {
|
||||
self as &::std::any::Any
|
||||
}
|
||||
fn as_any_mut(&mut self) -> &mut ::std::any::Any {
|
||||
self as &mut ::std::any::Any
|
||||
}
|
||||
fn into_any(self: Box<Self>) -> ::std::boxed::Box<::std::any::Any> {
|
||||
self
|
||||
}
|
||||
|
||||
fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
::protobuf::MessageStatic::descriptor_static(None::<Self>)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::MessageStatic for Propose {
|
||||
fn new() -> Propose {
|
||||
Propose::new()
|
||||
}
|
||||
|
||||
fn descriptor_static(_: ::std::option::Option<Propose>) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const ::protobuf::reflect::MessageDescriptor,
|
||||
};
|
||||
unsafe {
|
||||
descriptor.get(|| {
|
||||
let mut fields = ::std::vec::Vec::new();
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>(
|
||||
"rand",
|
||||
Propose::get_rand_for_reflect,
|
||||
Propose::mut_rand_for_reflect,
|
||||
));
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>(
|
||||
"pubkey",
|
||||
Propose::get_pubkey_for_reflect,
|
||||
Propose::mut_pubkey_for_reflect,
|
||||
));
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>(
|
||||
"exchanges",
|
||||
Propose::get_exchanges_for_reflect,
|
||||
Propose::mut_exchanges_for_reflect,
|
||||
));
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>(
|
||||
"ciphers",
|
||||
Propose::get_ciphers_for_reflect,
|
||||
Propose::mut_ciphers_for_reflect,
|
||||
));
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeString>(
|
||||
"hashes",
|
||||
Propose::get_hashes_for_reflect,
|
||||
Propose::mut_hashes_for_reflect,
|
||||
));
|
||||
::protobuf::reflect::MessageDescriptor::new::<Propose>(
|
||||
"Propose",
|
||||
fields,
|
||||
file_descriptor_proto()
|
||||
)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Clear for Propose {
|
||||
fn clear(&mut self) {
|
||||
self.clear_rand();
|
||||
self.clear_pubkey();
|
||||
self.clear_exchanges();
|
||||
self.clear_ciphers();
|
||||
self.clear_hashes();
|
||||
self.unknown_fields.clear();
|
||||
}
|
||||
}
|
||||
|
||||
impl ::std::fmt::Debug for Propose {
|
||||
fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result {
|
||||
::protobuf::text_format::fmt(self, f)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::reflect::ProtobufValue for Propose {
|
||||
fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef {
|
||||
::protobuf::reflect::ProtobufValueRef::Message(self)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(PartialEq,Clone,Default)]
|
||||
pub struct Exchange {
|
||||
// message fields
|
||||
epubkey: ::protobuf::SingularField<::std::vec::Vec<u8>>,
|
||||
signature: ::protobuf::SingularField<::std::vec::Vec<u8>>,
|
||||
// special fields
|
||||
unknown_fields: ::protobuf::UnknownFields,
|
||||
cached_size: ::protobuf::CachedSize,
|
||||
}
|
||||
|
||||
// see codegen.rs for the explanation why impl Sync explicitly
|
||||
unsafe impl ::std::marker::Sync for Exchange {}
|
||||
|
||||
impl Exchange {
|
||||
pub fn new() -> Exchange {
|
||||
::std::default::Default::default()
|
||||
}
|
||||
|
||||
pub fn default_instance() -> &'static Exchange {
|
||||
static mut instance: ::protobuf::lazy::Lazy<Exchange> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const Exchange,
|
||||
};
|
||||
unsafe {
|
||||
instance.get(Exchange::new)
|
||||
}
|
||||
}
|
||||
|
||||
// optional bytes epubkey = 1;
|
||||
|
||||
pub fn clear_epubkey(&mut self) {
|
||||
self.epubkey.clear();
|
||||
}
|
||||
|
||||
pub fn has_epubkey(&self) -> bool {
|
||||
self.epubkey.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_epubkey(&mut self, v: ::std::vec::Vec<u8>) {
|
||||
self.epubkey = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_epubkey(&mut self) -> &mut ::std::vec::Vec<u8> {
|
||||
if self.epubkey.is_none() {
|
||||
self.epubkey.set_default();
|
||||
}
|
||||
self.epubkey.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_epubkey(&mut self) -> ::std::vec::Vec<u8> {
|
||||
self.epubkey.take().unwrap_or_else(|| ::std::vec::Vec::new())
|
||||
}
|
||||
|
||||
pub fn get_epubkey(&self) -> &[u8] {
|
||||
match self.epubkey.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => &[],
|
||||
}
|
||||
}
|
||||
|
||||
fn get_epubkey_for_reflect(&self) -> &::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&self.epubkey
|
||||
}
|
||||
|
||||
fn mut_epubkey_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&mut self.epubkey
|
||||
}
|
||||
|
||||
// optional bytes signature = 2;
|
||||
|
||||
pub fn clear_signature(&mut self) {
|
||||
self.signature.clear();
|
||||
}
|
||||
|
||||
pub fn has_signature(&self) -> bool {
|
||||
self.signature.is_some()
|
||||
}
|
||||
|
||||
// Param is passed by value, moved
|
||||
pub fn set_signature(&mut self, v: ::std::vec::Vec<u8>) {
|
||||
self.signature = ::protobuf::SingularField::some(v);
|
||||
}
|
||||
|
||||
// Mutable pointer to the field.
|
||||
// If field is not initialized, it is initialized with default value first.
|
||||
pub fn mut_signature(&mut self) -> &mut ::std::vec::Vec<u8> {
|
||||
if self.signature.is_none() {
|
||||
self.signature.set_default();
|
||||
}
|
||||
self.signature.as_mut().unwrap()
|
||||
}
|
||||
|
||||
// Take field
|
||||
pub fn take_signature(&mut self) -> ::std::vec::Vec<u8> {
|
||||
self.signature.take().unwrap_or_else(|| ::std::vec::Vec::new())
|
||||
}
|
||||
|
||||
pub fn get_signature(&self) -> &[u8] {
|
||||
match self.signature.as_ref() {
|
||||
Some(v) => &v,
|
||||
None => &[],
|
||||
}
|
||||
}
|
||||
|
||||
fn get_signature_for_reflect(&self) -> &::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&self.signature
|
||||
}
|
||||
|
||||
fn mut_signature_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec<u8>> {
|
||||
&mut self.signature
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Message for Exchange {
|
||||
fn is_initialized(&self) -> bool {
|
||||
true
|
||||
}
|
||||
|
||||
fn merge_from(&mut self, is: &mut ::protobuf::CodedInputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
while !is.eof()? {
|
||||
let (field_number, wire_type) = is.read_tag_unpack()?;
|
||||
match field_number {
|
||||
1 => {
|
||||
::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.epubkey)?;
|
||||
},
|
||||
2 => {
|
||||
::protobuf::rt::read_singular_bytes_into(wire_type, is, &mut self.signature)?;
|
||||
},
|
||||
_ => {
|
||||
::protobuf::rt::read_unknown_or_skip_group(field_number, wire_type, is, self.mut_unknown_fields())?;
|
||||
},
|
||||
};
|
||||
}
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
// Compute sizes of nested messages
|
||||
#[allow(unused_variables)]
|
||||
fn compute_size(&self) -> u32 {
|
||||
let mut my_size = 0;
|
||||
if let Some(ref v) = self.epubkey.as_ref() {
|
||||
my_size += ::protobuf::rt::bytes_size(1, &v);
|
||||
}
|
||||
if let Some(ref v) = self.signature.as_ref() {
|
||||
my_size += ::protobuf::rt::bytes_size(2, &v);
|
||||
}
|
||||
my_size += ::protobuf::rt::unknown_fields_size(self.get_unknown_fields());
|
||||
self.cached_size.set(my_size);
|
||||
my_size
|
||||
}
|
||||
|
||||
fn write_to_with_cached_sizes(&self, os: &mut ::protobuf::CodedOutputStream) -> ::protobuf::ProtobufResult<()> {
|
||||
if let Some(ref v) = self.epubkey.as_ref() {
|
||||
os.write_bytes(1, &v)?;
|
||||
}
|
||||
if let Some(ref v) = self.signature.as_ref() {
|
||||
os.write_bytes(2, &v)?;
|
||||
}
|
||||
os.write_unknown_fields(self.get_unknown_fields())?;
|
||||
::std::result::Result::Ok(())
|
||||
}
|
||||
|
||||
fn get_cached_size(&self) -> u32 {
|
||||
self.cached_size.get()
|
||||
}
|
||||
|
||||
fn get_unknown_fields(&self) -> &::protobuf::UnknownFields {
|
||||
&self.unknown_fields
|
||||
}
|
||||
|
||||
fn mut_unknown_fields(&mut self) -> &mut ::protobuf::UnknownFields {
|
||||
&mut self.unknown_fields
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &::std::any::Any {
|
||||
self as &::std::any::Any
|
||||
}
|
||||
fn as_any_mut(&mut self) -> &mut ::std::any::Any {
|
||||
self as &mut ::std::any::Any
|
||||
}
|
||||
fn into_any(self: Box<Self>) -> ::std::boxed::Box<::std::any::Any> {
|
||||
self
|
||||
}
|
||||
|
||||
fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
::protobuf::MessageStatic::descriptor_static(None::<Self>)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::MessageStatic for Exchange {
|
||||
fn new() -> Exchange {
|
||||
Exchange::new()
|
||||
}
|
||||
|
||||
fn descriptor_static(_: ::std::option::Option<Exchange>) -> &'static ::protobuf::reflect::MessageDescriptor {
|
||||
static mut descriptor: ::protobuf::lazy::Lazy<::protobuf::reflect::MessageDescriptor> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const ::protobuf::reflect::MessageDescriptor,
|
||||
};
|
||||
unsafe {
|
||||
descriptor.get(|| {
|
||||
let mut fields = ::std::vec::Vec::new();
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>(
|
||||
"epubkey",
|
||||
Exchange::get_epubkey_for_reflect,
|
||||
Exchange::mut_epubkey_for_reflect,
|
||||
));
|
||||
fields.push(::protobuf::reflect::accessor::make_singular_field_accessor::<_, ::protobuf::types::ProtobufTypeBytes>(
|
||||
"signature",
|
||||
Exchange::get_signature_for_reflect,
|
||||
Exchange::mut_signature_for_reflect,
|
||||
));
|
||||
::protobuf::reflect::MessageDescriptor::new::<Exchange>(
|
||||
"Exchange",
|
||||
fields,
|
||||
file_descriptor_proto()
|
||||
)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::Clear for Exchange {
|
||||
fn clear(&mut self) {
|
||||
self.clear_epubkey();
|
||||
self.clear_signature();
|
||||
self.unknown_fields.clear();
|
||||
}
|
||||
}
|
||||
|
||||
impl ::std::fmt::Debug for Exchange {
|
||||
fn fmt(&self, f: &mut ::std::fmt::Formatter) -> ::std::fmt::Result {
|
||||
::protobuf::text_format::fmt(self, f)
|
||||
}
|
||||
}
|
||||
|
||||
impl ::protobuf::reflect::ProtobufValue for Exchange {
|
||||
fn as_ref(&self) -> ::protobuf::reflect::ProtobufValueRef {
|
||||
::protobuf::reflect::ProtobufValueRef::Message(self)
|
||||
}
|
||||
}
|
||||
|
||||
static file_descriptor_proto_data: &'static [u8] = b"\
|
||||
\n\rstructs.proto\x12\x08spipe.pb\"\x85\x01\n\x07Propose\x12\x12\n\x04ra\
|
||||
nd\x18\x01\x20\x01(\x0cR\x04rand\x12\x16\n\x06pubkey\x18\x02\x20\x01(\
|
||||
\x0cR\x06pubkey\x12\x1c\n\texchanges\x18\x03\x20\x01(\tR\texchanges\x12\
|
||||
\x18\n\x07ciphers\x18\x04\x20\x01(\tR\x07ciphers\x12\x16\n\x06hashes\x18\
|
||||
\x05\x20\x01(\tR\x06hashes\"B\n\x08Exchange\x12\x18\n\x07epubkey\x18\x01\
|
||||
\x20\x01(\x0cR\x07epubkey\x12\x1c\n\tsignature\x18\x02\x20\x01(\x0cR\tsi\
|
||||
gnatureJ\xa5\x04\n\x06\x12\x04\0\0\r\x01\n\x08\n\x01\x02\x12\x03\0\x08\
|
||||
\x10\n\n\n\x02\x04\0\x12\x04\x02\0\x08\x01\n\n\n\x03\x04\0\x01\x12\x03\
|
||||
\x02\x08\x0f\n\x0b\n\x04\x04\0\x02\0\x12\x03\x03\x08\x20\n\x0c\n\x05\x04\
|
||||
\0\x02\0\x04\x12\x03\x03\x08\x10\n\x0c\n\x05\x04\0\x02\0\x05\x12\x03\x03\
|
||||
\x11\x16\n\x0c\n\x05\x04\0\x02\0\x01\x12\x03\x03\x17\x1b\n\x0c\n\x05\x04\
|
||||
\0\x02\0\x03\x12\x03\x03\x1e\x1f\n\x0b\n\x04\x04\0\x02\x01\x12\x03\x04\
|
||||
\x08\"\n\x0c\n\x05\x04\0\x02\x01\x04\x12\x03\x04\x08\x10\n\x0c\n\x05\x04\
|
||||
\0\x02\x01\x05\x12\x03\x04\x11\x16\n\x0c\n\x05\x04\0\x02\x01\x01\x12\x03\
|
||||
\x04\x17\x1d\n\x0c\n\x05\x04\0\x02\x01\x03\x12\x03\x04\x20!\n\x0b\n\x04\
|
||||
\x04\0\x02\x02\x12\x03\x05\x08&\n\x0c\n\x05\x04\0\x02\x02\x04\x12\x03\
|
||||
\x05\x08\x10\n\x0c\n\x05\x04\0\x02\x02\x05\x12\x03\x05\x11\x17\n\x0c\n\
|
||||
\x05\x04\0\x02\x02\x01\x12\x03\x05\x18!\n\x0c\n\x05\x04\0\x02\x02\x03\
|
||||
\x12\x03\x05$%\n\x0b\n\x04\x04\0\x02\x03\x12\x03\x06\x08$\n\x0c\n\x05\
|
||||
\x04\0\x02\x03\x04\x12\x03\x06\x08\x10\n\x0c\n\x05\x04\0\x02\x03\x05\x12\
|
||||
\x03\x06\x11\x17\n\x0c\n\x05\x04\0\x02\x03\x01\x12\x03\x06\x18\x1f\n\x0c\
|
||||
\n\x05\x04\0\x02\x03\x03\x12\x03\x06\"#\n\x0b\n\x04\x04\0\x02\x04\x12\
|
||||
\x03\x07\x08#\n\x0c\n\x05\x04\0\x02\x04\x04\x12\x03\x07\x08\x10\n\x0c\n\
|
||||
\x05\x04\0\x02\x04\x05\x12\x03\x07\x11\x17\n\x0c\n\x05\x04\0\x02\x04\x01\
|
||||
\x12\x03\x07\x18\x1e\n\x0c\n\x05\x04\0\x02\x04\x03\x12\x03\x07!\"\n\n\n\
|
||||
\x02\x04\x01\x12\x04\n\0\r\x01\n\n\n\x03\x04\x01\x01\x12\x03\n\x08\x10\n\
|
||||
\x0b\n\x04\x04\x01\x02\0\x12\x03\x0b\x08#\n\x0c\n\x05\x04\x01\x02\0\x04\
|
||||
\x12\x03\x0b\x08\x10\n\x0c\n\x05\x04\x01\x02\0\x05\x12\x03\x0b\x11\x16\n\
|
||||
\x0c\n\x05\x04\x01\x02\0\x01\x12\x03\x0b\x17\x1e\n\x0c\n\x05\x04\x01\x02\
|
||||
\0\x03\x12\x03\x0b!\"\n\x0b\n\x04\x04\x01\x02\x01\x12\x03\x0c\x08%\n\x0c\
|
||||
\n\x05\x04\x01\x02\x01\x04\x12\x03\x0c\x08\x10\n\x0c\n\x05\x04\x01\x02\
|
||||
\x01\x05\x12\x03\x0c\x11\x16\n\x0c\n\x05\x04\x01\x02\x01\x01\x12\x03\x0c\
|
||||
\x17\x20\n\x0c\n\x05\x04\x01\x02\x01\x03\x12\x03\x0c#$\
|
||||
";
|
||||
|
||||
static mut file_descriptor_proto_lazy: ::protobuf::lazy::Lazy<::protobuf::descriptor::FileDescriptorProto> = ::protobuf::lazy::Lazy {
|
||||
lock: ::protobuf::lazy::ONCE_INIT,
|
||||
ptr: 0 as *const ::protobuf::descriptor::FileDescriptorProto,
|
||||
};
|
||||
|
||||
fn parse_descriptor_proto() -> ::protobuf::descriptor::FileDescriptorProto {
|
||||
::protobuf::parse_from_bytes(file_descriptor_proto_data).unwrap()
|
||||
}
|
||||
|
||||
pub fn file_descriptor_proto() -> &'static ::protobuf::descriptor::FileDescriptorProto {
|
||||
unsafe {
|
||||
file_descriptor_proto_lazy.get(|| {
|
||||
parse_descriptor_proto()
|
||||
})
|
||||
}
|
||||
}
|
14
libp2p-secio/structs.proto
Normal file
14
libp2p-secio/structs.proto
Normal file
@ -0,0 +1,14 @@
|
||||
package spipe.pb;
|
||||
|
||||
message Propose {
|
||||
optional bytes rand = 1;
|
||||
optional bytes pubkey = 2;
|
||||
optional string exchanges = 3;
|
||||
optional string ciphers = 4;
|
||||
optional string hashes = 5;
|
||||
}
|
||||
|
||||
message Exchange {
|
||||
optional bytes epubkey = 1;
|
||||
optional bytes signature = 2;
|
||||
}
|
BIN
libp2p-secio/tests/test-private-key-2.pk8
Normal file
BIN
libp2p-secio/tests/test-private-key-2.pk8
Normal file
Binary file not shown.
BIN
libp2p-secio/tests/test-private-key.pk8
Normal file
BIN
libp2p-secio/tests/test-private-key.pk8
Normal file
Binary file not shown.
BIN
libp2p-secio/tests/test-public-key-2.der
Normal file
BIN
libp2p-secio/tests/test-public-key-2.der
Normal file
Binary file not shown.
BIN
libp2p-secio/tests/test-public-key.der
Normal file
BIN
libp2p-secio/tests/test-public-key.der
Normal file
Binary file not shown.
Reference in New Issue
Block a user