From e7bc9d4d7d2b1e23f1e47e9afd89252d9597ad64 Mon Sep 17 00:00:00 2001 From: tomaka Date: Mon, 30 Oct 2017 10:22:38 +0100 Subject: [PATCH] Implement secio --- Cargo.toml | 5 + libp2p-secio/Cargo.toml | 15 + libp2p-secio/keys.proto | 15 + libp2p-secio/regen_structs_proto.sh | 14 + libp2p-secio/src/algo_support.rs | 95 +++ libp2p-secio/src/codec/decode.rs | 115 ++++ libp2p-secio/src/codec/encode.rs | 95 +++ libp2p-secio/src/codec/mod.rs | 166 +++++ libp2p-secio/src/error.rs | 144 +++++ libp2p-secio/src/handshake.rs | 488 ++++++++++++++ libp2p-secio/src/keys_proto.rs | 601 +++++++++++++++++ libp2p-secio/src/lib.rs | 127 ++++ libp2p-secio/src/structs_proto.rs | 747 ++++++++++++++++++++++ libp2p-secio/structs.proto | 14 + libp2p-secio/tests/test-private-key-2.pk8 | Bin 0 -> 1217 bytes libp2p-secio/tests/test-private-key.pk8 | Bin 0 -> 1219 bytes libp2p-secio/tests/test-public-key-2.der | Bin 0 -> 294 bytes libp2p-secio/tests/test-public-key.der | Bin 0 -> 294 bytes 18 files changed, 2641 insertions(+) create mode 100644 libp2p-secio/Cargo.toml create mode 100644 libp2p-secio/keys.proto create mode 100755 libp2p-secio/regen_structs_proto.sh create mode 100644 libp2p-secio/src/algo_support.rs create mode 100644 libp2p-secio/src/codec/decode.rs create mode 100644 libp2p-secio/src/codec/encode.rs create mode 100644 libp2p-secio/src/codec/mod.rs create mode 100644 libp2p-secio/src/error.rs create mode 100644 libp2p-secio/src/handshake.rs create mode 100644 libp2p-secio/src/keys_proto.rs create mode 100644 libp2p-secio/src/lib.rs create mode 100644 libp2p-secio/src/structs_proto.rs create mode 100644 libp2p-secio/structs.proto create mode 100644 libp2p-secio/tests/test-private-key-2.pk8 create mode 100644 libp2p-secio/tests/test-private-key.pk8 create mode 100644 libp2p-secio/tests/test-public-key-2.der create mode 100644 libp2p-secio/tests/test-public-key.der diff --git a/Cargo.toml b/Cargo.toml index ddec2615..3e6ebd95 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -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" } diff --git a/libp2p-secio/Cargo.toml b/libp2p-secio/Cargo.toml new file mode 100644 index 00000000..223ad513 --- /dev/null +++ b/libp2p-secio/Cargo.toml @@ -0,0 +1,15 @@ +[package] +name = "libp2p-secio" +version = "0.1.0" +authors = ["Parity Technologies "] + +[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" diff --git a/libp2p-secio/keys.proto b/libp2p-secio/keys.proto new file mode 100644 index 00000000..786c7a74 --- /dev/null +++ b/libp2p-secio/keys.proto @@ -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; +} \ No newline at end of file diff --git a/libp2p-secio/regen_structs_proto.sh b/libp2p-secio/regen_structs_proto.sh new file mode 100755 index 00000000..3ee8bb71 --- /dev/null +++ b/libp2p-secio/regen_structs_proto.sh @@ -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 diff --git a/libp2p-secio/src/algo_support.rs b/libp2p-secio/src/algo_support.rs new file mode 100644 index 00000000..454060cc --- /dev/null +++ b/libp2p-secio/src/algo_support.rs @@ -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, +); diff --git a/libp2p-secio/src/codec/decode.rs b/libp2p-secio/src/codec/decode.rs new file mode 100644 index 00000000..13d2f2c8 --- /dev/null +++ b/libp2p-secio/src/codec/decode.rs @@ -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`. 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 { + cipher_state: Box, + hmac_key: hmac::VerificationKey, + raw_stream: S, +} + +impl DecoderMiddleware { + #[inline] + pub fn new( + raw_stream: S, + cipher: Box, + hmac_key: hmac::VerificationKey, + ) -> DecoderMiddleware { + DecoderMiddleware { + cipher_state: cipher, + hmac_key: hmac_key, + raw_stream: raw_stream, + } + } +} + +impl Stream for DecoderMiddleware + where S: Stream, + S::Error: Into +{ + type Item = Vec; + type Error = SecioError; + + #[inline] + fn poll(&mut self) -> Poll, 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 Sink for DecoderMiddleware + where S: Sink +{ + type SinkItem = S::SinkItem; + type SinkError = S::SinkError; + + #[inline] + fn start_send(&mut self, item: Self::SinkItem) -> StartSend { + self.raw_stream.start_send(item) + } + + #[inline] + fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { + self.raw_stream.poll_complete() + } +} diff --git a/libp2p-secio/src/codec/encode.rs b/libp2p-secio/src/codec/encode.rs new file mode 100644 index 00000000..82b75299 --- /dev/null +++ b/libp2p-secio/src/codec/encode.rs @@ -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 { + cipher_state: Box, + hmac_key: hmac::SigningKey, + raw_sink: S, +} + +impl EncoderMiddleware { + pub fn new( + raw_sink: S, + cipher: Box, + hmac_key: hmac::SigningKey, + ) -> EncoderMiddleware { + EncoderMiddleware { + cipher_state: cipher, + hmac_key: hmac_key, + raw_sink: raw_sink, + } + } +} + +impl Sink for EncoderMiddleware + where S: Sink +{ + type SinkItem = BytesMut; + type SinkError = S::SinkError; + + fn start_send(&mut self, item: Self::SinkItem) -> StartSend { + 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 Stream for EncoderMiddleware + where S: Stream +{ + type Item = S::Item; + type Error = S::Error; + + #[inline] + fn poll(&mut self) -> Poll, Self::Error> { + self.raw_sink.poll() + } +} diff --git a/libp2p-secio/src/codec/mod.rs b/libp2p-secio/src/codec/mod.rs new file mode 100644 index 00000000..dc16840c --- /dev/null +++ b/libp2p-secio/src/codec/mod.rs @@ -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 = DecoderMiddleware>>; + +/// 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( + socket: length_delimited::Framed, + cipher_encoding: Box, + encoding_hmac: hmac::SigningKey, + cipher_decoder: Box, + decoding_hmac: hmac::VerificationKey, +) -> FullCodec + 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::(256); + let data_tx = data_tx.sink_map_err::<_, IoError>(|_| panic!()); + let data_rx = data_rx.map_err::(|_| 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::() + .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); + } +} diff --git a/libp2p-secio/src/error.rs b/libp2p-secio/src/error.rs new file mode 100644 index 00000000..bba96a02 --- /dev/null +++ b/libp2p-secio/src/error.rs @@ -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 for SecioError { + #[inline] + fn from(err: SymmetricCipherError) -> SecioError { + SecioError::CipherError(err) + } +} + +impl From for SecioError { + #[inline] + fn from(err: IoError) -> SecioError { + SecioError::IoError(err) + } +} diff --git a/libp2p-secio/src/handshake.rs b/libp2p-secio/src/handshake.rs new file mode 100644 index 00000000..f4869ad7 --- /dev/null +++ b/libp2p-secio/src/handshake.rs @@ -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, + local_private_key: Arc, +) -> Box, Vec), 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, + local_private_key: Arc, + + 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, + local_proposition_bytes: Vec, + + // The remote proposition's raw bytes. + remote_proposition_bytes: BytesMut, + remote_public_key_in_protobuf_bytes: Vec, + remote_public_key: Vec, + + // 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, + + // 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, + chosen_hash: Option<&'static digest::Algorithm>, + + // Ephemeral key generated for the handshake and then thrown away. + local_tmp_priv_key: Option, + 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::(&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::(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::(&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(); + } +} diff --git a/libp2p-secio/src/keys_proto.rs b/libp2p-secio/src/keys_proto.rs new file mode 100644 index 00000000..f9a0ecfb --- /dev/null +++ b/libp2p-secio/src/keys_proto.rs @@ -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, + Data: ::protobuf::SingularField<::std::vec::Vec>, + // 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 = ::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 { + &self.Type + } + + fn mut_Type_for_reflect(&mut self) -> &mut ::std::option::Option { + &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) { + 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 { + 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 { + 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> { + &self.Data + } + + fn mut_Data_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec> { + &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) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + ::protobuf::MessageStatic::descriptor_static(None::) + } +} + +impl ::protobuf::MessageStatic for PublicKey { + fn new() -> PublicKey { + PublicKey::new() + } + + fn descriptor_static(_: ::std::option::Option) -> &'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>( + "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", + 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, + Data: ::protobuf::SingularField<::std::vec::Vec>, + // 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 = ::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 { + &self.Type + } + + fn mut_Type_for_reflect(&mut self) -> &mut ::std::option::Option { + &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) { + 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 { + 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 { + 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> { + &self.Data + } + + fn mut_Data_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec> { + &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) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + ::protobuf::MessageStatic::descriptor_static(None::) + } +} + +impl ::protobuf::MessageStatic for PrivateKey { + fn new() -> PrivateKey { + PrivateKey::new() + } + + fn descriptor_static(_: ::std::option::Option) -> &'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>( + "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", + 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 { + 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) -> &'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() + }) + } +} diff --git a/libp2p-secio/src/lib.rs b/libp2p-secio/src/lib.rs new file mode 100644 index 00000000..9038e014 --- /dev/null +++ b/libp2p-secio/src/lib.rs @@ -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 { + inner: codec::FullCodec, + remote_pubkey_der: Vec, +} + +impl SecioMiddleware + 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, + local_private_key: Arc, + ) -> Box, 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 Sink for SecioMiddleware + where S: AsyncRead + AsyncWrite +{ + type SinkItem = BytesMut; + type SinkError = IoError; + + #[inline] + fn start_send(&mut self, item: Self::SinkItem) -> StartSend { + self.inner.start_send(item) + } + + #[inline] + fn poll_complete(&mut self) -> Poll<(), Self::SinkError> { + self.inner.poll_complete() + } +} + +impl Stream for SecioMiddleware + where S: AsyncRead + AsyncWrite +{ + type Item = Vec; + type Error = SecioError; + + #[inline] + fn poll(&mut self) -> Poll, Self::Error> { + self.inner.poll() + } +} diff --git a/libp2p-secio/src/structs_proto.rs b/libp2p-secio/src/structs_proto.rs new file mode 100644 index 00000000..4eaf9338 --- /dev/null +++ b/libp2p-secio/src/structs_proto.rs @@ -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>, + pubkey: ::protobuf::SingularField<::std::vec::Vec>, + 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 = ::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) { + 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 { + 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 { + 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> { + &self.rand + } + + fn mut_rand_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec> { + &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) { + 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 { + 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 { + 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> { + &self.pubkey + } + + fn mut_pubkey_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec> { + &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) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + ::protobuf::MessageStatic::descriptor_static(None::) + } +} + +impl ::protobuf::MessageStatic for Propose { + fn new() -> Propose { + Propose::new() + } + + fn descriptor_static(_: ::std::option::Option) -> &'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", + 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>, + signature: ::protobuf::SingularField<::std::vec::Vec>, + // 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 = ::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) { + 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 { + 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 { + 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> { + &self.epubkey + } + + fn mut_epubkey_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec> { + &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) { + 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 { + 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 { + 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> { + &self.signature + } + + fn mut_signature_for_reflect(&mut self) -> &mut ::protobuf::SingularField<::std::vec::Vec> { + &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) -> ::std::boxed::Box<::std::any::Any> { + self + } + + fn descriptor(&self) -> &'static ::protobuf::reflect::MessageDescriptor { + ::protobuf::MessageStatic::descriptor_static(None::) + } +} + +impl ::protobuf::MessageStatic for Exchange { + fn new() -> Exchange { + Exchange::new() + } + + fn descriptor_static(_: ::std::option::Option) -> &'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", + 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() + }) + } +} diff --git a/libp2p-secio/structs.proto b/libp2p-secio/structs.proto new file mode 100644 index 00000000..bdd1eecc --- /dev/null +++ b/libp2p-secio/structs.proto @@ -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; +} \ No newline at end of file diff --git a/libp2p-secio/tests/test-private-key-2.pk8 b/libp2p-secio/tests/test-private-key-2.pk8 new file mode 100644 index 0000000000000000000000000000000000000000..85ff4154f4bcc9610bf1f855eb7314080a81459f GIT binary patch literal 1217 zcmV;y1U~yPf&{$+0RS)!1_>&LNQUrr!ay9qXGc{0)hbn0JCW26QZK| zvn8zQ+xIj~72{qw@>%mDZ}9rSfI_J$gi+_i5vK+4~s71QJ%wo~<(4=2H|L8OEPpPu9BOx4%@-V~w@jIdW7t$+@R^GMju8mUWyFdTTv$-s2u( z6iu1)n%0^rcJ}-vnJZ#uTcxL&dXt!8GNX;M?GiKdoK)-$Z|ug@R;BRMCAx$kT}EiA zn5$WSgxeRlIl>)oS&`aKrkWc6nQ3i+PyoOyWkuu1@!CUzo$2&G{Q?64009Dm0RaGk z6o)*CR~WE+`u-IeOncDb3@5AE*~W9QULB~pN!^w&yX6OIl-6anWIrsf$`&)8FcPS{ z--mbQnqCdvy5=_vDLumC9Eby3YHINi%tq87lCSeMKmxZei6L0u@_;#RLX+Eh5jDBm zUnCx1YjXlH&6ECqeF#)CIf>d1M~i9(`9fL@^VFGF;A?6epfifb+D9uHWTFEeZhwfq?+) zF;f!_vn};XoJi4k3Kh~B+62;G%~Y{IhRI_rGzTlP{qRF_sf%zcIdD3hn z9h9+|7sZ9Jrd)zB*Ywc(OLK z6`VX1+J_wER!PK9A)UXbhgCHDAU7offq?+SNHFF&j0D^J&`jRy+c2@;MP!9_Xt|v& z-KpquY}$dNu*&>mMIqKF#t$pGo@KYHb07E$!0!x|ftE`&bpoWXA-M=10^EeOZ!?zU zESTHaYOR@s9s1eX8N2TnitRtNJaD_*MN%MIZ$DUh>}M^X*HxT!E0nip|1bVge{A^z zfq*4vuhp#W9aj*kux}UUlegJ|vKOBkI_UwAz(KT&1;p-e(uN~e#z*|9B|7ULZMMjT zv~ZkR9S7~$IRfRtQIf(VE>7zO$}gRQG%JDGt~7z@3cUh>fOK+?_BgTz6UWUaiKjB1j`O_5d~eRn zAXhomPM11SITbqZ zc>Pe>?x)ZUPL?=$|El_IY8yXER%S*Es|4xXz3@jy%74HIb2=5BffV9pe@$Pl!bX`$ f1k2W~M#~~I_4;=ykf&LNQUrsW5^Br2+u}0)hbn0K;(V^2bVN z^6JnD0u>N`%LEsu=I@2}mws$CbWZw13NOPfhRl(fuf3{^%zE zD<5MhzfrHEn>LrHpH`#5@t7bD8o-i^V=Z;bf&k)}fueF0QupYyHZRMH-uoYB#V%=N zP5Q+;IGb9t&-r`kVG&K-9bm~s`9D}dMdRgdzdCitb$gkA$N51Fe#0l^@v)BOi=pF! z`Ee_Vz+JC$&DjIIlc`C|K9#QE8LTjRz{;TxU0W)pfj@U~wc)_w(^C2{o?Rir)+30o zR2gNwrrG`fccEYmqUfxDN}|cJc!?HM;Mby!x^aOVzSRI+m}i|$69NMP009Dm0RTb? zWm%&A0V0z$x7K_|mXHEb`;!bfnuv44sQ)ihEVaJ_*9^e z`_Uz!2bc2bA_)plRLd)vZh*u zqaF!7WvpOc@2Q4m;}isE24<*f((VOG(2WS-8{^b@!Q|$h?w!1sMshmNeumGeJt?R( z^JelR{z+T^kR;KJ8xi#F@JhN8zK5m^^X?D z(FeGrDk+4fdJTs*j;f0FkaJ!?cj$D@iO#Ee5>61+=hO1 zeFCSr6T)Xnl#9T4hUB%qvK5|615cI3Ms;J8XTP-}L{F>I*{ENia^=mQaYm^ZmSY=~}B_sjk(cKnl}2vkQJzY(GH!h_WJV#ExlBBF?MyN#H|yG5Z36 zfdHS|G=S4qRA+@;Ro?vd5N;!;{e)P)lZ}gA}fdHS(jQkNk$Uy_0jtgnPEOu>zSC&I2 z0q>-J_7Jj_5XdEvQ8)gXH2kmtO5oRwN&7!ebDW7At+Mw^knOjkiVjd*4dD5zS$p1; z{ui_P!?rM=V@our1ArXI^38s-z&Q@%ik>o?U%xoQbbJm=5!+q5rYUeBL1F{f>v_I% zKxYDhfdI?^vg%7f{7?uV0L(D$=Uk}TOx#P}#7fUFJw)onjI-GyOsQ6JP>Uv* hv60PPFuRS$Ty2a9Lxa2c4}sY^u8x&NYgA)~zP3@lPz3-0 literal 0 HcmV?d00001 diff --git a/libp2p-secio/tests/test-public-key-2.der b/libp2p-secio/tests/test-public-key-2.der new file mode 100644 index 0000000000000000000000000000000000000000..db7b01d42279b358a75feba8e3d9ac50a673a714 GIT binary patch literal 294 zcmV+>0ondAf&n5h4F(A+hDe6@4FLfG1potr0S^E$f&mHwf&l>lvuNcLqN4e;C9N2L zO*{8^{dV{{ut_BS^TO-Q0Qfi^^mR4%h8mRIwQn~-%HN4(` zqcdPZq=W>Yp5y3D)z|&r6rv4`uvZEW|0Kh9{MdQcu+Cze!>GVGR0s{d60UG6rCIA2c literal 0 HcmV?d00001 diff --git a/libp2p-secio/tests/test-public-key.der b/libp2p-secio/tests/test-public-key.der new file mode 100644 index 0000000000000000000000000000000000000000..9e62c93ec1938e6e003e97d46d595414be26460c GIT binary patch literal 294 zcmV+>0ondAf&n5h4F(A+hDe6@4FLfG1potr0S^E$f&mHwf&l>l!*J^I$4Y1N>d*-S z6%c;Q1Q({}?;Z;rN9D3rUM+P2KG5e0NG@T=mA+iGf6Hx8P4D;7{U~7m=qCRwA7d!L zQLmz#HkYTLR-?f2m>>-rz>