Complete initial testing of new RPC

This commit is contained in:
Age Manning 2019-07-23 22:45:42 +10:00
parent b350a78fec
commit 89ff7fb6b8
No known key found for this signature in database
GPG Key ID: 05EED64B79E06A93
6 changed files with 54 additions and 126 deletions

View File

@ -68,6 +68,7 @@ impl<TSubstream> Discovery<TSubstream> {
info!(log, "Local ENR: {}", local_enr.to_base64()); info!(log, "Local ENR: {}", local_enr.to_base64());
debug!(log, "Local Node Id: {}", local_enr.node_id()); debug!(log, "Local Node Id: {}", local_enr.node_id());
debug!(log, "Local ENR seq: {}", local_enr.seq());
let mut discovery = Discv5::new(local_enr, local_key.clone(), config.listen_address) let mut discovery = Discv5::new(local_enr, local_key.clone(), config.listen_address)
.map_err(|e| format!("Discv5 service failed: {:?}", e))?; .map_err(|e| format!("Discv5 service failed: {:?}", e))?;

View File

@ -113,11 +113,6 @@ where
resp_code_byte.copy_from_slice(&resp_byte); resp_code_byte.copy_from_slice(&resp_byte);
let resp_code = u8::from_be_bytes(resp_code_byte); let resp_code = u8::from_be_bytes(resp_code_byte);
if let Some(response) = RPCErrorResponse::internal_data(resp_code) {
self.response_code = None;
return Ok(Some(response));
}
self.response_code = Some(resp_code); self.response_code = Some(resp_code);
resp_code resp_code
} }

View File

@ -2,7 +2,7 @@
use ssz::{impl_decode_via_from, impl_encode_via_from}; use ssz::{impl_decode_via_from, impl_encode_via_from};
use ssz_derive::{Decode, Encode}; use ssz_derive::{Decode, Encode};
use types::{BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; use types::{BeaconBlockBody, Epoch, Hash256, Slot};
/* Request/Response data structures for RPC methods */ /* Request/Response data structures for RPC methods */
@ -136,11 +136,6 @@ pub struct BeaconBlockHeadersResponse {
pub headers: Vec<u8>, pub headers: Vec<u8>,
} }
#[derive(Encode, Decode, Debug)]
pub struct EncodeableBeaconBlockHeadersResponse {
pub headers: Vec<BeaconBlockHeader>,
}
/// Request a number of beacon block bodies from a peer. /// Request a number of beacon block bodies from a peer.
#[derive(Encode, Decode, Clone, Debug, PartialEq)] #[derive(Encode, Decode, Clone, Debug, PartialEq)]
pub struct BeaconBlockBodiesRequest { pub struct BeaconBlockBodiesRequest {
@ -208,16 +203,6 @@ pub enum RPCErrorResponse {
} }
impl RPCErrorResponse { impl RPCErrorResponse {
/// If a response has no payload, returns the variant corresponding to the code.
pub fn internal_data(response_code: u8) -> Option<RPCErrorResponse> {
match response_code {
// EncodingError
1 => Some(RPCErrorResponse::EncodingError),
// All others require further data
_ => None,
}
}
/// Used to encode the response. /// Used to encode the response.
pub fn as_u8(&self) -> u8 { pub fn as_u8(&self) -> u8 {
match self { match self {

View File

@ -20,7 +20,7 @@ use tokio::util::FutureExt;
/// The maximum bytes that can be sent across the RPC. /// The maximum bytes that can be sent across the RPC.
const MAX_RPC_SIZE: usize = 4_194_304; // 4M const MAX_RPC_SIZE: usize = 4_194_304; // 4M
/// The protocol prefix the RPC protocol id. /// The protocol prefix the RPC protocol id.
const PROTOCOL_PREFIX: &str = "/eth/serenity/rpc/"; const PROTOCOL_PREFIX: &str = "/eth2/beacon_node/rpc";
/// The number of seconds to wait for a request once a protocol has been established before the stream is terminated. /// The number of seconds to wait for a request once a protocol has been established before the stream is terminated.
const REQUEST_TIMEOUT: u64 = 3; const REQUEST_TIMEOUT: u64 = 3;
@ -72,16 +72,16 @@ impl ProtocolId {
pub fn from_bytes(bytes: &[u8]) -> Result<Self, RPCError> { pub fn from_bytes(bytes: &[u8]) -> Result<Self, RPCError> {
let protocol_string = String::from_utf8(bytes.to_vec()) let protocol_string = String::from_utf8(bytes.to_vec())
.map_err(|_| RPCError::InvalidProtocol("Invalid protocol Id"))?; .map_err(|_| RPCError::InvalidProtocol("Invalid protocol Id"))?;
let protocol_list: Vec<&str> = protocol_string.as_str().split('/').take(5).collect(); let protocol_list: Vec<&str> = protocol_string.as_str().split('/').take(7).collect();
if protocol_list.len() != 5 { if protocol_list.len() != 7 {
return Err(RPCError::InvalidProtocol("Not enough '/'")); return Err(RPCError::InvalidProtocol("Not enough '/'"));
} }
Ok(ProtocolId { Ok(ProtocolId {
message_name: protocol_list[3].into(), message_name: protocol_list[4].into(),
version: protocol_list[4].into(), version: protocol_list[5].into(),
encoding: protocol_list[5].into(), encoding: protocol_list[6].into(),
}) })
} }
} }

View File

@ -10,12 +10,11 @@ use eth2_libp2p::{
}; };
use futures::future::Future; use futures::future::Future;
use futures::stream::Stream; use futures::stream::Stream;
use slog::{debug, error, warn}; use slog::{debug, warn};
use ssz::Decode; use ssz::{Decode, DecodeError};
use std::collections::HashMap;
use std::sync::Arc; use std::sync::Arc;
use std::time::Instant;
use tokio::sync::mpsc; use tokio::sync::mpsc;
use types::BeaconBlockHeader;
/// Handles messages received from the network and client and organises syncing. /// Handles messages received from the network and client and organises syncing.
pub struct MessageHandler<T: BeaconChainTypes> { pub struct MessageHandler<T: BeaconChainTypes> {
@ -97,8 +96,6 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
HandlerMessage::PubsubMessage(peer_id, gossip) => { HandlerMessage::PubsubMessage(peer_id, gossip) => {
self.handle_gossip(peer_id, *gossip); self.handle_gossip(peer_id, *gossip);
} }
//TODO: Handle all messages
_ => {}
} }
} }
@ -115,7 +112,6 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
/// A new RPC request has been received from the network. /// A new RPC request has been received from the network.
fn handle_rpc_request(&mut self, peer_id: PeerId, request_id: RequestId, request: RPCRequest) { fn handle_rpc_request(&mut self, peer_id: PeerId, request_id: RequestId, request: RPCRequest) {
// TODO: process the `id`.
match request { match request {
RPCRequest::Hello(hello_message) => self.sync.on_hello_request( RPCRequest::Hello(hello_message) => self.sync.on_hello_request(
peer_id, peer_id,
@ -158,30 +154,6 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
id: RequestId, id: RequestId,
error_response: RPCErrorResponse, error_response: RPCErrorResponse,
) { ) {
//TODO: Potentially do not need to keep track of this at all. This has all been shifted
//into libp2p stack. Tracking Id's will only be necessary if a response is important
//relative to a specific request. Note: BeaconBlockBodies already returns with the data
//associated with its request.
// Currently leave this here for testing, to ensure it is redundant.
if self
.network_context
.outstanding_outgoing_request_ids
.remove(&(peer_id.clone(), id))
.is_none()
{
// This should never happen. The RPC layer handles all timeouts and ensures a response
// matches a request.
debug_assert!(false);
error!(
self.log,
"Unknown ResponseId for incoming RPCRequest";
"peer" => format!("{:?}", peer_id),
"request_id" => format!("{}", id)
);
return;
}
// an error could have occurred. // an error could have occurred.
// TODO: Handle Error gracefully // TODO: Handle Error gracefully
match error_response { match error_response {
@ -214,27 +186,33 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
); );
} }
RPCResponse::BeaconBlockHeaders(response) => { RPCResponse::BeaconBlockHeaders(response) => {
if let Some(decoded_block_headers) = self.decode_block_headers(response) { match self.decode_block_headers(response) {
Ok(decoded_block_headers) => {
self.sync.on_beacon_block_headers_response( self.sync.on_beacon_block_headers_response(
peer_id, peer_id,
decoded_block_headers, decoded_block_headers,
&mut self.network_context, &mut self.network_context,
); );
} else { }
Err(_e) => {
warn!(self.log, "Peer sent invalid block headers";"peer" => format!("{:?}", peer_id)) warn!(self.log, "Peer sent invalid block headers";"peer" => format!("{:?}", peer_id))
} }
} }
}
RPCResponse::BeaconBlockBodies(response) => { RPCResponse::BeaconBlockBodies(response) => {
if let Some(decoded_block_bodies) = self.decode_block_bodies(response) { match self.decode_block_bodies(response) {
Ok(decoded_block_bodies) => {
self.sync.on_beacon_block_bodies_response( self.sync.on_beacon_block_bodies_response(
peer_id, peer_id,
decoded_block_bodies, decoded_block_bodies,
&mut self.network_context, &mut self.network_context,
); );
} else { }
Err(_e) => {
warn!(self.log, "Peer sent invalid block bodies";"peer" => format!("{:?}", peer_id)) warn!(self.log, "Peer sent invalid block bodies";"peer" => format!("{:?}", peer_id))
} }
} }
}
RPCResponse::BeaconChainState(_) => { RPCResponse::BeaconChainState(_) => {
// We do not implement this endpoint, it is not required and will only likely be // We do not implement this endpoint, it is not required and will only likely be
// useful for light-client support in later phases. // useful for light-client support in later phases.
@ -252,27 +230,24 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
fn decode_block_bodies( fn decode_block_bodies(
&self, &self,
bodies_response: BeaconBlockBodiesResponse, bodies_response: BeaconBlockBodiesResponse,
) -> Option<DecodedBeaconBlockBodiesResponse> { ) -> Result<DecodedBeaconBlockBodiesResponse, DecodeError> {
//TODO: Implement faster block verification before decoding entirely //TODO: Implement faster block verification before decoding entirely
if let Ok(simple_decoded_bodies) = simple_decoded_bodies { let block_bodies = Vec::from_ssz_bytes(&bodies_response.block_bodies)?;
Some(DecodedBeaconBlockBodiesResponse { Ok(DecodedBeaconBlockBodiesResponse {
block_roots: bodies_response block_roots: bodies_response
.block_roots .block_roots
.expect("Responses must have associated roots"), .expect("Responses must have associated roots"),
block_bodies: Vec::from_ssz_bytes(&bodies_response.block_bodies).unwrap(), block_bodies,
}) })
} else {
None
}
} }
/// Verifies and decodes the ssz-encoded block headers received from peers. /// Verifies and decodes the ssz-encoded block headers received from peers.
fn decode_block_headers( fn decode_block_headers(
&self, &self,
headers_response: BeaconBlockHeadersResponse, headers_response: BeaconBlockHeadersResponse,
) -> Option<EncodeableBeaconBlockHeadersResponse> { ) -> Result<Vec<BeaconBlockHeader>, DecodeError> {
//TODO: Implement faster header verification before decoding entirely //TODO: Implement faster header verification before decoding entirely
EncodeableBeaconBlockHeadersResponse::from_ssz_bytes(&headers_response.headers).ok() Vec::from_ssz_bytes(&headers_response.headers)
} }
/// Handle various RPC errors /// Handle various RPC errors
@ -297,25 +272,17 @@ impl<T: BeaconChainTypes + 'static> MessageHandler<T> {
} }
} }
// TODO: RPC Rewrite makes this struct fairly pointless
pub struct NetworkContext { pub struct NetworkContext {
/// The network channel to relay messages to the Network service. /// The network channel to relay messages to the Network service.
network_send: mpsc::UnboundedSender<NetworkMessage>, network_send: mpsc::UnboundedSender<NetworkMessage>,
/// A mapping of peers and the RPC id we have sent an RPC request to.
outstanding_outgoing_request_ids: HashMap<(PeerId, RequestId), Instant>,
/// Stores the next `RequestId` we should include on an outgoing `RPCRequest` to a `PeerId`.
outgoing_request_ids: HashMap<PeerId, RequestId>,
/// The `MessageHandler` logger. /// The `MessageHandler` logger.
log: slog::Logger, log: slog::Logger,
} }
impl NetworkContext { impl NetworkContext {
pub fn new(network_send: mpsc::UnboundedSender<NetworkMessage>, log: slog::Logger) -> Self { pub fn new(network_send: mpsc::UnboundedSender<NetworkMessage>, log: slog::Logger) -> Self {
Self { Self { network_send, log }
network_send,
outstanding_outgoing_request_ids: HashMap::new(),
outgoing_request_ids: HashMap::new(),
log,
}
} }
pub fn disconnect(&mut self, peer_id: PeerId, reason: GoodbyeReason) { pub fn disconnect(&mut self, peer_id: PeerId, reason: GoodbyeReason) {
@ -324,12 +291,9 @@ impl NetworkContext {
} }
pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest) { pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest) {
let id = self.generate_request_id(&peer_id); // Note: There is currently no use of keeping track of requests. However the functionality
// is left here for future revisions.
self.outstanding_outgoing_request_ids self.send_rpc_event(peer_id, RPCEvent::Request(0, rpc_request));
.insert((peer_id.clone(), id), Instant::now());
self.send_rpc_event(peer_id, RPCEvent::Request(id, rpc_request));
} }
//TODO: Handle Error responses //TODO: Handle Error responses
@ -359,15 +323,4 @@ impl NetworkContext {
) )
}); });
} }
/// Returns the next `RequestId` for sending an `RPCRequest` to the `peer_id`.
fn generate_request_id(&mut self, peer_id: &PeerId) -> RequestId {
let next_id = self
.outgoing_request_ids
.entry(peer_id.clone())
.and_modify(|id| *id += 1)
.or_insert_with(|| 0);
*next_id
}
} }

View File

@ -422,11 +422,7 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
.collect(); .collect();
// ssz-encode the headers // ssz-encode the headers
//TODO: Make this more elegant let headers = headers.as_ssz_bytes();
let headers = {
let resp = EncodeableBeaconBlockHeadersResponse { headers };
resp.as_ssz_bytes()
};
network.send_rpc_response( network.send_rpc_response(
peer_id, peer_id,
@ -439,17 +435,17 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
pub fn on_beacon_block_headers_response( pub fn on_beacon_block_headers_response(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
res: EncodeableBeaconBlockHeadersResponse, headers: Vec<BeaconBlockHeader>,
network: &mut NetworkContext, network: &mut NetworkContext,
) { ) {
debug!( debug!(
self.log, self.log,
"BlockHeadersResponse"; "BlockHeadersResponse";
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
"count" => res.headers.len(), "count" => headers.len(),
); );
if res.headers.is_empty() { if headers.is_empty() {
warn!( warn!(
self.log, self.log,
"Peer returned empty block headers response. PeerId: {:?}", peer_id "Peer returned empty block headers response. PeerId: {:?}", peer_id
@ -459,9 +455,7 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
// Enqueue the headers, obtaining a list of the roots of the headers which were newly added // Enqueue the headers, obtaining a list of the roots of the headers which were newly added
// to the queue. // to the queue.
let block_roots = self let block_roots = self.import_queue.enqueue_headers(headers, peer_id.clone());
.import_queue
.enqueue_headers(res.headers, peer_id.clone());
if !block_roots.is_empty() { if !block_roots.is_empty() {
self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network); self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network);
@ -503,7 +497,7 @@ impl<T: BeaconChainTypes> SimpleSync<T> {
"returned" => block_bodies.len(), "returned" => block_bodies.len(),
); );
let bytes = block_bodes.as_ssz_bytes(); let bytes = block_bodies.as_ssz_bytes();
network.send_rpc_response( network.send_rpc_response(
peer_id, peer_id,