Super tiny RPC refactor (#1187)

* wip: mwake the request id optional

* make the request_id optional

* cleanup

* address clippy lints inside rpc

* WIP: Separate sent RPC events from received ones

* WIP: Separate sent RPC events from received ones

* cleanup

* Separate request ids from substream ids

* Make RPC's message handling independent of RequestIds

* Change behaviour RPC events to be more outside-crate friendly

* Propage changes across the network + router + processor

* Propage changes across the network + router + processor

* fmt

* "tiny" refactor

* more tiny refactors

* fmt eth2-libp2p

* wip: propagating changes

* wip: propagating changes

* cleaning up

* more cleanup

* fmt

* tests HOT fix

Co-authored-by: Age Manning <Age@AgeManning.com>
This commit is contained in:
divma 2020-06-04 22:07:59 -05:00 committed by GitHub
parent 042e80570c
commit 0e37a16927
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 1445 additions and 1175 deletions

View File

@ -26,7 +26,7 @@ use std::{
sync::Arc, sync::Arc,
task::{Context, Poll}, task::{Context, Poll},
}; };
use types::{EnrForkId, EthSpec, SubnetId}; use types::{EnrForkId, EthSpec, SignedBeaconBlock, SubnetId};
mod handler; mod handler;
@ -393,8 +393,36 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
/* Eth2 RPC behaviour functions */ /* Eth2 RPC behaviour functions */
/// Send a request to a peer over RPC.
pub fn send_request(&mut self, peer_id: PeerId, request_id: RequestId, request: Request) {
self.send_rpc(peer_id, RPCSend::Request(request_id, request.into()))
}
/// Send a successful response to a peer over RPC.
pub fn send_successful_response(
&mut self,
peer_id: PeerId,
stream_id: SubstreamId,
response: Response<TSpec>,
) {
self.send_rpc(peer_id, RPCSend::Response(stream_id, response.into()))
}
/// Inform the peer that their request produced an error.
pub fn _send_error_reponse(
&mut self,
peer_id: PeerId,
stream_id: SubstreamId,
error: RPCResponseErrorCode,
reason: String,
) {
self.send_rpc(
peer_id,
RPCSend::Response(stream_id, RPCCodedResponse::from_error_code(error, reason)),
)
}
/// Sends an RPC Request/Response via the RPC protocol. /// Sends an RPC Request/Response via the RPC protocol.
pub fn send_rpc(&mut self, peer_id: PeerId, rpc_event: RPCEvent<TSpec>) { fn send_rpc(&mut self, peer_id: PeerId, rpc_event: RPCSend<TSpec>) {
self.eth2_rpc.send_rpc(peer_id, rpc_event); self.eth2_rpc.send_rpc(peer_id, rpc_event);
} }
@ -476,32 +504,38 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
.expect("Local discovery must have bitfield"); .expect("Local discovery must have bitfield");
} }
/// Sends a PING/PONG request/response to a peer. /// Sends a Ping request to the peer.
fn send_ping(&mut self, id: RequestId, peer_id: PeerId, is_request: bool) { fn ping(&mut self, id: RequestId, peer_id: PeerId) {
let ping = crate::rpc::methods::Ping { let ping = crate::rpc::Ping {
data: self.meta_data.seq_number, data: self.meta_data.seq_number,
}; };
let event = if is_request {
debug!(self.log, "Sending Ping"; "request_id" => id, "peer_id" => peer_id.to_string()); debug!(self.log, "Sending Ping"; "request_id" => id, "peer_id" => peer_id.to_string());
RPCEvent::Request(id, RPCRequest::Ping(ping)) let event = RPCSend::Request(id, RPCRequest::Ping(ping));
} else {
debug!(self.log, "Sending Pong"; "request_id" => id, "peer_id" => peer_id.to_string()); self.send_rpc(peer_id, event);
RPCEvent::Response(id, RPCCodedResponse::Success(RPCResponse::Pong(ping))) }
/// Sends a Pong response to the peer.
fn pong(&mut self, id: SubstreamId, peer_id: PeerId) {
let ping = crate::rpc::Ping {
data: self.meta_data.seq_number,
}; };
debug!(self.log, "Sending Pong"; "request_id" => id, "peer_id" => peer_id.to_string());
let event = RPCSend::Response(id, RPCCodedResponse::Success(RPCResponse::Pong(ping)));
self.send_rpc(peer_id, event); self.send_rpc(peer_id, event);
} }
/// Sends a METADATA request to a peer. /// Sends a METADATA request to a peer.
fn send_meta_data_request(&mut self, peer_id: PeerId) { fn send_meta_data_request(&mut self, peer_id: PeerId) {
let metadata_request = let metadata_request =
RPCEvent::Request(RequestId::from(0usize), RPCRequest::MetaData(PhantomData)); RPCSend::Request(RequestId::Behaviour, RPCRequest::MetaData(PhantomData));
self.send_rpc(peer_id, metadata_request); self.send_rpc(peer_id, metadata_request);
} }
/// Sends a METADATA response to a peer. /// Sends a METADATA response to a peer.
fn send_meta_data_response(&mut self, id: RequestId, peer_id: PeerId) { fn send_meta_data_response(&mut self, id: SubstreamId, peer_id: PeerId) {
let metadata_response = RPCEvent::Response( let metadata_response = RPCSend::Response(
id, id,
RPCCodedResponse::Success(RPCResponse::MetaData(self.meta_data.clone())), RPCCodedResponse::Success(RPCResponse::MetaData(self.meta_data.clone())),
); );
@ -587,45 +621,112 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
} }
} }
/// Queues the response to be sent upwards as long at it was requested outside the Behaviour.
fn propagate_response(&mut self, id: RequestId, peer_id: PeerId, response: Response<TSpec>) {
if !matches!(id, RequestId::Behaviour) {
self.events.push(BehaviourEvent::ResponseReceived {
peer_id,
id,
response,
});
}
}
/// Convenience function to propagate a request.
fn propagate_request(&mut self, id: SubstreamId, peer_id: PeerId, request: Request) {
self.events.push(BehaviourEvent::RequestReceived {
peer_id,
id,
request,
});
}
fn on_rpc_event(&mut self, message: RPCMessage<TSpec>) { fn on_rpc_event(&mut self, message: RPCMessage<TSpec>) {
let peer_id = message.peer_id; let peer_id = message.peer_id;
// The METADATA and PING RPC responses are handled within the behaviour and not // The METADATA and PING RPC responses are handled within the behaviour and not propagated
// propagated
// TODO: Improve the RPC types to better handle this logic discrepancy
match message.event { match message.event {
RPCEvent::Request(id, RPCRequest::Ping(ping)) => { Err(handler_err) => {
match handler_err {
HandlerErr::Inbound {
id: _,
proto,
error,
} => {
// Inform the peer manager of the error.
// An inbound error here means we sent an error to the peer, or the stream
// timed out.
self.peer_manager.handle_rpc_error(&peer_id, proto, &error);
}
HandlerErr::Outbound { id, proto, error } => {
// Inform the peer manager that a request we sent to the peer failed
self.peer_manager.handle_rpc_error(&peer_id, proto, &error);
// inform failures of requests comming outside the behaviour
if !matches!(id, RequestId::Behaviour) {
self.events
.push(BehaviourEvent::RPCFailed { peer_id, id, error });
}
}
}
}
Ok(RPCReceived::Request(id, request)) => match request {
/* Behaviour managed protocols: Ping and Metadata */
RPCRequest::Ping(ping) => {
// inform the peer manager and send the response // inform the peer manager and send the response
self.peer_manager.ping_request(&peer_id, ping.data); self.peer_manager.ping_request(&peer_id, ping.data);
// send a ping response // send a ping response
self.send_ping(id, peer_id, false); self.pong(id, peer_id);
} }
RPCEvent::Request(id, RPCRequest::MetaData(_)) => { RPCRequest::MetaData(_) => {
// send the requested meta-data // send the requested meta-data
self.send_meta_data_response(id, peer_id); self.send_meta_data_response(id, peer_id);
// TODO: inform the peer manager?
} }
RPCEvent::Response(_, RPCCodedResponse::Success(RPCResponse::Pong(ping))) => { /* Protocols propagated to the Network */
self.peer_manager.pong_response(&peer_id, ping.data); RPCRequest::Status(msg) => {
}
RPCEvent::Response(_, RPCCodedResponse::Success(RPCResponse::MetaData(meta_data))) => {
self.peer_manager.meta_data_response(&peer_id, meta_data);
}
RPCEvent::Request(_, RPCRequest::Status(_))
| RPCEvent::Response(_, RPCCodedResponse::Success(RPCResponse::Status(_))) => {
// inform the peer manager that we have received a status from a peer // inform the peer manager that we have received a status from a peer
self.peer_manager.peer_statusd(&peer_id); self.peer_manager.peer_statusd(&peer_id);
// propagate the STATUS message upwards // propagate the STATUS message upwards
self.events self.propagate_request(id, peer_id, Request::Status(msg))
.push(BehaviourEvent::RPC(peer_id, message.event));
} }
RPCEvent::Error(_, protocol, ref err) => { RPCRequest::BlocksByRange(req) => {
self.peer_manager.handle_rpc_error(&peer_id, protocol, err); self.propagate_request(id, peer_id, Request::BlocksByRange(req))
self.events
.push(BehaviourEvent::RPC(peer_id, message.event));
} }
_ => { RPCRequest::BlocksByRoot(req) => {
// propagate all other RPC messages upwards self.propagate_request(id, peer_id, Request::BlocksByRoot(req))
self.events }
.push(BehaviourEvent::RPC(peer_id, message.event)) RPCRequest::Goodbye(reason) => {
// TODO: do not propagate
self.propagate_request(id, peer_id, Request::Goodbye(reason));
}
},
Ok(RPCReceived::Response(id, resp)) => {
match resp {
/* Behaviour managed protocols */
RPCResponse::Pong(ping) => self.peer_manager.pong_response(&peer_id, ping.data),
RPCResponse::MetaData(meta_data) => {
self.peer_manager.meta_data_response(&peer_id, meta_data)
}
/* Network propagated protocols */
RPCResponse::Status(msg) => {
// inform the peer manager that we have received a status from a peer
self.peer_manager.peer_statusd(&peer_id);
// propagate the STATUS message upwards
self.propagate_response(id, peer_id, Response::Status(msg));
}
RPCResponse::BlocksByRange(resp) => {
self.propagate_response(id, peer_id, Response::BlocksByRange(Some(resp)))
}
RPCResponse::BlocksByRoot(resp) => {
self.propagate_response(id, peer_id, Response::BlocksByRoot(Some(resp)))
}
}
}
Ok(RPCReceived::EndOfStream(id, termination)) => {
let response = match termination {
ResponseTermination::BlocksByRange => Response::BlocksByRange(None),
ResponseTermination::BlocksByRoot => Response::BlocksByRoot(None),
};
self.propagate_response(id, peer_id, response);
} }
} }
} }
@ -648,7 +749,7 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
} }
PeerManagerEvent::Ping(peer_id) => { PeerManagerEvent::Ping(peer_id) => {
// send a ping request to this peer // send a ping request to this peer
self.send_ping(RequestId::from(0usize), peer_id, true); self.ping(RequestId::Behaviour, peer_id);
} }
PeerManagerEvent::MetaData(peer_id) => { PeerManagerEvent::MetaData(peer_id) => {
self.send_meta_data_request(peer_id); self.send_meta_data_request(peer_id);
@ -707,11 +808,96 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
} }
} }
/* Public API types */
/// The type of RPC requests the Behaviour informs it has received and allows for sending.
///
// NOTE: This is an application-level wrapper over the lower network leve requests that can be
// sent. The main difference is the absense of the Ping and Metadata protocols, which don't
// leave the Behaviour. For all protocols managed by RPC see `RPCRequest`.
#[derive(Debug, Clone, PartialEq)]
pub enum Request {
/// A Status message.
Status(StatusMessage),
/// A Goobye message.
Goodbye(GoodbyeReason),
/// A blocks by range request.
BlocksByRange(BlocksByRangeRequest),
/// A request blocks root request.
BlocksByRoot(BlocksByRootRequest),
}
impl<TSpec: EthSpec> std::convert::From<Request> for RPCRequest<TSpec> {
fn from(req: Request) -> RPCRequest<TSpec> {
match req {
Request::BlocksByRoot(r) => RPCRequest::BlocksByRoot(r),
Request::BlocksByRange(r) => RPCRequest::BlocksByRange(r),
Request::Goodbye(r) => RPCRequest::Goodbye(r),
Request::Status(s) => RPCRequest::Status(s),
}
}
}
/// The type of RPC responses the Behaviour informs it has received, and allows for sending.
///
// NOTE: This is an application-level wrapper over the lower network level responses that can be
// sent. The main difference is the absense of Pong and Metadata, which don't leave the
// Behaviour. For all protocol reponses managed by RPC see `RPCResponse` and
// `RPCCodedResponse`.
#[derive(Debug, Clone, PartialEq)]
pub enum Response<TSpec: EthSpec> {
/// A Status message.
Status(StatusMessage),
/// A response to a get BLOCKS_BY_RANGE request. A None response signals the end of the batch.
BlocksByRange(Option<Box<SignedBeaconBlock<TSpec>>>),
/// A response to a get BLOCKS_BY_ROOT request.
BlocksByRoot(Option<Box<SignedBeaconBlock<TSpec>>>),
}
impl<TSpec: EthSpec> std::convert::From<Response<TSpec>> for RPCCodedResponse<TSpec> {
fn from(resp: Response<TSpec>) -> RPCCodedResponse<TSpec> {
match resp {
Response::BlocksByRoot(r) => match r {
Some(b) => RPCCodedResponse::Success(RPCResponse::BlocksByRoot(b)),
None => RPCCodedResponse::StreamTermination(ResponseTermination::BlocksByRoot),
},
Response::BlocksByRange(r) => match r {
Some(b) => RPCCodedResponse::Success(RPCResponse::BlocksByRange(b)),
None => RPCCodedResponse::StreamTermination(ResponseTermination::BlocksByRange),
},
Response::Status(s) => RPCCodedResponse::Success(RPCResponse::Status(s)),
}
}
}
/// The types of events than can be obtained from polling the behaviour. /// The types of events than can be obtained from polling the behaviour.
#[derive(Debug)] #[derive(Debug)]
pub enum BehaviourEvent<TSpec: EthSpec> { pub enum BehaviourEvent<TSpec: EthSpec> {
/// A received RPC event and the peer that it was received from. /// An RPC Request that was sent failed.
RPC(PeerId, RPCEvent<TSpec>), RPCFailed {
/// The id of the failed request.
id: RequestId,
/// The peer to which this request was sent.
peer_id: PeerId,
/// The error that occurred.
error: RPCError,
},
RequestReceived {
/// The peer that sent the request.
peer_id: PeerId,
/// Identifier of the request. All responses to this request must use this id.
id: SubstreamId,
/// Request the peer sent.
request: Request,
},
ResponseReceived {
/// Peer that sent the response.
peer_id: PeerId,
/// Id of the request to which the peer is responding.
id: RequestId,
/// Response the peer sent.
response: Response<TSpec>,
},
PubsubMessage { PubsubMessage {
/// The gossipsub message id. Used when propagating blocks after validation. /// The gossipsub message id. Used when propagating blocks after validation.
id: MessageId, id: MessageId,

View File

@ -15,12 +15,11 @@ mod service;
pub mod types; pub mod types;
pub use crate::types::{error, Enr, GossipTopic, NetworkGlobals, PubsubMessage}; pub use crate::types::{error, Enr, GossipTopic, NetworkGlobals, PubsubMessage};
pub use behaviour::BehaviourEvent; pub use behaviour::{BehaviourEvent, Request, Response};
pub use config::Config as NetworkConfig; pub use config::Config as NetworkConfig;
pub use discovery::enr_ext::{CombinedKeyExt, EnrExt}; pub use discovery::enr_ext::{CombinedKeyExt, EnrExt};
pub use libp2p::gossipsub::{MessageId, Topic, TopicHash}; pub use libp2p::gossipsub::{MessageId, Topic, TopicHash};
pub use libp2p::{core::ConnectedPoint, PeerId, Swarm}; pub use libp2p::{core::ConnectedPoint, PeerId, Swarm};
pub use libp2p::{multiaddr, Multiaddr}; pub use libp2p::{multiaddr, Multiaddr};
pub use peer_manager::{client::Client, PeerDB, PeerInfo, PeerSyncStatus, SyncInfo}; pub use peer_manager::{client::Client, PeerDB, PeerInfo, PeerSyncStatus, SyncInfo};
pub use rpc::RPCEvent;
pub use service::{Libp2pEvent, Service, NETWORK_KEY_FILENAME}; pub use service::{Libp2pEvent, Service, NETWORK_KEY_FILENAME};

View File

@ -279,7 +279,7 @@ impl<TSpec: EthSpec> PeerManager<TSpec> {
// this could their fault or ours, so we tolerate this // this could their fault or ours, so we tolerate this
PeerAction::HighToleranceError PeerAction::HighToleranceError
} }
RPCError::ErrorResponse(code) => match code { RPCError::ErrorResponse(code, _) => match code {
RPCResponseErrorCode::Unknown => PeerAction::HighToleranceError, RPCResponseErrorCode::Unknown => PeerAction::HighToleranceError,
RPCResponseErrorCode::ServerError => PeerAction::MidToleranceError, RPCResponseErrorCode::ServerError => PeerAction::MidToleranceError,
RPCResponseErrorCode::InvalidRequest => PeerAction::LowToleranceError, RPCResponseErrorCode::InvalidRequest => PeerAction::LowToleranceError,

View File

@ -3,7 +3,7 @@
use super::methods::{RPCCodedResponse, RequestId, ResponseTermination}; use super::methods::{RPCCodedResponse, RequestId, ResponseTermination};
use super::protocol::{Protocol, RPCError, RPCProtocol, RPCRequest}; use super::protocol::{Protocol, RPCError, RPCProtocol, RPCRequest};
use super::RPCEvent; use super::{RPCReceived, RPCSend};
use crate::rpc::protocol::{InboundFramed, OutboundFramed}; use crate::rpc::protocol::{InboundFramed, OutboundFramed};
use fnv::FnvHashMap; use fnv::FnvHashMap;
use futures::prelude::*; use futures::prelude::*;
@ -33,12 +33,34 @@ pub const RESPONSE_TIMEOUT: u64 = 10;
/// The number of times to retry an outbound upgrade in the case of IO errors. /// The number of times to retry an outbound upgrade in the case of IO errors.
const IO_ERROR_RETRIES: u8 = 3; const IO_ERROR_RETRIES: u8 = 3;
/// Inbound requests are given a sequential `RequestId` to keep track of. All inbound streams are /// Identifier of inbound and outbound substreams from the handler's perspective.
/// identified by their substream ID which is identical to the RPC Id. #[derive(Debug, Clone, Copy, Hash, Eq, PartialEq)]
type InboundRequestId = RequestId; pub struct SubstreamId(usize);
/// Outbound requests are associated with an id that is given by the application that sent the
/// request. /// An error encoutered by the handler.
type OutboundRequestId = RequestId; pub enum HandlerErr {
/// An error ocurred for this peer's request. This can occurr during protocol negotiation,
/// message passing, or if the handler identifies that we are sending an error reponse to the peer.
Inbound {
/// Id of the peer's request for which an error occurred.
id: SubstreamId,
/// Information of the negotiated protocol.
proto: Protocol,
/// The error that ocurred.
error: RPCError,
},
/// An error ocurred for this request. Such error can occurr during protocol negotiation,
/// message passing, or if we successfully received a response from the peer, but this response
/// indicates an error.
Outbound {
/// Application-given Id of the request for which an error occurred.
id: RequestId,
/// Information of the protocol.
proto: Protocol,
/// The error that ocurred.
error: RPCError,
},
}
/// Implementation of `ProtocolsHandler` for the RPC protocol. /// Implementation of `ProtocolsHandler` for the RPC protocol.
pub struct RPCHandler<TSpec> pub struct RPCHandler<TSpec>
@ -48,11 +70,11 @@ where
/// The upgrade for inbound substreams. /// The upgrade for inbound substreams.
listen_protocol: SubstreamProtocol<RPCProtocol<TSpec>>, listen_protocol: SubstreamProtocol<RPCProtocol<TSpec>>,
/// If something bad happened and we should shut down the handler with an error. /// Errors ocurring on outbound and inbound connections queued for reporting back.
pending_error: Vec<(RequestId, Protocol, RPCError)>, pending_errors: Vec<HandlerErr>,
/// Queue of events to produce in `poll()`. /// Queue of events to produce in `poll()`.
events_out: SmallVec<[RPCEvent<TSpec>; 4]>, events_out: SmallVec<[RPCReceived<TSpec>; 4]>,
/// Queue of outbound substreams to open. /// Queue of outbound substreams to open.
dial_queue: SmallVec<[(RequestId, RPCRequest<TSpec>); 4]>, dial_queue: SmallVec<[(RequestId, RPCRequest<TSpec>); 4]>,
@ -62,7 +84,7 @@ where
/// Current inbound substreams awaiting processing. /// Current inbound substreams awaiting processing.
inbound_substreams: FnvHashMap< inbound_substreams: FnvHashMap<
InboundRequestId, SubstreamId,
( (
InboundSubstreamState<TSpec>, InboundSubstreamState<TSpec>,
Option<delay_queue::Key>, Option<delay_queue::Key>,
@ -71,29 +93,22 @@ where
>, >,
/// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout. /// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout.
inbound_substreams_delay: DelayQueue<InboundRequestId>, inbound_substreams_delay: DelayQueue<SubstreamId>,
/// Map of outbound substreams that need to be driven to completion. The `RequestId` is /// Map of outbound substreams that need to be driven to completion.
/// maintained by the application sending the request. outbound_substreams: FnvHashMap<SubstreamId, OutboundInfo<TSpec>>,
/// For Responses with multiple expected response chunks a counter is added to be able to terminate the stream when the expected number has been received
outbound_substreams: FnvHashMap<
OutboundRequestId,
(
OutboundSubstreamState<TSpec>,
delay_queue::Key,
Protocol,
Option<u64>,
),
>,
/// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout. /// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout.
outbound_substreams_delay: DelayQueue<OutboundRequestId>, outbound_substreams_delay: DelayQueue<SubstreamId>,
/// Map of outbound items that are queued as the stream processes them. /// Map of outbound items that are queued as the stream processes them.
queued_outbound_items: FnvHashMap<RequestId, Vec<RPCCodedResponse<TSpec>>>, queued_outbound_items: FnvHashMap<SubstreamId, Vec<RPCCodedResponse<TSpec>>>,
/// Sequential ID for waiting substreams. For inbound substreams, this is also the inbound request ID. /// Sequential ID for waiting substreams. For inbound substreams, this is also the inbound request ID.
current_inbound_substream_id: RequestId, current_inbound_substream_id: SubstreamId,
/// Sequential ID for outbound substreams.
current_outbound_substream_id: SubstreamId,
/// Maximum number of concurrent outbound substreams being opened. Value is never modified. /// Maximum number of concurrent outbound substreams being opened. Value is never modified.
max_dial_negotiated: u32, max_dial_negotiated: u32,
@ -112,6 +127,23 @@ where
log: slog::Logger, log: slog::Logger,
} }
/// Contains the information the handler keeps on established outbound substreams.
struct OutboundInfo<TSpec: EthSpec> {
/// State of the substream.
state: OutboundSubstreamState<TSpec>,
/// Key to keep track of the substream's timeout via `self.outbound_substreams_delay`.
delay_key: delay_queue::Key,
/// Info over the protocol this substream is handling.
proto: Protocol,
/// Number of chunks to be seen from the peer's response.
// TODO: removing the option could allow clossing the streams after the number of
// expected responses is met for all protocols.
// TODO: the type of this is wrong
remaining_chunks: Option<usize>,
/// RequestId as given by the application that sent the request.
req_id: RequestId,
}
pub enum InboundSubstreamState<TSpec> pub enum InboundSubstreamState<TSpec>
where where
TSpec: EthSpec, TSpec: EthSpec,
@ -208,7 +240,7 @@ where
} }
InboundSubstreamState::ResponseIdle(substream) => { InboundSubstreamState::ResponseIdle(substream) => {
*self = InboundSubstreamState::ResponsePendingSend { *self = InboundSubstreamState::ResponsePendingSend {
substream: substream, substream,
message: error, message: error,
closing: true, closing: true,
}; };
@ -235,7 +267,7 @@ where
) -> Self { ) -> Self {
RPCHandler { RPCHandler {
listen_protocol, listen_protocol,
pending_error: Vec::new(), pending_errors: Vec::new(),
events_out: SmallVec::new(), events_out: SmallVec::new(),
dial_queue: SmallVec::new(), dial_queue: SmallVec::new(),
dial_negotiated: 0, dial_negotiated: 0,
@ -244,7 +276,8 @@ where
outbound_substreams: FnvHashMap::default(), outbound_substreams: FnvHashMap::default(),
inbound_substreams_delay: DelayQueue::new(), inbound_substreams_delay: DelayQueue::new(),
outbound_substreams_delay: DelayQueue::new(), outbound_substreams_delay: DelayQueue::new(),
current_inbound_substream_id: 1, current_inbound_substream_id: SubstreamId(0),
current_outbound_substream_id: SubstreamId(0),
max_dial_negotiated: 8, max_dial_negotiated: 8,
keep_alive: KeepAlive::Yes, keep_alive: KeepAlive::Yes,
inactive_timeout, inactive_timeout,
@ -300,8 +333,8 @@ impl<TSpec> ProtocolsHandler for RPCHandler<TSpec>
where where
TSpec: EthSpec, TSpec: EthSpec,
{ {
type InEvent = RPCEvent<TSpec>; type InEvent = RPCSend<TSpec>;
type OutEvent = RPCEvent<TSpec>; type OutEvent = Result<RPCReceived<TSpec>, HandlerErr>;
type Error = RPCError; type Error = RPCError;
type InboundProtocol = RPCProtocol<TSpec>; type InboundProtocol = RPCProtocol<TSpec>;
type OutboundProtocol = RPCRequest<TSpec>; type OutboundProtocol = RPCRequest<TSpec>;
@ -316,9 +349,11 @@ where
substream: <Self::InboundProtocol as InboundUpgrade<NegotiatedSubstream>>::Output, substream: <Self::InboundProtocol as InboundUpgrade<NegotiatedSubstream>>::Output,
) { ) {
let (req, substream) = substream; let (req, substream) = substream;
// drop the stream and return a 0 id for goodbye "requests" // drop the stream
if let r @ RPCRequest::Goodbye(_) = req { if let RPCRequest::Goodbye(_) = req {
self.events_out.push(RPCEvent::Request(0, r)); self.events_out
.push(RPCReceived::Request(self.current_inbound_substream_id, req));
self.current_inbound_substream_id.0 += 1;
return; return;
} }
@ -334,8 +369,8 @@ where
); );
self.events_out self.events_out
.push(RPCEvent::Request(self.current_inbound_substream_id, req)); .push(RPCReceived::Request(self.current_inbound_substream_id, req));
self.current_inbound_substream_id += 1; self.current_inbound_substream_id.0 += 1;
} }
fn inject_fully_negotiated_outbound( fn inject_fully_negotiated_outbound(
@ -346,43 +381,42 @@ where
self.dial_negotiated -= 1; self.dial_negotiated -= 1;
// add the stream to substreams if we expect a response, otherwise drop the stream. // add the stream to substreams if we expect a response, otherwise drop the stream.
let (mut id, request) = request_info; let (id, request) = request_info;
if request.expect_response() { let expected_responses = request.expected_responses();
// outbound requests can be sent from various aspects of lighthouse which don't if expected_responses > 0 {
// track request ids. In the future these will be flagged as None, currently they
// are flagged as 0. These can overlap. In this case, we pick the highest request
// Id available
if id == 0 && self.outbound_substreams.get(&id).is_some() {
// have duplicate outbound request with no id. Pick one that will not collide
let mut new_id = std::usize::MAX;
while self.outbound_substreams.get(&new_id).is_some() {
// panic all outbound substreams are full
new_id -= 1;
}
trace!(self.log, "New outbound stream id created"; "id" => new_id);
id = RequestId::from(new_id);
}
// new outbound request. Store the stream and tag the output. // new outbound request. Store the stream and tag the output.
let delay_key = self let delay_key = self.outbound_substreams_delay.insert(
.outbound_substreams_delay self.current_outbound_substream_id,
.insert(id, Duration::from_secs(RESPONSE_TIMEOUT)); Duration::from_secs(RESPONSE_TIMEOUT),
let protocol = request.protocol(); );
let response_chunk_count = match request { let proto = request.protocol();
RPCRequest::BlocksByRange(ref req) => Some(req.count),
RPCRequest::BlocksByRoot(ref req) => Some(req.block_roots.len() as u64),
_ => None, // Other requests do not have a known response chunk length,
};
let awaiting_stream = OutboundSubstreamState::RequestPendingResponse { let awaiting_stream = OutboundSubstreamState::RequestPendingResponse {
substream: out, substream: out,
request: request, request,
}; };
if let Some(_) = self.outbound_substreams.insert( let expected_responses = if expected_responses > 1 {
id, // Currently enforced only for multiple responses
(awaiting_stream, delay_key, protocol, response_chunk_count), Some(expected_responses)
) { } else {
crit!(self.log, "Duplicate outbound substream id"; "id" => format!("{:?}", id)); None
};
if self
.outbound_substreams
.insert(
self.current_outbound_substream_id,
OutboundInfo {
state: awaiting_stream,
delay_key,
proto,
remaining_chunks: expected_responses,
req_id: id,
},
)
.is_some()
{
crit!(self.log, "Duplicate outbound substream id"; "id" => format!("{:?}", self.current_outbound_substream_id));
} }
self.current_outbound_substream_id.0 += 1;
} }
self.update_keep_alive(); self.update_keep_alive();
@ -392,38 +426,54 @@ where
// wrong state a response will fail silently. // wrong state a response will fail silently.
fn inject_event(&mut self, rpc_event: Self::InEvent) { fn inject_event(&mut self, rpc_event: Self::InEvent) {
match rpc_event { match rpc_event {
RPCEvent::Request(id, req) => self.send_request(id, req), RPCSend::Request(id, req) => self.send_request(id, req),
RPCEvent::Response(rpc_id, response) => { RPCSend::Response(inbound_id, response) => {
// Variables indicating if the response is an error response or a multi-part // Variables indicating if the response is an error response or a multi-part
// response // response
let res_is_error = response.is_error(); let res_is_error = response.is_error();
let res_is_multiple = response.multiple_responses(); let res_is_multiple = response.multiple_responses();
// check if the stream matching the response still exists // check if the stream matching the response still exists
match self.inbound_substreams.get_mut(&rpc_id) { let (substream_state, protocol) = match self.inbound_substreams.get_mut(&inbound_id)
Some((substream_state, _, protocol)) => { {
Some((substream_state, _, protocol)) => (substream_state, protocol),
None => {
warn!(self.log, "Stream has expired. Response not sent";
"response" => response.to_string(), "id" => inbound_id);
return;
}
};
// If the response we are sending is an error, report back for handling
match response {
RPCCodedResponse::InvalidRequest(ref reason)
| RPCCodedResponse::ServerError(ref reason)
| RPCCodedResponse::Unknown(ref reason) => {
let code = &response
.error_code()
.expect("Error response should map to an error code");
let err = HandlerErr::Inbound {
id: inbound_id,
proto: *protocol,
error: RPCError::ErrorResponse(*code, reason.clone()),
};
self.pending_errors.push(err);
}
_ => {} // not an error, continue.
}
match std::mem::replace(substream_state, InboundSubstreamState::Poisoned) { match std::mem::replace(substream_state, InboundSubstreamState::Poisoned) {
InboundSubstreamState::ResponseIdle(substream) => { InboundSubstreamState::ResponseIdle(substream) => {
// close the stream if there is no response // close the stream if there is no response
match response { match response {
RPCCodedResponse::StreamTermination(_) => { RPCCodedResponse::StreamTermination(_) => {
//trace!(self.log, "Stream termination sent. Ending the stream"); *substream_state = InboundSubstreamState::Closing(substream);
*substream_state =
InboundSubstreamState::Closing(substream);
} }
_ => { _ => {
if let Some(error_code) = response.error_code() {
self.pending_error.push((
rpc_id,
*protocol,
RPCError::ErrorResponse(error_code),
));
}
// send the response // send the response
// if it's a single rpc request or an error, close the stream after // if it's a single rpc request or an error, close the stream after
*substream_state = *substream_state = InboundSubstreamState::ResponsePendingSend {
InboundSubstreamState::ResponsePendingSend { substream,
substream: substream,
message: response, message: response,
closing: !res_is_multiple | res_is_error, // close if an error or we are not expecting more responses closing: !res_is_multiple | res_is_error, // close if an error or we are not expecting more responses
}; };
@ -437,7 +487,7 @@ where
} if res_is_multiple => { } if res_is_multiple => {
// the stream is in use, add the request to a pending queue // the stream is in use, add the request to a pending queue
self.queued_outbound_items self.queued_outbound_items
.entry(rpc_id) .entry(inbound_id)
.or_insert_with(Vec::new) .or_insert_with(Vec::new)
.push(response); .push(response);
@ -453,38 +503,40 @@ where
{ {
// the stream is in use, add the request to a pending queue // the stream is in use, add the request to a pending queue
self.queued_outbound_items self.queued_outbound_items
.entry(rpc_id) .entry(inbound_id)
.or_insert_with(Vec::new) .or_insert_with(Vec::new)
.push(response); .push(response);
// return the state // return the state
*substream_state = InboundSubstreamState::ResponsePendingFlush { *substream_state =
substream, InboundSubstreamState::ResponsePendingFlush { substream, closing };
closing,
};
} }
InboundSubstreamState::Closing(substream) => { InboundSubstreamState::Closing(substream) => {
*substream_state = InboundSubstreamState::Closing(substream); *substream_state = InboundSubstreamState::Closing(substream);
debug!(self.log, "Response not sent. Stream is closing"; "response" => format!("{}",response)); debug!(self.log, "Response not sent. Stream is closing"; "response" => format!("{}",response));
} }
InboundSubstreamState::ResponsePendingSend { InboundSubstreamState::ResponsePendingSend {
substream, substream, message, ..
message,
..
} => { } => {
*substream_state = InboundSubstreamState::ResponsePendingSend { *substream_state = InboundSubstreamState::ResponsePendingSend {
substream, substream,
message, message,
closing: true, closing: true,
}; };
error!(self.log, "Attempted sending multiple responses to a single response request"); error!(
self.log,
"Attempted sending multiple responses to a single response request"
);
} }
InboundSubstreamState::ResponsePendingFlush { substream, .. } => { InboundSubstreamState::ResponsePendingFlush { substream, .. } => {
*substream_state = InboundSubstreamState::ResponsePendingFlush { *substream_state = InboundSubstreamState::ResponsePendingFlush {
substream, substream,
closing: true, closing: true,
}; };
error!(self.log, "Attempted sending multiple responses to a single response request"); error!(
self.log,
"Attempted sending multiple responses to a single response request"
);
} }
InboundSubstreamState::Poisoned => { InboundSubstreamState::Poisoned => {
crit!(self.log, "Poisoned inbound substream"); crit!(self.log, "Poisoned inbound substream");
@ -492,13 +544,6 @@ where
} }
} }
} }
None => {
warn!(self.log, "Stream has expired. Response not sent"; "response" => response.to_string(), "id" => rpc_id);
}
};
}
// We do not send errors as responses
RPCEvent::Error(..) => {}
} }
} }
@ -520,7 +565,7 @@ where
self.outbound_io_error_retries = 0; self.outbound_io_error_retries = 0;
// map the error // map the error
let rpc_error = match error { let error = match error {
ProtocolsHandlerUpgrErr::Timer => RPCError::InternalError("Timer failed"), ProtocolsHandlerUpgrErr::Timer => RPCError::InternalError("Timer failed"),
ProtocolsHandlerUpgrErr::Timeout => RPCError::NegotiationTimeout, ProtocolsHandlerUpgrErr::Timeout => RPCError::NegotiationTimeout,
ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(e)) => e, ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(e)) => e,
@ -541,7 +586,11 @@ where
} }
}, },
}; };
self.pending_error.push((id, req.protocol(), rpc_error)); self.pending_errors.push(HandlerErr::Outbound {
id,
proto: req.protocol(),
error,
});
} }
fn connection_keep_alive(&self) -> KeepAlive { fn connection_keep_alive(&self) -> KeepAlive {
@ -559,16 +608,15 @@ where
Self::Error, Self::Error,
>, >,
> { > {
if !self.pending_error.is_empty() { // report failures
let (id, protocol, err) = self.pending_error.remove(0); if !self.pending_errors.is_empty() {
return Poll::Ready(ProtocolsHandlerEvent::Custom(RPCEvent::Error( let err_info = self.pending_errors.remove(0);
id, protocol, err, return Poll::Ready(ProtocolsHandlerEvent::Custom(Err(err_info)));
)));
} }
// return any events that need to be reported // return any events that need to be reported
if !self.events_out.is_empty() { if !self.events_out.is_empty() {
return Poll::Ready(ProtocolsHandlerEvent::Custom(self.events_out.remove(0))); return Poll::Ready(ProtocolsHandlerEvent::Custom(Ok(self.events_out.remove(0))));
} else { } else {
self.events_out.shrink_to_fit(); self.events_out.shrink_to_fit();
} }
@ -576,17 +624,23 @@ where
// purge expired inbound substreams and send an error // purge expired inbound substreams and send an error
loop { loop {
match self.inbound_substreams_delay.poll_next_unpin(cx) { match self.inbound_substreams_delay.poll_next_unpin(cx) {
Poll::Ready(Some(Ok(stream_id))) => { Poll::Ready(Some(Ok(inbound_id))) => {
// handle a stream timeout for various states // handle a stream timeout for various states
if let Some((substream_state, delay_key, _)) = if let Some((substream_state, delay_key, protocol)) =
self.inbound_substreams.get_mut(stream_id.get_ref()) self.inbound_substreams.get_mut(inbound_id.get_ref())
{ {
// the delay has been removed // the delay has been removed
*delay_key = None; *delay_key = None;
self.pending_errors.push(HandlerErr::Inbound {
id: *inbound_id.get_ref(),
proto: *protocol,
error: RPCError::StreamTimeout,
});
let outbound_queue = self let outbound_queue = self
.queued_outbound_items .queued_outbound_items
.entry(stream_id.into_inner()) .entry(inbound_id.into_inner())
.or_insert_with(Vec::new); .or_insert_with(Vec::new);
substream_state.close(outbound_queue); substream_state.close(outbound_queue);
} }
@ -605,20 +659,21 @@ where
// purge expired outbound substreams // purge expired outbound substreams
loop { loop {
match self.outbound_substreams_delay.poll_next_unpin(cx) { match self.outbound_substreams_delay.poll_next_unpin(cx) {
Poll::Ready(Some(Ok(stream_id))) => { Poll::Ready(Some(Ok(outbound_id))) => {
if let Some((_id, _stream, protocol, _)) = if let Some(OutboundInfo { proto, req_id, .. }) =
self.outbound_substreams.remove(stream_id.get_ref()) self.outbound_substreams.remove(outbound_id.get_ref())
{ {
self.update_keep_alive(); self.update_keep_alive();
let outbound_err = HandlerErr::Outbound {
id: req_id,
proto,
error: RPCError::StreamTimeout,
};
// notify the user // notify the user
return Poll::Ready(ProtocolsHandlerEvent::Custom(RPCEvent::Error( return Poll::Ready(ProtocolsHandlerEvent::Custom(Err(outbound_err)));
*stream_id.get_ref(),
protocol,
RPCError::StreamTimeout,
)));
} else { } else {
crit!(self.log, "timed out substream not in the books"; "stream_id" => stream_id.get_ref()); crit!(self.log, "timed out substream not in the books"; "stream_id" => outbound_id.get_ref());
} }
} }
Poll::Ready(Some(Err(e))) => { Poll::Ready(Some(Err(e))) => {
@ -797,53 +852,77 @@ where
} }
// drive outbound streams that need to be processed // drive outbound streams that need to be processed
for request_id in self.outbound_substreams.keys().copied().collect::<Vec<_>>() { for outbound_id in self.outbound_substreams.keys().copied().collect::<Vec<_>>() {
match self.outbound_substreams.entry(request_id) { // get the state and mark it as poisoned
let (mut entry, state) = match self.outbound_substreams.entry(outbound_id) {
Entry::Occupied(mut entry) => { Entry::Occupied(mut entry) => {
match std::mem::replace( let state = std::mem::replace(
&mut entry.get_mut().0, &mut entry.get_mut().state,
OutboundSubstreamState::Poisoned, OutboundSubstreamState::Poisoned,
) { );
(entry, state)
}
Entry::Vacant(_) => unreachable!(),
};
match state {
OutboundSubstreamState::RequestPendingResponse { OutboundSubstreamState::RequestPendingResponse {
mut substream, mut substream,
request, request,
} => match substream.poll_next_unpin(cx) { } => match substream.poll_next_unpin(cx) {
Poll::Ready(Some(Ok(response))) => { Poll::Ready(Some(Ok(response))) => {
if request.multiple_responses() && !response.is_error() { if request.expected_responses() > 1 && !response.is_error() {
let substream_entry = entry.get_mut(); let substream_entry = entry.get_mut();
let delay_key = &substream_entry.1; let delay_key = &substream_entry.delay_key;
// chunks left after this one // chunks left after this one
let remaining_chunks = substream_entry let remaining_chunks = substream_entry
.3 .remaining_chunks
.map(|count| count.saturating_sub(1)) .map(|count| count.saturating_sub(1))
.unwrap_or_else(|| 0); .unwrap_or_else(|| 0);
if remaining_chunks == 0 { if remaining_chunks == 0 {
// this is the last expected message, close the stream as all expected chunks have been received // this is the last expected message, close the stream as all expected chunks have been received
substream_entry.0 = substream_entry.state = OutboundSubstreamState::Closing(substream);
OutboundSubstreamState::Closing(substream);
} else { } else {
// If the response chunk was expected update the remaining number of chunks expected and reset the Timeout // If the response chunk was expected update the remaining number of chunks expected and reset the Timeout
substream_entry.0 = substream_entry.state =
OutboundSubstreamState::RequestPendingResponse { OutboundSubstreamState::RequestPendingResponse {
substream, substream,
request, request,
}; };
substream_entry.3 = Some(remaining_chunks); substream_entry.remaining_chunks = Some(remaining_chunks);
self.outbound_substreams_delay.reset( self.outbound_substreams_delay
delay_key, .reset(delay_key, Duration::from_secs(RESPONSE_TIMEOUT));
Duration::from_secs(RESPONSE_TIMEOUT),
);
} }
} else { } else {
// either this is a single response request or we received an // either this is a single response request or we received an
// error // error only expect a single response, close the stream
// only expect a single response, close the stream entry.get_mut().state = OutboundSubstreamState::Closing(substream);
entry.get_mut().0 = OutboundSubstreamState::Closing(substream);
} }
return Poll::Ready(ProtocolsHandlerEvent::Custom( // Check what type of response we got and report it accordingly
RPCEvent::Response(request_id, response), let id = entry.get().req_id;
)); let proto = entry.get().proto;
let received = match response {
RPCCodedResponse::StreamTermination(t) => {
Ok(RPCReceived::EndOfStream(id, t))
}
RPCCodedResponse::Success(resp) => Ok(RPCReceived::Response(id, resp)),
RPCCodedResponse::InvalidRequest(ref r)
| RPCCodedResponse::ServerError(ref r)
| RPCCodedResponse::Unknown(ref r) => {
let code = response.error_code().expect(
"Response indicating and error should map to an error code",
);
Err(HandlerErr::Outbound {
id,
proto,
error: RPCError::ErrorResponse(code, r.clone()),
})
}
};
return Poll::Ready(ProtocolsHandlerEvent::Custom(received));
} }
Poll::Ready(None) => { Poll::Ready(None) => {
// stream closed // stream closed
@ -851,55 +930,52 @@ where
// else report the stream terminating only. // else report the stream terminating only.
//trace!(self.log, "RPC Response - stream closed by remote"); //trace!(self.log, "RPC Response - stream closed by remote");
// drop the stream // drop the stream
let delay_key = &entry.get().1; let delay_key = &entry.get().delay_key;
let request_id = *&entry.get().req_id;
self.outbound_substreams_delay.remove(delay_key); self.outbound_substreams_delay.remove(delay_key);
entry.remove_entry(); entry.remove_entry();
self.update_keep_alive(); self.update_keep_alive();
// notify the application error // notify the application error
if request.multiple_responses() { if request.expected_responses() > 1 {
// return an end of stream result // return an end of stream result
return Poll::Ready(ProtocolsHandlerEvent::Custom( return Poll::Ready(ProtocolsHandlerEvent::Custom(Ok(
RPCEvent::Response( RPCReceived::EndOfStream(request_id, request.stream_termination()),
request_id, )));
RPCCodedResponse::StreamTermination( }
request.stream_termination(),
), // else we return an error, stream should not have closed early.
), let outbound_err = HandlerErr::Outbound {
)); id: request_id,
} // else we return an error, stream should not have closed early. proto: request.protocol(),
return Poll::Ready(ProtocolsHandlerEvent::Custom( error: RPCError::IncompleteStream,
RPCEvent::Error( };
request_id, return Poll::Ready(ProtocolsHandlerEvent::Custom(Err(outbound_err)));
request.protocol(),
RPCError::IncompleteStream,
),
));
} }
Poll::Pending => { Poll::Pending => {
entry.get_mut().0 = OutboundSubstreamState::RequestPendingResponse { entry.get_mut().state =
substream, OutboundSubstreamState::RequestPendingResponse { substream, request }
request,
}
} }
Poll::Ready(Some(Err(e))) => { Poll::Ready(Some(Err(e))) => {
// drop the stream // drop the stream
let delay_key = &entry.get().1; let delay_key = &entry.get().delay_key;
self.outbound_substreams_delay.remove(delay_key); self.outbound_substreams_delay.remove(delay_key);
let protocol = entry.get().2; let outbound_err = HandlerErr::Outbound {
id: entry.get().req_id,
proto: entry.get().proto,
error: e,
};
entry.remove_entry(); entry.remove_entry();
self.update_keep_alive(); self.update_keep_alive();
return Poll::Ready(ProtocolsHandlerEvent::Custom( return Poll::Ready(ProtocolsHandlerEvent::Custom(Err(outbound_err)));
RPCEvent::Error(request_id, protocol, e),
));
} }
}, },
OutboundSubstreamState::Closing(mut substream) => { OutboundSubstreamState::Closing(mut substream) => {
match Sink::poll_close(Pin::new(&mut substream), cx) { match Sink::poll_close(Pin::new(&mut substream), cx) {
Poll::Ready(_) => { Poll::Ready(_) => {
// drop the stream and its corresponding timeout // drop the stream and its corresponding timeout
let delay_key = &entry.get().1; let delay_key = &entry.get().delay_key;
let protocol = entry.get().2; let protocol = entry.get().proto;
let request_id = entry.get().req_id;
self.outbound_substreams_delay.remove(delay_key); self.outbound_substreams_delay.remove(delay_key);
entry.remove_entry(); entry.remove_entry();
self.update_keep_alive(); self.update_keep_alive();
@ -910,32 +986,20 @@ where
// continue sending responses beyond what we would expect. Here // continue sending responses beyond what we would expect. Here
// we simply terminate the stream and report a stream // we simply terminate the stream and report a stream
// termination to the application // termination to the application
match protocol { let termination = match protocol {
Protocol::BlocksByRange => { Protocol::BlocksByRange => Some(ResponseTermination::BlocksByRange),
return Poll::Ready(ProtocolsHandlerEvent::Custom( Protocol::BlocksByRoot => Some(ResponseTermination::BlocksByRoot),
RPCEvent::Response( _ => None, // all other protocols are do not have multiple responses and we do not inform the user, we simply drop the stream.
request_id, };
RPCCodedResponse::StreamTermination(
ResponseTermination::BlocksByRange, if let Some(termination) = termination {
), return Poll::Ready(ProtocolsHandlerEvent::Custom(Ok(
), RPCReceived::EndOfStream(request_id, termination),
)); )));
}
Protocol::BlocksByRoot => {
return Poll::Ready(ProtocolsHandlerEvent::Custom(
RPCEvent::Response(
request_id,
RPCCodedResponse::StreamTermination(
ResponseTermination::BlocksByRoot,
),
),
));
}
_ => {} // all other protocols are do not have multiple responses and we do not inform the user, we simply drop the stream.
} }
} }
Poll::Pending => { Poll::Pending => {
entry.get_mut().0 = OutboundSubstreamState::Closing(substream); entry.get_mut().state = OutboundSubstreamState::Closing(substream);
} }
} }
} }
@ -945,9 +1009,6 @@ where
} }
} }
} }
Entry::Vacant(_) => unreachable!(),
}
}
// establish outbound substreams // establish outbound substreams
if !self.dial_queue.is_empty() && self.dial_negotiated < self.max_dial_negotiated { if !self.dial_queue.is_empty() && self.dial_negotiated < self.max_dial_negotiated {
@ -980,7 +1041,7 @@ fn apply_queued_responses<TSpec: EthSpec>(
InboundSubstreamState::Closing(substream) InboundSubstreamState::Closing(substream)
} }
chunk => InboundSubstreamState::ResponsePendingSend { chunk => InboundSubstreamState::ResponsePendingSend {
substream: substream, substream,
message: chunk, message: chunk,
closing: false, closing: false,
}, },
@ -992,3 +1053,14 @@ fn apply_queued_responses<TSpec: EthSpec>(
} }
} }
} }
impl slog::Value for SubstreamId {
fn serialize(
&self,
record: &slog::Record,
key: slog::Key,
serializer: &mut dyn slog::Serializer,
) -> slog::Result {
slog::Value::serialize(&self.0, record, key, serializer)
}
}

View File

@ -9,7 +9,16 @@ use types::{Epoch, EthSpec, Hash256, SignedBeaconBlock, Slot};
/* Requests */ /* Requests */
pub type RequestId = usize; /// Identifier of a request.
///
// NOTE: The handler stores the `RequestId` to inform back of responses and errors, but it's execution
// is independent of the contents on this type.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum RequestId {
Router,
Sync(usize),
Behaviour,
}
/// The STATUS request/response handshake message. /// The STATUS request/response handshake message.
#[derive(Encode, Decode, Clone, Debug, PartialEq)] #[derive(Encode, Decode, Clone, Debug, PartialEq)]
@ -194,7 +203,7 @@ pub enum RPCCodedResponse<T: EthSpec> {
} }
/// The code assigned to an erroneous `RPCResponse`. /// The code assigned to an erroneous `RPCResponse`.
#[derive(Debug, Clone)] #[derive(Debug, Clone, Copy)]
pub enum RPCResponseErrorCode { pub enum RPCResponseErrorCode {
InvalidRequest, InvalidRequest,
ServerError, ServerError,
@ -230,6 +239,15 @@ impl<T: EthSpec> RPCCodedResponse<T> {
} }
} }
/// Builds an RPCCodedResponse from a response code and an ErrorMessage
pub fn from_error_code(response_code: RPCResponseErrorCode, err: String) -> Self {
match response_code {
RPCResponseErrorCode::InvalidRequest => RPCCodedResponse::InvalidRequest(err),
RPCResponseErrorCode::ServerError => RPCCodedResponse::ServerError(err),
RPCResponseErrorCode::Unknown => RPCCodedResponse::Unknown(err),
}
}
/// Specifies which response allows for multiple chunks for the stream handler. /// Specifies which response allows for multiple chunks for the stream handler.
pub fn multiple_responses(&self) -> bool { pub fn multiple_responses(&self) -> bool {
match self { match self {
@ -333,3 +351,18 @@ impl std::fmt::Display for BlocksByRangeRequest {
) )
} }
} }
impl slog::Value for RequestId {
fn serialize(
&self,
record: &slog::Record,
key: slog::Key,
serializer: &mut dyn slog::Serializer,
) -> slog::Result {
match self {
RequestId::Behaviour => slog::Value::serialize("Behaviour", record, key, serializer),
RequestId::Router => slog::Value::serialize("Router", record, key, serializer),
RequestId::Sync(ref id) => slog::Value::serialize(id, record, key, serializer),
}
}
}

View File

@ -11,34 +11,69 @@ use libp2p::swarm::{
PollParameters, SubstreamProtocol, PollParameters, SubstreamProtocol,
}; };
use libp2p::{Multiaddr, PeerId}; use libp2p::{Multiaddr, PeerId};
pub use methods::{
MetaData, RPCCodedResponse, RPCResponse, RPCResponseErrorCode, RequestId, ResponseTermination,
StatusMessage,
};
pub use protocol::{Protocol, RPCError, RPCProtocol, RPCRequest};
use slog::{debug, o}; use slog::{debug, o};
use std::marker::PhantomData; use std::marker::PhantomData;
use std::task::{Context, Poll}; use std::task::{Context, Poll};
use std::time::Duration; use std::time::Duration;
use types::EthSpec; use types::EthSpec;
pub(crate) use handler::HandlerErr;
pub(crate) use methods::{MetaData, Ping, RPCCodedResponse, RPCResponse};
pub(crate) use protocol::{RPCProtocol, RPCRequest};
pub use handler::SubstreamId;
pub use methods::{
BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, RPCResponseErrorCode, RequestId,
ResponseTermination, StatusMessage,
};
pub use protocol::{Protocol, RPCError};
pub(crate) mod codec; pub(crate) mod codec;
mod handler; mod handler;
pub mod methods; pub mod methods;
mod protocol; mod protocol;
/// The return type used in the behaviour and the resultant event from the protocols handler. /// RPC events sent from Lighthouse.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum RPCEvent<T: EthSpec> { pub enum RPCSend<T: EthSpec> {
/// An inbound/outbound request for RPC protocol. The first parameter is a sequential /// A request sent from Lighthouse.
/// id which tracks an awaiting substream for the response. ///
/// The `RequestId` is given by the application making the request. These
/// go over *outbound* connections.
Request(RequestId, RPCRequest<T>), Request(RequestId, RPCRequest<T>),
/// A response that is being sent or has been received from the RPC protocol. The first parameter returns /// A response sent from Lighthouse.
/// that which was sent with the corresponding request, the second is a single chunk of a ///
/// response. /// The `SubstreamId` must correspond to the RPC-given ID of the original request received from the
Response(RequestId, RPCCodedResponse<T>), /// peer. The second parameter is a single chunk of a response. These go over *inbound*
/// An Error occurred. /// connections.
Error(RequestId, Protocol, RPCError), Response(SubstreamId, RPCCodedResponse<T>),
}
/// RPC events received from outside Lighthouse.
#[derive(Debug, Clone)]
pub enum RPCReceived<T: EthSpec> {
/// A request received from the outside.
///
/// The `SubstreamId` is given by the `RPCHandler` as it identifies this request with the
/// *inbound* substream over which it is managed.
Request(SubstreamId, RPCRequest<T>),
/// A response received from the outside.
///
/// The `RequestId` corresponds to the application given ID of the original request sent to the
/// peer. The second parameter is a single chunk of a response. These go over *outbound*
/// connections.
Response(RequestId, RPCResponse<T>),
/// Marks a request as completed
EndOfStream(RequestId, ResponseTermination),
}
impl<T: EthSpec> std::fmt::Display for RPCSend<T> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
RPCSend::Request(id, req) => write!(f, "RPC Request(id: {:?}, {})", id, req),
RPCSend::Response(id, res) => write!(f, "RPC Response(id: {:?}, {})", id, res),
}
}
} }
/// Messages sent to the user from the RPC protocol. /// Messages sent to the user from the RPC protocol.
@ -46,38 +81,14 @@ pub struct RPCMessage<TSpec: EthSpec> {
/// The peer that sent the message. /// The peer that sent the message.
pub peer_id: PeerId, pub peer_id: PeerId,
/// The message that was sent. /// The message that was sent.
pub event: RPCEvent<TSpec>, pub event: <RPCHandler<TSpec> as ProtocolsHandler>::OutEvent,
}
impl<T: EthSpec> RPCEvent<T> {
pub fn id(&self) -> usize {
match *self {
RPCEvent::Request(id, _) => id,
RPCEvent::Response(id, _) => id,
RPCEvent::Error(id, _, _) => id,
}
}
}
impl<T: EthSpec> std::fmt::Display for RPCEvent<T> {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
match self {
RPCEvent::Request(id, req) => write!(f, "RPC Request(id: {}, {})", id, req),
RPCEvent::Response(id, res) => write!(f, "RPC Response(id: {}, {})", id, res),
RPCEvent::Error(id, prot, err) => write!(
f,
"RPC Error(id: {}, protocol: {:?} error: {:?})",
id, prot, err
),
}
}
} }
/// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level /// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level
/// logic. /// logic.
pub struct RPC<TSpec: EthSpec> { pub struct RPC<TSpec: EthSpec> {
/// Queue of events to processed. /// Queue of events to be processed.
events: Vec<NetworkBehaviourAction<RPCEvent<TSpec>, RPCMessage<TSpec>>>, events: Vec<NetworkBehaviourAction<RPCSend<TSpec>, RPCMessage<TSpec>>>,
/// Slog logger for RPC behaviour. /// Slog logger for RPC behaviour.
log: slog::Logger, log: slog::Logger,
} }
@ -94,11 +105,11 @@ impl<TSpec: EthSpec> RPC<TSpec> {
/// Submits an RPC request. /// Submits an RPC request.
/// ///
/// The peer must be connected for this to succeed. /// The peer must be connected for this to succeed.
pub fn send_rpc(&mut self, peer_id: PeerId, rpc_event: RPCEvent<TSpec>) { pub fn send_rpc(&mut self, peer_id: PeerId, event: RPCSend<TSpec>) {
self.events.push(NetworkBehaviourAction::NotifyHandler { self.events.push(NetworkBehaviourAction::NotifyHandler {
peer_id, peer_id,
handler: NotifyHandler::Any, handler: NotifyHandler::Any,
event: rpc_event, event,
}); });
} }
} }
@ -129,8 +140,7 @@ where
fn inject_connected(&mut self, peer_id: &PeerId) { fn inject_connected(&mut self, peer_id: &PeerId) {
// find the peer's meta-data // find the peer's meta-data
debug!(self.log, "Requesting new peer's metadata"; "peer_id" => format!("{}",peer_id)); debug!(self.log, "Requesting new peer's metadata"; "peer_id" => format!("{}",peer_id));
let rpc_event = let rpc_event = RPCSend::Request(RequestId::Behaviour, RPCRequest::MetaData(PhantomData));
RPCEvent::Request(RequestId::from(0usize), RPCRequest::MetaData(PhantomData));
self.events.push(NetworkBehaviourAction::NotifyHandler { self.events.push(NetworkBehaviourAction::NotifyHandler {
peer_id: peer_id.clone(), peer_id: peer_id.clone(),
handler: NotifyHandler::Any, handler: NotifyHandler::Any,
@ -158,14 +168,14 @@ where
fn inject_event( fn inject_event(
&mut self, &mut self,
source: PeerId, peer_id: PeerId,
_: ConnectionId, _: ConnectionId,
event: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent, event: <Self::ProtocolsHandler as ProtocolsHandler>::OutEvent,
) { ) {
// send the event to the user // send the event to the user
self.events self.events
.push(NetworkBehaviourAction::GenerateEvent(RPCMessage { .push(NetworkBehaviourAction::GenerateEvent(RPCMessage {
peer_id: source, peer_id,
event, event,
})); }));
} }

View File

@ -290,32 +290,19 @@ impl<TSpec: EthSpec> RPCRequest<TSpec> {
/* These functions are used in the handler for stream management */ /* These functions are used in the handler for stream management */
/// This specifies whether a stream should remain open and await a response, given a request. /// Number of responses expected for this request.
/// A GOODBYE request has no response. pub fn expected_responses(&self) -> usize {
pub fn expect_response(&self) -> bool {
match self { match self {
RPCRequest::Status(_) => true, RPCRequest::Status(_) => 1,
RPCRequest::Goodbye(_) => false, RPCRequest::Goodbye(_) => 0,
RPCRequest::BlocksByRange(_) => true, RPCRequest::BlocksByRange(req) => req.count as usize,
RPCRequest::BlocksByRoot(_) => true, RPCRequest::BlocksByRoot(req) => req.block_roots.len(),
RPCRequest::Ping(_) => true, RPCRequest::Ping(_) => 1,
RPCRequest::MetaData(_) => true, RPCRequest::MetaData(_) => 1,
}
}
/// Returns which methods expect multiple responses from the stream. If this is false and
/// the stream terminates, an error is given.
pub fn multiple_responses(&self) -> bool {
match self {
RPCRequest::Status(_) => false,
RPCRequest::Goodbye(_) => false,
RPCRequest::BlocksByRange(_) => true,
RPCRequest::BlocksByRoot(_) => true,
RPCRequest::Ping(_) => false,
RPCRequest::MetaData(_) => false,
} }
} }
/// Gives the corresponding `Protocol` to this request.
pub fn protocol(&self) -> Protocol { pub fn protocol(&self) -> Protocol {
match self { match self {
RPCRequest::Status(_) => Protocol::Status, RPCRequest::Status(_) => Protocol::Status,
@ -390,7 +377,7 @@ pub enum RPCError {
/// IO Error. /// IO Error.
IoError(String), IoError(String),
/// The peer returned a valid response but the response indicated an error. /// The peer returned a valid response but the response indicated an error.
ErrorResponse(RPCResponseErrorCode), ErrorResponse(RPCResponseErrorCode, String),
/// Timed out waiting for a response. /// Timed out waiting for a response.
StreamTimeout, StreamTimeout,
/// Peer does not support the protocol. /// Peer does not support the protocol.
@ -430,7 +417,11 @@ impl std::fmt::Display for RPCError {
RPCError::SSZDecodeError(ref err) => write!(f, "Error while decoding ssz: {:?}", err), RPCError::SSZDecodeError(ref err) => write!(f, "Error while decoding ssz: {:?}", err),
RPCError::InvalidData => write!(f, "Peer sent unexpected data"), RPCError::InvalidData => write!(f, "Peer sent unexpected data"),
RPCError::IoError(ref err) => write!(f, "IO Error: {}", err), RPCError::IoError(ref err) => write!(f, "IO Error: {}", err),
RPCError::ErrorResponse(ref code) => write!(f, "RPC response was an error: {}", code), RPCError::ErrorResponse(ref code, ref reason) => write!(
f,
"RPC response was an error: {} with reason: {}",
code, reason
),
RPCError::StreamTimeout => write!(f, "Stream Timeout"), RPCError::StreamTimeout => write!(f, "Stream Timeout"),
RPCError::UnsupportedProtocol => write!(f, "Peer does not support the protocol"), RPCError::UnsupportedProtocol => write!(f, "Peer does not support the protocol"),
RPCError::IncompleteStream => write!(f, "Stream ended unexpectedly"), RPCError::IncompleteStream => write!(f, "Stream ended unexpectedly"),
@ -451,7 +442,7 @@ impl std::error::Error for RPCError {
RPCError::IncompleteStream => None, RPCError::IncompleteStream => None,
RPCError::InvalidData => None, RPCError::InvalidData => None,
RPCError::InternalError(_) => None, RPCError::InternalError(_) => None,
RPCError::ErrorResponse(_) => None, RPCError::ErrorResponse(_, _) => None,
RPCError::NegotiationTimeout => None, RPCError::NegotiationTimeout => None,
} }
} }

View File

@ -1,6 +1,7 @@
use crate::behaviour::{Behaviour, BehaviourEvent}; use crate::behaviour::{Behaviour, BehaviourEvent, Request, Response};
use crate::discovery::enr; use crate::discovery::enr;
use crate::multiaddr::Protocol; use crate::multiaddr::Protocol;
use crate::rpc::{RPCResponseErrorCode, RequestId, SubstreamId};
use crate::types::{error, GossipKind}; use crate::types::{error, GossipKind};
use crate::EnrExt; use crate::EnrExt;
use crate::{NetworkConfig, NetworkGlobals}; use crate::{NetworkConfig, NetworkGlobals};
@ -229,6 +230,34 @@ impl<TSpec: EthSpec> Service<TSpec> {
self.peer_ban_timeout.insert(peer_id, timeout); self.peer_ban_timeout.insert(peer_id, timeout);
} }
/// Sends a request to a peer, with a given Id.
pub fn send_request(&mut self, peer_id: PeerId, request_id: RequestId, request: Request) {
self.swarm.send_request(peer_id, request_id, request);
}
/// Informs the peer that their request failed.
pub fn respond_with_error(
&mut self,
peer_id: PeerId,
stream_id: SubstreamId,
error: RPCResponseErrorCode,
reason: String,
) {
self.swarm
._send_error_reponse(peer_id, stream_id, error, reason);
}
/// Sends a response to a peer's request.
pub fn send_response(
&mut self,
peer_id: PeerId,
stream_id: SubstreamId,
response: Response<TSpec>,
) {
self.swarm
.send_successful_response(peer_id, stream_id, response);
}
pub async fn next_event(&mut self) -> Libp2pEvent<TSpec> { pub async fn next_event(&mut self) -> Libp2pEvent<TSpec> {
loop { loop {
tokio::select! { tokio::select! {
@ -242,7 +271,7 @@ impl<TSpec: EthSpec> Service<TSpec> {
endpoint, endpoint,
num_established, num_established,
} => { } => {
debug!(self.log, "Connection established"; "peer_id"=> peer_id.to_string(), "connections" => num_established.get()); debug!(self.log, "Connection established"; "peer_id" => peer_id.to_string(), "connections" => num_established.get());
// if this is the first connection inform the network layer a new connection // if this is the first connection inform the network layer a new connection
// has been established and update the db // has been established and update the db
if num_established.get() == 1 { if num_established.get() == 1 {
@ -347,8 +376,8 @@ impl<TSpec: EthSpec> Service<TSpec> {
} }
} }
/// The implementation supports TCP/IP, WebSockets over TCP/IP, noise/secio as the encryption layer, and /// The implementation supports TCP/IP, WebSockets over TCP/IP, noise/secio as the encryption
/// mplex or yamux as the multiplexing layer. /// layer, and mplex or yamux as the multiplexing layer.
fn build_transport( fn build_transport(
local_private_key: Keypair, local_private_key: Keypair,
) -> Result<Boxed<(PeerId, StreamMuxerBox), Error>, Error> { ) -> Result<Boxed<(PeerId, StreamMuxerBox), Error>, Error> {

View File

@ -1,7 +1,6 @@
#![cfg(test)] #![cfg(test)]
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::rpc::*; use eth2_libp2p::{BehaviourEvent, Libp2pEvent, Request, Response};
use eth2_libp2p::{BehaviourEvent, Libp2pEvent, RPCEvent};
use slog::{debug, warn, Level}; use slog::{debug, warn, Level};
use std::time::Duration; use std::time::Duration;
use tokio::time::delay_for; use tokio::time::delay_for;
@ -26,7 +25,7 @@ async fn test_status_rpc() {
let (mut sender, mut receiver) = common::build_node_pair(&log).await; let (mut sender, mut receiver) = common::build_node_pair(&log).await;
// Dummy STATUS RPC message // Dummy STATUS RPC message
let rpc_request = RPCRequest::Status(StatusMessage { let rpc_request = Request::Status(StatusMessage {
fork_digest: [0; 4], fork_digest: [0; 4],
finalized_root: Hash256::from_low_u64_be(0), finalized_root: Hash256::from_low_u64_be(0),
finalized_epoch: Epoch::new(1), finalized_epoch: Epoch::new(1),
@ -35,7 +34,7 @@ async fn test_status_rpc() {
}); });
// Dummy STATUS RPC message // Dummy STATUS RPC message
let rpc_response = RPCResponse::Status(StatusMessage { let rpc_response = Response::Status(StatusMessage {
fork_digest: [0; 4], fork_digest: [0; 4],
finalized_root: Hash256::from_low_u64_be(0), finalized_root: Hash256::from_low_u64_be(0),
finalized_epoch: Epoch::new(1), finalized_epoch: Epoch::new(1),
@ -52,26 +51,19 @@ async fn test_status_rpc() {
debug!(log, "Sending RPC"); debug!(log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(10, rpc_request.clone())); .send_request(peer_id, RequestId::Sync(10), rpc_request.clone());
} }
Libp2pEvent::Behaviour(BehaviourEvent::RPC(_, event)) => match event { Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
peer_id: _,
id: RequestId::Sync(10),
response,
}) => {
// Should receive the RPC response // Should receive the RPC response
RPCEvent::Response(id, response @ RPCCodedResponse::Success(_)) => {
if id == 10 {
debug!(log, "Sender Received"); debug!(log, "Sender Received");
let response = {
match response {
RPCCodedResponse::Success(r) => r,
_ => unreachable!(),
}
};
assert_eq!(response, rpc_response.clone()); assert_eq!(response, rpc_response.clone());
debug!(log, "Sender Completed"); debug!(log, "Sender Completed");
return; return;
} }
}
_ => {} // Ignore other RPC messages
},
_ => {} _ => {}
} }
} }
@ -81,23 +73,17 @@ async fn test_status_rpc() {
let receiver_future = async { let receiver_future = async {
loop { loop {
match receiver.next_event().await { match receiver.next_event().await {
Libp2pEvent::Behaviour(BehaviourEvent::RPC(peer_id, event)) => { Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
match event { peer_id,
// Should receive sent RPC request id,
RPCEvent::Request(id, request) => { request,
}) => {
if request == rpc_request { if request == rpc_request {
// send the response // send the response
debug!(log, "Receiver Received"); debug!(log, "Receiver Received");
receiver.swarm.send_rpc( receiver
peer_id, .swarm
RPCEvent::Response( .send_successful_response(peer_id, id, rpc_response.clone());
id,
RPCCodedResponse::Success(rpc_response.clone()),
),
);
}
}
_ => {} // Ignore other RPC requests
} }
} }
_ => {} // Ignore other events _ => {} // Ignore other events
@ -129,7 +115,7 @@ async fn test_blocks_by_range_chunked_rpc() {
let (mut sender, mut receiver) = common::build_node_pair(&log).await; let (mut sender, mut receiver) = common::build_node_pair(&log).await;
// BlocksByRange Request // BlocksByRange Request
let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest { let rpc_request = Request::BlocksByRange(BlocksByRangeRequest {
start_slot: 0, start_slot: 0,
count: messages_to_send, count: messages_to_send,
step: 0, step: 0,
@ -142,7 +128,7 @@ async fn test_blocks_by_range_chunked_rpc() {
message: empty_block, message: empty_block,
signature: Signature::empty_signature(), signature: Signature::empty_signature(),
}; };
let rpc_response = RPCResponse::BlocksByRange(Box::new(empty_signed)); let rpc_response = Response::BlocksByRange(Some(Box::new(empty_signed)));
// keep count of the number of messages received // keep count of the number of messages received
let mut messages_received = 0; let mut messages_received = 0;
@ -155,20 +141,21 @@ async fn test_blocks_by_range_chunked_rpc() {
debug!(log, "Sending RPC"); debug!(log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(10, rpc_request.clone())); .send_request(peer_id, RequestId::Sync(10), rpc_request.clone());
} }
Libp2pEvent::Behaviour(BehaviourEvent::RPC(_, event)) => match event { Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
// Should receive the RPC response peer_id: _,
RPCEvent::Response(id, response) => { id: RequestId::Sync(10),
if id == 10 { response,
}) => {
warn!(log, "Sender received a response"); warn!(log, "Sender received a response");
match response { match response {
RPCCodedResponse::Success(res) => { Response::BlocksByRange(Some(_)) => {
assert_eq!(res, rpc_response.clone()); assert_eq!(response, rpc_response.clone());
messages_received += 1; messages_received += 1;
warn!(log, "Chunk received"); warn!(log, "Chunk received");
} }
RPCCodedResponse::StreamTermination(_) => { Response::BlocksByRange(None) => {
// should be exactly 10 messages before terminating // should be exactly 10 messages before terminating
assert_eq!(messages_received, messages_to_send); assert_eq!(messages_received, messages_to_send);
// end the test // end the test
@ -177,9 +164,6 @@ async fn test_blocks_by_range_chunked_rpc() {
_ => panic!("Invalid RPC received"), _ => panic!("Invalid RPC received"),
} }
} }
}
_ => {} // Ignore other RPC messages
},
_ => {} // Ignore other behaviour events _ => {} // Ignore other behaviour events
} }
} }
@ -189,41 +173,32 @@ async fn test_blocks_by_range_chunked_rpc() {
let receiver_future = async { let receiver_future = async {
loop { loop {
match receiver.next_event().await { match receiver.next_event().await {
Libp2pEvent::Behaviour(BehaviourEvent::RPC(peer_id, event)) => { Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
match event { peer_id,
// Should receive sent RPC request id,
RPCEvent::Request(id, request) => { request,
}) => {
if request == rpc_request { if request == rpc_request {
// send the response // send the response
warn!(log, "Receiver got request"); warn!(log, "Receiver got request");
for _ in 1..=messages_to_send { for _ in 1..=messages_to_send {
receiver.swarm.send_rpc( receiver.swarm.send_successful_response(
peer_id.clone(), peer_id.clone(),
RPCEvent::Response(
id, id,
RPCCodedResponse::Success(rpc_response.clone()), rpc_response.clone(),
),
); );
} }
// send the stream termination // send the stream termination
receiver.swarm.send_rpc( receiver.swarm.send_successful_response(
peer_id, peer_id,
RPCEvent::Response(
id, id,
RPCCodedResponse::StreamTermination( Response::BlocksByRange(None),
ResponseTermination::BlocksByRange,
),
),
); );
} }
} }
_ => {} // Ignore other events _ => {} // Ignore other events
} }
} }
_ => {} // Ignore other events
}
}
}; };
tokio::select! { tokio::select! {
@ -251,7 +226,7 @@ async fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
let (mut sender, mut receiver) = common::build_node_pair(&log).await; let (mut sender, mut receiver) = common::build_node_pair(&log).await;
// BlocksByRange Request // BlocksByRange Request
let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest { let rpc_request = Request::BlocksByRange(BlocksByRangeRequest {
start_slot: 0, start_slot: 0,
count: messages_to_send, count: messages_to_send,
step: 0, step: 0,
@ -264,7 +239,7 @@ async fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
message: empty_block, message: empty_block,
signature: Signature::empty_signature(), signature: Signature::empty_signature(),
}; };
let rpc_response = RPCResponse::BlocksByRange(Box::new(empty_signed)); let rpc_response = Response::BlocksByRange(Some(Box::new(empty_signed)));
// keep count of the number of messages received // keep count of the number of messages received
let mut messages_received: u64 = 0; let mut messages_received: u64 = 0;
@ -277,28 +252,29 @@ async fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
debug!(log, "Sending RPC"); debug!(log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(10, rpc_request.clone())); .send_request(peer_id, RequestId::Sync(10), rpc_request.clone());
} }
Libp2pEvent::Behaviour(BehaviourEvent::RPC(_, event)) => match event { Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
peer_id: _,
id: RequestId::Sync(10),
response,
}) =>
// Should receive the RPC response // Should receive the RPC response
RPCEvent::Response(id, response) => { {
if id == 10 {
debug!(log, "Sender received a response"); debug!(log, "Sender received a response");
match response { match response {
RPCCodedResponse::Success(res) => { Response::BlocksByRange(Some(_)) => {
assert_eq!(res, rpc_response.clone()); assert_eq!(response, rpc_response.clone());
messages_received += 1; messages_received += 1;
} }
RPCCodedResponse::StreamTermination(_) => { Response::BlocksByRange(None) => {
// should be exactly 10 messages, as requested // should be exactly 10 messages, as requested
assert_eq!(messages_received, messages_to_send); assert_eq!(messages_received, messages_to_send);
} }
_ => panic!("Invalid RPC received"), _ => panic!("Invalid RPC received"),
} }
} }
}
_ => {} // Ignore other RPC messages
},
_ => {} // Ignore other behaviour events _ => {} // Ignore other behaviour events
} }
} }
@ -320,21 +296,17 @@ async fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
.await .await
{ {
futures::future::Either::Left(( futures::future::Either::Left((
Libp2pEvent::Behaviour(BehaviourEvent::RPC(peer_id, event)), Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
peer_id,
id,
request,
}),
_, _,
)) => { )) => {
match event {
// Should receive sent RPC request
RPCEvent::Request(id, request) => {
if request == rpc_request { if request == rpc_request {
// send the response // send the response
warn!(log, "Receiver got request"); warn!(log, "Receiver got request");
message_info = Some((peer_id, id)); message_info = Some((peer_id, id));
} else {
continue;
}
}
_ => continue, // Ignore other events, don't send messages until ready
} }
} }
futures::future::Either::Right((_, _)) => {} // The timeout hit, send messages if required futures::future::Either::Right((_, _)) => {} // The timeout hit, send messages if required
@ -344,12 +316,11 @@ async fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
// if we need to send messages send them here. This will happen after a delay // if we need to send messages send them here. This will happen after a delay
if message_info.is_some() { if message_info.is_some() {
messages_sent += 1; messages_sent += 1;
receiver.swarm.send_rpc( let (peer_id, stream_id) = message_info.as_ref().unwrap();
message_info.as_ref().unwrap().0.clone(), receiver.swarm.send_successful_response(
RPCEvent::Response( peer_id.clone(),
message_info.as_ref().unwrap().1.clone(), stream_id.clone(),
RPCCodedResponse::Success(rpc_response.clone()), rpc_response.clone(),
),
); );
debug!(log, "Sending message {}", messages_sent); debug!(log, "Sending message {}", messages_sent);
if messages_sent == messages_to_send + extra_messages_to_send { if messages_sent == messages_to_send + extra_messages_to_send {
@ -382,7 +353,7 @@ async fn test_blocks_by_range_single_empty_rpc() {
let (mut sender, mut receiver) = common::build_node_pair(&log).await; let (mut sender, mut receiver) = common::build_node_pair(&log).await;
// BlocksByRange Request // BlocksByRange Request
let rpc_request = RPCRequest::BlocksByRange(BlocksByRangeRequest { let rpc_request = Request::BlocksByRange(BlocksByRangeRequest {
start_slot: 0, start_slot: 0,
count: 10, count: 10,
step: 0, step: 0,
@ -395,7 +366,7 @@ async fn test_blocks_by_range_single_empty_rpc() {
message: empty_block, message: empty_block,
signature: Signature::empty_signature(), signature: Signature::empty_signature(),
}; };
let rpc_response = RPCResponse::BlocksByRange(Box::new(empty_signed)); let rpc_response = Response::BlocksByRange(Some(Box::new(empty_signed)));
let messages_to_send = 1; let messages_to_send = 1;
@ -410,30 +381,25 @@ async fn test_blocks_by_range_single_empty_rpc() {
debug!(log, "Sending RPC"); debug!(log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(10, rpc_request.clone())); .send_request(peer_id, RequestId::Sync(10), rpc_request.clone());
} }
Libp2pEvent::Behaviour(BehaviourEvent::RPC(_, event)) => match event { Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
// Should receive the RPC response peer_id: _,
RPCEvent::Response(id, response) => { id: RequestId::Sync(10),
if id == 10 { response,
warn!(log, "Sender received a response"); }) => match response {
match response { Response::BlocksByRange(Some(_)) => {
RPCCodedResponse::Success(res) => { assert_eq!(response, rpc_response.clone());
assert_eq!(res, rpc_response.clone());
messages_received += 1; messages_received += 1;
warn!(log, "Chunk received"); warn!(log, "Chunk received");
} }
RPCCodedResponse::StreamTermination(_) => { Response::BlocksByRange(None) => {
// should be exactly 10 messages before terminating // should be exactly 10 messages before terminating
assert_eq!(messages_received, messages_to_send); assert_eq!(messages_received, messages_to_send);
// end the test // end the test
return; return;
} }
_ => panic!("Invalid RPC received"), _ => panic!("Invalid RPC received"),
}
}
}
_ => {} // Ignore other RPC messages
}, },
_ => {} // Ignore other behaviour events _ => {} // Ignore other behaviour events
} }
@ -444,41 +410,33 @@ async fn test_blocks_by_range_single_empty_rpc() {
let receiver_future = async { let receiver_future = async {
loop { loop {
match receiver.next_event().await { match receiver.next_event().await {
Libp2pEvent::Behaviour(BehaviourEvent::RPC(peer_id, event)) => { Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
match event { peer_id,
// Should receive sent RPC request id,
RPCEvent::Request(id, request) => { request,
}) => {
if request == rpc_request { if request == rpc_request {
// send the response // send the response
warn!(log, "Receiver got request"); warn!(log, "Receiver got request");
for _ in 1..=messages_to_send { for _ in 1..=messages_to_send {
receiver.swarm.send_rpc( receiver.swarm.send_successful_response(
peer_id.clone(), peer_id.clone(),
RPCEvent::Response(
id, id,
RPCCodedResponse::Success(rpc_response.clone()), rpc_response.clone(),
),
); );
} }
// send the stream termination // send the stream termination
receiver.swarm.send_rpc( receiver.swarm.send_successful_response(
peer_id, peer_id,
RPCEvent::Response(
id, id,
RPCCodedResponse::StreamTermination( Response::BlocksByRange(None),
ResponseTermination::BlocksByRange,
),
),
); );
} }
} }
_ => {} // Ignore other events _ => {} // Ignore other events
} }
} }
_ => {} // Ignore other events
}
}
}; };
tokio::select! { tokio::select! {
_ = sender_future => {} _ = sender_future => {}
@ -508,7 +466,7 @@ async fn test_blocks_by_root_chunked_rpc() {
let (mut sender, mut receiver) = common::build_node_pair(&log).await; let (mut sender, mut receiver) = common::build_node_pair(&log).await;
// BlocksByRoot Request // BlocksByRoot Request
let rpc_request = RPCRequest::BlocksByRoot(BlocksByRootRequest { let rpc_request = Request::BlocksByRoot(BlocksByRootRequest {
block_roots: vec![ block_roots: vec![
Hash256::from_low_u64_be(0), Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0), Hash256::from_low_u64_be(0),
@ -522,7 +480,7 @@ async fn test_blocks_by_root_chunked_rpc() {
message: full_block, message: full_block,
signature: Signature::empty_signature(), signature: Signature::empty_signature(),
}; };
let rpc_response = RPCResponse::BlocksByRoot(Box::new(signed_full_block)); let rpc_response = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
// keep count of the number of messages received // keep count of the number of messages received
let mut messages_received = 0; let mut messages_received = 0;
@ -535,30 +493,25 @@ async fn test_blocks_by_root_chunked_rpc() {
debug!(log, "Sending RPC"); debug!(log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(10, rpc_request.clone())); .send_request(peer_id, RequestId::Sync(10), rpc_request.clone());
} }
Libp2pEvent::Behaviour(BehaviourEvent::RPC(_, event)) => match event { Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
// Should receive the RPC response peer_id: _,
RPCEvent::Response(id, response) => { id: RequestId::Sync(10),
if id == 10 { response,
debug!(log, "Sender received a response"); }) => match response {
match response { Response::BlocksByRoot(Some(_)) => {
RPCCodedResponse::Success(res) => { assert_eq!(response, rpc_response.clone());
assert_eq!(res, rpc_response.clone());
messages_received += 1; messages_received += 1;
debug!(log, "Chunk received"); debug!(log, "Chunk received");
} }
RPCCodedResponse::StreamTermination(_) => { Response::BlocksByRoot(None) => {
// should be exactly messages_to_send // should be exactly messages_to_send
assert_eq!(messages_received, messages_to_send); assert_eq!(messages_received, messages_to_send);
// end the test // end the test
return; return;
} }
_ => {} // Ignore other RPC messages _ => {} // Ignore other RPC messages
}
}
}
_ => {} // Ignore other RPC messages
}, },
_ => {} // Ignore other behaviour events _ => {} // Ignore other behaviour events
} }
@ -569,33 +522,28 @@ async fn test_blocks_by_root_chunked_rpc() {
let receiver_future = async { let receiver_future = async {
loop { loop {
match receiver.next_event().await { match receiver.next_event().await {
Libp2pEvent::Behaviour(BehaviourEvent::RPC(peer_id, event)) => { Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
match event { peer_id,
// Should receive sent RPC request id,
RPCEvent::Request(id, request) => { request,
}) => {
if request == rpc_request { if request == rpc_request {
// send the response // send the response
debug!(log, "Receiver got request"); debug!(log, "Receiver got request");
for _ in 1..=messages_to_send { for _ in 1..=messages_to_send {
receiver.swarm.send_rpc( receiver.swarm.send_successful_response(
peer_id.clone(), peer_id.clone(),
RPCEvent::Response(
id, id,
RPCCodedResponse::Success(rpc_response.clone()), rpc_response.clone(),
),
); );
debug!(log, "Sending message"); debug!(log, "Sending message");
} }
// send the stream termination // send the stream termination
receiver.swarm.send_rpc( receiver.swarm.send_successful_response(
peer_id, peer_id,
RPCEvent::Response(
id, id,
RPCCodedResponse::StreamTermination( Response::BlocksByRange(None),
ResponseTermination::BlocksByRange,
),
),
); );
debug!(log, "Send stream term"); debug!(log, "Send stream term");
} }
@ -603,9 +551,6 @@ async fn test_blocks_by_root_chunked_rpc() {
_ => {} // Ignore other events _ => {} // Ignore other events
} }
} }
_ => {} // Ignore other events
}
}
}; };
tokio::select! { tokio::select! {
_ = sender_future => {} _ = sender_future => {}
@ -633,7 +578,7 @@ async fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
let (mut sender, mut receiver) = common::build_node_pair(&log).await; let (mut sender, mut receiver) = common::build_node_pair(&log).await;
// BlocksByRoot Request // BlocksByRoot Request
let rpc_request = RPCRequest::BlocksByRoot(BlocksByRootRequest { let rpc_request = Request::BlocksByRoot(BlocksByRootRequest {
block_roots: vec![ block_roots: vec![
Hash256::from_low_u64_be(0), Hash256::from_low_u64_be(0),
Hash256::from_low_u64_be(0), Hash256::from_low_u64_be(0),
@ -654,7 +599,7 @@ async fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
message: full_block, message: full_block,
signature: Signature::empty_signature(), signature: Signature::empty_signature(),
}; };
let rpc_response = RPCResponse::BlocksByRoot(Box::new(signed_full_block)); let rpc_response = Response::BlocksByRoot(Some(Box::new(signed_full_block)));
// keep count of the number of messages received // keep count of the number of messages received
let mut messages_received = 0; let mut messages_received = 0;
@ -667,20 +612,21 @@ async fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
debug!(log, "Sending RPC"); debug!(log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(10, rpc_request.clone())); .send_request(peer_id, RequestId::Sync(10), rpc_request.clone());
} }
Libp2pEvent::Behaviour(BehaviourEvent::RPC(_, event)) => match event { Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
// Should receive the RPC response peer_id: _,
RPCEvent::Response(id, response) => { id: RequestId::Sync(10),
if id == 10 { response,
}) => {
debug!(log, "Sender received a response"); debug!(log, "Sender received a response");
match response { match response {
RPCCodedResponse::Success(res) => { Response::BlocksByRoot(Some(_)) => {
assert_eq!(res, rpc_response.clone()); assert_eq!(response, rpc_response.clone());
messages_received += 1; messages_received += 1;
debug!(log, "Chunk received"); debug!(log, "Chunk received");
} }
RPCCodedResponse::StreamTermination(_) => { Response::BlocksByRoot(None) => {
// should be exactly messages_to_send // should be exactly messages_to_send
assert_eq!(messages_received, messages_to_send); assert_eq!(messages_received, messages_to_send);
// end the test // end the test
@ -689,9 +635,6 @@ async fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
_ => {} // Ignore other RPC messages _ => {} // Ignore other RPC messages
} }
} }
}
_ => {} // Ignore other RPC messages
},
_ => {} // Ignore other behaviour events _ => {} // Ignore other behaviour events
} }
} }
@ -713,21 +656,17 @@ async fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
.await .await
{ {
futures::future::Either::Left(( futures::future::Either::Left((
Libp2pEvent::Behaviour(BehaviourEvent::RPC(peer_id, event)), Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
peer_id,
id,
request,
}),
_, _,
)) => { )) => {
match event {
// Should receive sent RPC request
RPCEvent::Request(id, request) => {
if request == rpc_request { if request == rpc_request {
// send the response // send the response
warn!(log, "Receiver got request"); warn!(log, "Receiver got request");
message_info = Some((peer_id, id)); message_info = Some((peer_id, id));
} else {
continue;
}
}
_ => continue, // Ignore other events, don't send messages until ready
} }
} }
futures::future::Either::Right((_, _)) => {} // The timeout hit, send messages if required futures::future::Either::Right((_, _)) => {} // The timeout hit, send messages if required
@ -737,12 +676,11 @@ async fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
// if we need to send messages send them here. This will happen after a delay // if we need to send messages send them here. This will happen after a delay
if message_info.is_some() { if message_info.is_some() {
messages_sent += 1; messages_sent += 1;
receiver.swarm.send_rpc( let (peer_id, stream_id) = message_info.as_ref().unwrap();
message_info.as_ref().unwrap().0.clone(), receiver.swarm.send_successful_response(
RPCEvent::Response( peer_id.clone(),
message_info.as_ref().unwrap().1.clone(), stream_id.clone(),
RPCCodedResponse::Success(rpc_response.clone()), rpc_response.clone(),
),
); );
debug!(log, "Sending message {}", messages_sent); debug!(log, "Sending message {}", messages_sent);
if messages_sent == messages_to_send + extra_messages_to_send { if messages_sent == messages_to_send + extra_messages_to_send {
@ -775,7 +713,7 @@ async fn test_goodbye_rpc() {
let (mut sender, mut receiver) = common::build_node_pair(&log).await; let (mut sender, mut receiver) = common::build_node_pair(&log).await;
// Goodbye Request // Goodbye Request
let rpc_request = RPCRequest::Goodbye(GoodbyeReason::ClientShutdown); let rpc_request = Request::Goodbye(GoodbyeReason::ClientShutdown);
// build the sender future // build the sender future
let sender_future = async { let sender_future = async {
@ -786,7 +724,7 @@ async fn test_goodbye_rpc() {
debug!(log, "Sending RPC"); debug!(log, "Sending RPC");
sender sender
.swarm .swarm
.send_rpc(peer_id, RPCEvent::Request(10, rpc_request.clone())); .send_request(peer_id, RequestId::Sync(10), rpc_request.clone());
} }
_ => {} // Ignore other RPC messages _ => {} // Ignore other RPC messages
} }
@ -797,19 +735,15 @@ async fn test_goodbye_rpc() {
let receiver_future = async { let receiver_future = async {
loop { loop {
match receiver.next_event().await { match receiver.next_event().await {
Libp2pEvent::Behaviour(BehaviourEvent::RPC(_peer_id, event)) => { Libp2pEvent::Behaviour(BehaviourEvent::RequestReceived {
match event { peer_id: _,
id: _,
request,
}) => {
// Should receive sent RPC request // Should receive sent RPC request
RPCEvent::Request(id, request) => {
if request == rpc_request {
assert_eq!(id, 0);
assert_eq!(rpc_request.clone(), request); // receives the goodbye. Nothing left to do assert_eq!(rpc_request.clone(), request); // receives the goodbye. Nothing left to do
return; return;
} }
}
_ => {} // Ignore other events
}
}
_ => {} // Ignore other events _ => {} // Ignore other events
} }
} }

View File

@ -10,8 +10,8 @@ use crate::error;
use crate::service::NetworkMessage; use crate::service::NetworkMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError};
use eth2_libp2p::{ use eth2_libp2p::{
rpc::{RPCCodedResponse, RPCRequest, RPCResponse, RequestId, ResponseTermination}, rpc::{RPCError, RequestId, SubstreamId},
MessageId, NetworkGlobals, PeerId, PubsubMessage, RPCEvent, MessageId, NetworkGlobals, PeerId, PubsubMessage, Request, Response,
}; };
use futures::prelude::*; use futures::prelude::*;
use processor::Processor; use processor::Processor;
@ -43,8 +43,24 @@ pub enum RouterMessage<T: EthSpec> {
PeerDialed(PeerId), PeerDialed(PeerId),
/// Peer has disconnected, /// Peer has disconnected,
PeerDisconnected(PeerId), PeerDisconnected(PeerId),
/// An RPC response/request has been received. /// An RPC request has been received.
RPC(PeerId, RPCEvent<T>), RPCRequestReceived {
peer_id: PeerId,
stream_id: SubstreamId,
request: Request,
},
/// An RPC response has been received.
RPCResponseReceived {
peer_id: PeerId,
request_id: RequestId,
response: Response<T>,
},
/// An RPC request failed
RPCFailed {
peer_id: PeerId,
request_id: RequestId,
error: RPCError,
},
/// A gossip message has been received. The fields are: message id, the peer that sent us this /// A gossip message has been received. The fields are: message id, the peer that sent us this
/// message and the message itself. /// message and the message itself.
PubsubMessage(MessageId, PeerId, PubsubMessage<T>), PubsubMessage(MessageId, PeerId, PubsubMessage<T>),
@ -109,11 +125,32 @@ impl<T: BeaconChainTypes> Router<T> {
RouterMessage::PeerDisconnected(peer_id) => { RouterMessage::PeerDisconnected(peer_id) => {
self.processor.on_disconnect(peer_id); self.processor.on_disconnect(peer_id);
} }
// An RPC message request/response has been received RouterMessage::RPCRequestReceived {
RouterMessage::RPC(peer_id, rpc_event) => { peer_id,
self.handle_rpc_message(peer_id, rpc_event); stream_id,
request,
} => {
self.handle_rpc_request(peer_id, stream_id, request);
}
RouterMessage::RPCResponseReceived {
peer_id,
request_id,
response,
} => {
self.handle_rpc_response(peer_id, request_id, response);
}
RouterMessage::RPCFailed {
peer_id,
request_id,
error,
} => {
warn!(self.log, "RPC Error";
"peer_id" => peer_id.to_string(),
"request_id" => request_id,
"error" => error.to_string(),
"client" => self.network_globals.client(&peer_id).to_string());
self.processor.on_rpc_error(peer_id, request_id);
} }
// An RPC message request/response has been received
RouterMessage::PubsubMessage(id, peer_id, gossip) => { RouterMessage::PubsubMessage(id, peer_id, gossip) => {
self.handle_gossip(id, peer_id, gossip); self.handle_gossip(id, peer_id, gossip);
} }
@ -122,32 +159,14 @@ impl<T: BeaconChainTypes> Router<T> {
/* RPC - Related functionality */ /* RPC - Related functionality */
/// Handle RPC messages
fn handle_rpc_message(&mut self, peer_id: PeerId, rpc_message: RPCEvent<T::EthSpec>) {
match rpc_message {
RPCEvent::Request(id, req) => self.handle_rpc_request(peer_id, id, req),
RPCEvent::Response(id, resp) => self.handle_rpc_response(peer_id, id, resp),
RPCEvent::Error(id, _protocol, error) => {
warn!(self.log, "RPC Error"; "peer_id" => peer_id.to_string(), "request_id" => id, "error" => error.to_string(),
"client" => self.network_globals.client(&peer_id).to_string());
self.processor.on_rpc_error(peer_id, id);
}
}
}
/// A new RPC request has been received from the network. /// A new RPC request has been received from the network.
fn handle_rpc_request( fn handle_rpc_request(&mut self, peer_id: PeerId, stream_id: SubstreamId, request: Request) {
&mut self,
peer_id: PeerId,
request_id: RequestId,
request: RPCRequest<T::EthSpec>,
) {
match request { match request {
RPCRequest::Status(status_message) => { Request::Status(status_message) => {
self.processor self.processor
.on_status_request(peer_id, request_id, status_message) .on_status_request(peer_id, stream_id, status_message)
} }
RPCRequest::Goodbye(goodbye_reason) => { Request::Goodbye(goodbye_reason) => {
debug!( debug!(
self.log, "Peer sent Goodbye"; self.log, "Peer sent Goodbye";
"peer_id" => peer_id.to_string(), "peer_id" => peer_id.to_string(),
@ -156,14 +175,12 @@ impl<T: BeaconChainTypes> Router<T> {
); );
self.processor.on_disconnect(peer_id); self.processor.on_disconnect(peer_id);
} }
RPCRequest::BlocksByRange(request) => self Request::BlocksByRange(request) => self
.processor .processor
.on_blocks_by_range_request(peer_id, request_id, request), .on_blocks_by_range_request(peer_id, stream_id, request),
RPCRequest::BlocksByRoot(request) => self Request::BlocksByRoot(request) => self
.processor .processor
.on_blocks_by_root_request(peer_id, request_id, request), .on_blocks_by_root_request(peer_id, stream_id, request),
RPCRequest::Ping(_) => unreachable!("Ping MUST be handled in the behaviour"),
RPCRequest::MetaData(_) => unreachable!("MetaData MUST be handled in the behaviour"),
} }
} }
@ -173,71 +190,20 @@ impl<T: BeaconChainTypes> Router<T> {
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: RequestId,
error_response: RPCCodedResponse<T::EthSpec>, response: Response<T::EthSpec>,
) { ) {
// an error could have occurred. // an error could have occurred.
match error_response { match response {
RPCCodedResponse::InvalidRequest(error) => { Response::Status(status_message) => {
warn!(self.log, "RPC Invalid Request";
"peer_id" => peer_id.to_string(),
"request_id" => request_id,
"error" => error.to_string(),
"client" => self.network_globals.client(&peer_id).to_string());
self.processor.on_rpc_error(peer_id, request_id);
}
RPCCodedResponse::ServerError(error) => {
warn!(self.log, "RPC Server Error" ;
"peer_id" => peer_id.to_string(),
"request_id" => request_id,
"error" => error.to_string(),
"client" => self.network_globals.client(&peer_id).to_string());
self.processor.on_rpc_error(peer_id, request_id);
}
RPCCodedResponse::Unknown(error) => {
warn!(self.log, "RPC Unknown Error";
"peer_id" => peer_id.to_string(),
"request_id" => request_id,
"error" => error.to_string(),
"client" => self.network_globals.client(&peer_id).to_string());
self.processor.on_rpc_error(peer_id, request_id);
}
RPCCodedResponse::Success(response) => match response {
RPCResponse::Status(status_message) => {
self.processor.on_status_response(peer_id, status_message); self.processor.on_status_response(peer_id, status_message);
} }
RPCResponse::BlocksByRange(beacon_block) => { Response::BlocksByRange(beacon_block) => {
self.processor.on_blocks_by_range_response(
peer_id,
request_id,
Some(beacon_block),
);
}
RPCResponse::BlocksByRoot(beacon_block) => {
self.processor.on_blocks_by_root_response(
peer_id,
request_id,
Some(beacon_block),
);
}
RPCResponse::Pong(_) => {
unreachable!("Ping must be handled in the behaviour");
}
RPCResponse::MetaData(_) => {
unreachable!("Meta data must be handled in the behaviour");
}
},
RPCCodedResponse::StreamTermination(response_type) => {
// have received a stream termination, notify the processing functions
match response_type {
ResponseTermination::BlocksByRange => {
self.processor self.processor
.on_blocks_by_range_response(peer_id, request_id, None); .on_blocks_by_range_response(peer_id, request_id, beacon_block);
} }
ResponseTermination::BlocksByRoot => { Response::BlocksByRoot(beacon_block) => {
self.processor self.processor
.on_blocks_by_root_response(peer_id, request_id, None); .on_blocks_by_root_response(peer_id, request_id, beacon_block);
}
}
} }
} }
} }

View File

@ -7,9 +7,8 @@ use beacon_chain::{
}, },
BeaconChain, BeaconChainTypes, BlockError, BlockProcessingOutcome, GossipVerifiedBlock, BeaconChain, BeaconChainTypes, BlockError, BlockProcessingOutcome, GossipVerifiedBlock,
}; };
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::*;
use eth2_libp2p::rpc::{RPCCodedResponse, RPCEvent, RPCRequest, RPCResponse, RequestId}; use eth2_libp2p::{NetworkGlobals, PeerId, Request, Response};
use eth2_libp2p::{NetworkGlobals, PeerId};
use slog::{debug, error, o, trace, warn}; use slog::{debug, error, o, trace, warn};
use ssz::Encode; use ssz::Encode;
use std::sync::Arc; use std::sync::Arc;
@ -86,7 +85,10 @@ impl<T: BeaconChainTypes> Processor<T> {
/// An error occurred during an RPC request. The state is maintained by the sync manager, so /// An error occurred during an RPC request. The state is maintained by the sync manager, so
/// this function notifies the sync manager of the error. /// this function notifies the sync manager of the error.
pub fn on_rpc_error(&mut self, peer_id: PeerId, request_id: RequestId) { pub fn on_rpc_error(&mut self, peer_id: PeerId, request_id: RequestId) {
self.send_to_sync(SyncMessage::RPCError(peer_id, request_id)); // Check if the failed RPC belongs to sync
if let RequestId::Sync(id) = request_id {
self.send_to_sync(SyncMessage::RPCError(peer_id, id));
}
} }
/// Sends a `Status` message to the peer. /// Sends a `Status` message to the peer.
@ -106,7 +108,7 @@ impl<T: BeaconChainTypes> Processor<T> {
"head_slot" => format!("{}", status_message.head_slot), "head_slot" => format!("{}", status_message.head_slot),
); );
self.network self.network
.send_rpc_request(peer_id, RPCRequest::Status(status_message)); .send_processor_request(peer_id, Request::Status(status_message));
} }
} }
@ -116,7 +118,7 @@ impl<T: BeaconChainTypes> Processor<T> {
pub fn on_status_request( pub fn on_status_request(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: SubstreamId,
status: StatusMessage, status: StatusMessage,
) { ) {
debug!( debug!(
@ -133,10 +135,10 @@ impl<T: BeaconChainTypes> Processor<T> {
// ignore status responses if we are shutting down // ignore status responses if we are shutting down
if let Some(status_message) = status_message(&self.chain) { if let Some(status_message) = status_message(&self.chain) {
// Say status back. // Say status back.
self.network.send_rpc_response( self.network.send_response(
peer_id.clone(), peer_id.clone(),
Response::Status(status_message),
request_id, request_id,
RPCResponse::Status(status_message),
); );
} }
@ -281,16 +283,16 @@ impl<T: BeaconChainTypes> Processor<T> {
pub fn on_blocks_by_root_request( pub fn on_blocks_by_root_request(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: SubstreamId,
request: BlocksByRootRequest, request: BlocksByRootRequest,
) { ) {
let mut send_block_count = 0; let mut send_block_count = 0;
for root in request.block_roots.iter() { for root in request.block_roots.iter() {
if let Ok(Some(block)) = self.chain.store.get_block(root) { if let Ok(Some(block)) = self.chain.store.get_block(root) {
self.network.send_rpc_response( self.network.send_response(
peer_id.clone(), peer_id.clone(),
Response::BlocksByRoot(Some(Box::new(block))),
request_id, request_id,
RPCResponse::BlocksByRoot(Box::new(block)),
); );
send_block_count += 1; send_block_count += 1;
} else { } else {
@ -311,18 +313,15 @@ impl<T: BeaconChainTypes> Processor<T> {
); );
// send stream termination // send stream termination
self.network.send_rpc_error_response( self.network
peer_id, .send_response(peer_id, Response::BlocksByRoot(None), request_id);
request_id,
RPCCodedResponse::StreamTermination(ResponseTermination::BlocksByRoot),
);
} }
/// Handle a `BlocksByRange` request from the peer. /// Handle a `BlocksByRange` request from the peer.
pub fn on_blocks_by_range_request( pub fn on_blocks_by_range_request(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, request_id: SubstreamId,
req: BlocksByRangeRequest, req: BlocksByRangeRequest,
) { ) {
debug!( debug!(
@ -388,10 +387,10 @@ impl<T: BeaconChainTypes> Processor<T> {
&& block.slot() < req.start_slot + req.count * req.step && block.slot() < req.start_slot + req.count * req.step
{ {
blocks_sent += 1; blocks_sent += 1;
self.network.send_rpc_response( self.network.send_response(
peer_id.clone(), peer_id.clone(),
Response::BlocksByRange(Some(Box::new(block))),
request_id, request_id,
RPCResponse::BlocksByRange(Box::new(block)),
); );
} }
} else { } else {
@ -425,11 +424,8 @@ impl<T: BeaconChainTypes> Processor<T> {
} }
// send the stream terminator // send the stream terminator
self.network.send_rpc_error_response( self.network
peer_id, .send_response(peer_id, Response::BlocksByRange(None), request_id);
request_id,
RPCCodedResponse::StreamTermination(ResponseTermination::BlocksByRange),
);
} }
/// Handle a `BlocksByRange` response from the peer. /// Handle a `BlocksByRange` response from the peer.
@ -446,11 +442,18 @@ impl<T: BeaconChainTypes> Processor<T> {
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
); );
if let RequestId::Sync(id) = request_id {
self.send_to_sync(SyncMessage::BlocksByRangeResponse { self.send_to_sync(SyncMessage::BlocksByRangeResponse {
peer_id, peer_id,
request_id, request_id: id,
beacon_block, beacon_block,
}); });
} else {
debug!(
self.log,
"All blocks by range responses should belong to sync"
);
}
} }
/// Handle a `BlocksByRoot` response from the peer. /// Handle a `BlocksByRoot` response from the peer.
@ -466,11 +469,18 @@ impl<T: BeaconChainTypes> Processor<T> {
"peer" => format!("{:?}", peer_id), "peer" => format!("{:?}", peer_id),
); );
if let RequestId::Sync(id) = request_id {
self.send_to_sync(SyncMessage::BlocksByRootResponse { self.send_to_sync(SyncMessage::BlocksByRootResponse {
peer_id, peer_id,
request_id, request_id: id,
beacon_block, beacon_block,
}); });
} else {
debug!(
self.log,
"All Blocks by Root responses should belong to sync"
)
}
} }
/// Template function to be called on a block to determine if the block should be propagated /// Template function to be called on a block to determine if the block should be propagated
@ -902,8 +912,6 @@ pub(crate) fn status_message<T: BeaconChainTypes>(
/// Wraps a Network Channel to employ various RPC related network functionality for the /// Wraps a Network Channel to employ various RPC related network functionality for the
/// processor. /// processor.
/// The Processor doesn't manage it's own request Id's and can therefore only send
/// responses or requests with 0 request Ids.
pub struct HandlerNetworkContext<T: EthSpec> { pub struct HandlerNetworkContext<T: EthSpec> {
/// 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<T>>, network_send: mpsc::UnboundedSender<NetworkMessage<T>>,
@ -916,6 +924,12 @@ impl<T: EthSpec> HandlerNetworkContext<T> {
Self { network_send, log } Self { network_send, log }
} }
fn inform_network(&mut self, msg: NetworkMessage<T>) {
self.network_send
.send(msg)
.unwrap_or_else(|_| warn!(self.log, "Could not send message to the network service"))
}
pub fn disconnect(&mut self, peer_id: PeerId, reason: GoodbyeReason) { pub fn disconnect(&mut self, peer_id: PeerId, reason: GoodbyeReason) {
warn!( warn!(
&self.log, &self.log,
@ -923,55 +937,42 @@ impl<T: EthSpec> HandlerNetworkContext<T> {
"reason" => format!("{:?}", reason), "reason" => format!("{:?}", reason),
"peer_id" => format!("{:?}", peer_id), "peer_id" => format!("{:?}", peer_id),
); );
self.send_rpc_request(peer_id.clone(), RPCRequest::Goodbye(reason)); self.send_processor_request(peer_id.clone(), Request::Goodbye(reason));
self.network_send self.inform_network(NetworkMessage::Disconnect { peer_id });
.send(NetworkMessage::Disconnect { peer_id })
.unwrap_or_else(|_| {
warn!(
self.log,
"Could not send a Disconnect to the network service"
)
});
} }
pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest<T>) { pub fn send_processor_request(&mut self, peer_id: PeerId, request: Request) {
// the message handler cannot send requests with ids. Id's are managed by the sync self.inform_network(NetworkMessage::SendRequest {
// manager.
let request_id = 0;
self.send_rpc_event(peer_id, RPCEvent::Request(request_id, rpc_request));
}
/// Convenience function to wrap successful RPC Responses.
pub fn send_rpc_response(
&mut self,
peer_id: PeerId,
request_id: RequestId,
rpc_response: RPCResponse<T>,
) {
self.send_rpc_event(
peer_id, peer_id,
RPCEvent::Response(request_id, RPCCodedResponse::Success(rpc_response)), request_id: RequestId::Router,
); request,
})
} }
/// Send an RPCCodedResponse. This handles errors and stream terminations. pub fn send_response(
pub fn send_rpc_error_response(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request_id: RequestId, response: Response<T>,
rpc_error_response: RPCCodedResponse<T>, stream_id: SubstreamId,
) { ) {
self.send_rpc_event(peer_id, RPCEvent::Response(request_id, rpc_error_response)); self.inform_network(NetworkMessage::SendResponse {
peer_id,
stream_id,
response,
})
} }
pub fn _send_error_response(
fn send_rpc_event(&mut self, peer_id: PeerId, rpc_event: RPCEvent<T>) { &mut self,
self.network_send peer_id: PeerId,
.send(NetworkMessage::RPC(peer_id, rpc_event)) substream_id: SubstreamId,
.unwrap_or_else(|_| { error: RPCResponseErrorCode,
warn!( reason: String,
self.log, ) {
"Could not send RPC message to the network service" self.inform_network(NetworkMessage::SendError {
) peer_id,
}); error,
substream_id,
reason,
})
} }
} }

View File

@ -7,8 +7,11 @@ use crate::{
use crate::{error, metrics}; use crate::{error, metrics};
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::Service as LibP2PService; use eth2_libp2p::Service as LibP2PService;
use eth2_libp2p::{rpc::RPCRequest, BehaviourEvent, Enr, MessageId, NetworkGlobals, PeerId}; use eth2_libp2p::{
use eth2_libp2p::{Libp2pEvent, PubsubMessage, RPCEvent}; rpc::{RPCResponseErrorCode, RequestId, SubstreamId},
Libp2pEvent, PubsubMessage, Request, Response,
};
use eth2_libp2p::{BehaviourEvent, Enr, MessageId, NetworkGlobals, PeerId};
use futures::prelude::*; use futures::prelude::*;
use rest_types::ValidatorSubscription; use rest_types::ValidatorSubscription;
use slog::{debug, error, info, o, trace}; use slog::{debug, error, info, o, trace};
@ -123,6 +126,9 @@ fn spawn_service<T: BeaconChainTypes>(
// spawn on the current executor // spawn on the current executor
executor.spawn_without_exit(async move { executor.spawn_without_exit(async move {
// TODO: there is something with this code that prevents cargo fmt from doing anything at
// all. Ok, it is worse, the compiler doesn't show errors over this code beyond ast
// checking
loop { loop {
// build the futures to check simultaneously // build the futures to check simultaneously
tokio::select! { tokio::select! {
@ -154,9 +160,14 @@ fn spawn_service<T: BeaconChainTypes>(
// handle a message sent to the network // handle a message sent to the network
Some(message) = service.network_recv.recv() => { Some(message) = service.network_recv.recv() => {
match message { match message {
NetworkMessage::RPC(peer_id, rpc_event) => { NetworkMessage::SendRequest{ peer_id, request, request_id } => {
trace!(service.log, "Sending RPC"; "rpc" => format!("{}", rpc_event)); service.libp2p.send_request(peer_id, request_id, request);
service.libp2p.swarm.send_rpc(peer_id, rpc_event); }
NetworkMessage::SendResponse{ peer_id, response, stream_id } => {
service.libp2p.send_response(peer_id, stream_id, response);
}
NetworkMessage::SendError{ peer_id, error, substream_id, reason } => {
service.libp2p.respond_with_error(peer_id, substream_id, error, reason);
} }
NetworkMessage::Propagate { NetworkMessage::Propagate {
propagation_source, propagation_source,
@ -256,10 +267,11 @@ fn spawn_service<T: BeaconChainTypes>(
// poll the swarm // poll the swarm
match libp2p_event { match libp2p_event {
Libp2pEvent::Behaviour(event) => match event { Libp2pEvent::Behaviour(event) => match event {
BehaviourEvent::RPC(peer_id, rpc_event) => { BehaviourEvent::RequestReceived{peer_id, id, request} => {
if let Request::Goodbye(_) = request {
// if we received a Goodbye message, drop and ban the peer // if we received a Goodbye message, drop and ban the peer
if let RPCEvent::Request(_, RPCRequest::Goodbye(_)) = rpc_event {
//peers_to_ban.push(peer_id.clone()); //peers_to_ban.push(peer_id.clone());
// TODO: remove this: https://github.com/sigp/lighthouse/issues/1240
service.libp2p.disconnect_and_ban_peer( service.libp2p.disconnect_and_ban_peer(
peer_id.clone(), peer_id.clone(),
std::time::Duration::from_secs(BAN_PEER_TIMEOUT), std::time::Duration::from_secs(BAN_PEER_TIMEOUT),
@ -267,11 +279,29 @@ fn spawn_service<T: BeaconChainTypes>(
}; };
let _ = service let _ = service
.router_send .router_send
.send(RouterMessage::RPC(peer_id, rpc_event)) .send(RouterMessage::RPCRequestReceived{peer_id, stream_id:id, request})
.map_err(|_| { .map_err(|_| {
debug!(service.log, "Failed to send RPC to router"); debug!(service.log, "Failed to send RPC to router");
}); });
} }
BehaviourEvent::ResponseReceived{peer_id, id, response} => {
let _ = service
.router_send
.send(RouterMessage::RPCResponseReceived{ peer_id, request_id:id, response })
.map_err(|_| {
debug!(service.log, "Failed to send RPC to router");
});
}
BehaviourEvent::RPCFailed{id, peer_id, error} => {
let _ = service
.router_send
.send(RouterMessage::RPCFailed{ peer_id, request_id:id, error })
.map_err(|_| {
debug!(service.log, "Failed to send RPC to router");
});
}
BehaviourEvent::StatusPeer(peer_id) => { BehaviourEvent::StatusPeer(peer_id) => {
let _ = service let _ = service
.router_send .router_send
@ -384,8 +414,27 @@ pub enum NetworkMessage<T: EthSpec> {
Subscribe { Subscribe {
subscriptions: Vec<ValidatorSubscription>, subscriptions: Vec<ValidatorSubscription>,
}, },
/// Send an RPC message to the libp2p service. /// Send an RPC request to the libp2p service.
RPC(PeerId, RPCEvent<T>), SendRequest {
peer_id: PeerId,
request: Request,
request_id: RequestId,
},
/// Send a successful Response to the libp2p service.
SendResponse {
peer_id: PeerId,
response: Response<T>,
stream_id: SubstreamId,
},
/// Respond to a peer's request with an error.
SendError {
// TODO: note that this is never used, we just say goodbye without nicely clossing the
// stream assigned to the request
peer_id: PeerId,
error: RPCResponseErrorCode,
reason: String,
substream_id: SubstreamId,
},
/// Publish a list of messages to the gossipsub protocol. /// Publish a list of messages to the gossipsub protocol.
Publish { messages: Vec<PubsubMessage<T>> }, Publish { messages: Vec<PubsubMessage<T>> },
/// Propagate a received gossipsub message. /// Propagate a received gossipsub message.

View File

@ -37,9 +37,10 @@ use super::block_processor::{spawn_block_processor, BatchProcessResult, ProcessI
use super::network_context::SyncNetworkContext; use super::network_context::SyncNetworkContext;
use super::peer_sync_info::{PeerSyncInfo, PeerSyncType}; use super::peer_sync_info::{PeerSyncInfo, PeerSyncType};
use super::range_sync::{BatchId, ChainId, RangeSync}; use super::range_sync::{BatchId, ChainId, RangeSync};
use super::RequestId;
use crate::service::NetworkMessage; use crate::service::NetworkMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockProcessingOutcome};
use eth2_libp2p::rpc::{methods::*, RequestId}; use eth2_libp2p::rpc::BlocksByRootRequest;
use eth2_libp2p::types::NetworkGlobals; use eth2_libp2p::types::NetworkGlobals;
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
use fnv::FnvHashMap; use fnv::FnvHashMap;

View File

@ -9,3 +9,6 @@ mod range_sync;
pub use manager::SyncMessage; pub use manager::SyncMessage;
pub use peer_sync_info::PeerSyncInfo; pub use peer_sync_info::PeerSyncInfo;
/// Type of id of rpc requests sent by sync
pub type RequestId = usize;

View File

@ -4,9 +4,8 @@
use crate::router::processor::status_message; use crate::router::processor::status_message;
use crate::service::NetworkMessage; use crate::service::NetworkMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, RequestId};
use eth2_libp2p::rpc::{RPCEvent, RPCRequest, RequestId}; use eth2_libp2p::{Client, NetworkGlobals, PeerId, Request};
use eth2_libp2p::{Client, NetworkGlobals, PeerId};
use slog::{debug, trace, warn}; use slog::{debug, trace, warn};
use std::sync::Arc; use std::sync::Arc;
use tokio::sync::mpsc; use tokio::sync::mpsc;
@ -22,7 +21,7 @@ pub struct SyncNetworkContext<T: EthSpec> {
network_globals: Arc<NetworkGlobals<T>>, network_globals: Arc<NetworkGlobals<T>>,
/// A sequential ID for all RPC requests. /// A sequential ID for all RPC requests.
request_id: RequestId, request_id: usize,
/// Logger for the `SyncNetworkContext`. /// Logger for the `SyncNetworkContext`.
log: slog::Logger, log: slog::Logger,
} }
@ -68,7 +67,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
"head_slot" => format!("{}", status_message.head_slot), "head_slot" => format!("{}", status_message.head_slot),
); );
let _ = self.send_rpc_request(peer_id, RPCRequest::Status(status_message)); let _ = self.send_rpc_request(peer_id, Request::Status(status_message));
} }
} }
@ -76,7 +75,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request: BlocksByRangeRequest, request: BlocksByRangeRequest,
) -> Result<RequestId, &'static str> { ) -> Result<usize, &'static str> {
trace!( trace!(
self.log, self.log,
"Sending BlocksByRange Request"; "Sending BlocksByRange Request";
@ -84,14 +83,14 @@ impl<T: EthSpec> SyncNetworkContext<T> {
"count" => request.count, "count" => request.count,
"peer" => format!("{:?}", peer_id) "peer" => format!("{:?}", peer_id)
); );
self.send_rpc_request(peer_id, RPCRequest::BlocksByRange(request)) self.send_rpc_request(peer_id, Request::BlocksByRange(request))
} }
pub fn blocks_by_root_request( pub fn blocks_by_root_request(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
request: BlocksByRootRequest, request: BlocksByRootRequest,
) -> Result<RequestId, &'static str> { ) -> Result<usize, &'static str> {
trace!( trace!(
self.log, self.log,
"Sending BlocksByRoot Request"; "Sending BlocksByRoot Request";
@ -99,7 +98,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
"count" => request.block_roots.len(), "count" => request.block_roots.len(),
"peer" => format!("{:?}", peer_id) "peer" => format!("{:?}", peer_id)
); );
self.send_rpc_request(peer_id, RPCRequest::BlocksByRoot(request)) self.send_rpc_request(peer_id, Request::BlocksByRoot(request))
} }
pub fn downvote_peer(&mut self, peer_id: PeerId) { pub fn downvote_peer(&mut self, peer_id: PeerId) {
@ -109,6 +108,10 @@ impl<T: EthSpec> SyncNetworkContext<T> {
"peer" => format!("{:?}", peer_id) "peer" => format!("{:?}", peer_id)
); );
// TODO: Implement reputation // TODO: Implement reputation
// TODO: what if we first close the channel sending a response
// RPCResponseErrorCode::InvalidRequest (or something)
// and then disconnect the peer? either request dc or let the behaviour have that logic
// itself
self.disconnect(peer_id, GoodbyeReason::Fault); self.disconnect(peer_id, GoodbyeReason::Fault);
} }
@ -121,7 +124,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
); );
// ignore the error if the channel send fails // ignore the error if the channel send fails
let _ = self.send_rpc_request(peer_id.clone(), RPCRequest::Goodbye(reason)); let _ = self.send_rpc_request(peer_id.clone(), Request::Goodbye(reason));
self.network_send self.network_send
.send(NetworkMessage::Disconnect { peer_id }) .send(NetworkMessage::Disconnect { peer_id })
.unwrap_or_else(|_| { .unwrap_or_else(|_| {
@ -135,26 +138,21 @@ impl<T: EthSpec> SyncNetworkContext<T> {
pub fn send_rpc_request( pub fn send_rpc_request(
&mut self, &mut self,
peer_id: PeerId, peer_id: PeerId,
rpc_request: RPCRequest<T>, request: Request,
) -> Result<RequestId, &'static str> { ) -> Result<usize, &'static str> {
let request_id = self.request_id; let request_id = self.request_id;
self.request_id += 1; self.request_id += 1;
self.send_rpc_event(peer_id, RPCEvent::Request(request_id, rpc_request))?; self.send_network_msg(NetworkMessage::SendRequest {
peer_id,
request_id: RequestId::Sync(request_id),
request,
})?;
Ok(request_id) Ok(request_id)
} }
fn send_rpc_event( fn send_network_msg(&mut self, msg: NetworkMessage<T>) -> Result<(), &'static str> {
&mut self, self.network_send.send(msg).map_err(|_| {
peer_id: PeerId, debug!(self.log, "Could not send message to the network service");
rpc_event: RPCEvent<T>,
) -> Result<(), &'static str> {
self.network_send
.send(NetworkMessage::RPC(peer_id, rpc_event))
.map_err(|_| {
debug!(
self.log,
"Could not send RPC message to the network service"
);
"Network channel send Failed" "Network channel send Failed"
}) })
} }

View File

@ -1,7 +1,7 @@
use super::manager::SLOT_IMPORT_TOLERANCE; use super::manager::SLOT_IMPORT_TOLERANCE;
use crate::router::processor::status_message; use crate::router::processor::status_message;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::*;
use eth2_libp2p::SyncInfo; use eth2_libp2p::SyncInfo;
use std::ops::Sub; use std::ops::Sub;
use std::sync::Arc; use std::sync::Arc;

View File

@ -1,6 +1,5 @@
use super::chain::EPOCHS_PER_BATCH; use super::chain::EPOCHS_PER_BATCH;
use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::methods::*;
use eth2_libp2p::rpc::RequestId;
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
use fnv::FnvHashMap; use fnv::FnvHashMap;
use ssz::Encode; use ssz::Encode;
@ -112,9 +111,9 @@ impl<T: EthSpec> PartialOrd for Batch<T> {
/// This is used to optimise searches for idle peers (peers that have no outbound batch requests). /// This is used to optimise searches for idle peers (peers that have no outbound batch requests).
pub struct PendingBatches<T: EthSpec> { pub struct PendingBatches<T: EthSpec> {
/// The current pending batches. /// The current pending batches.
batches: FnvHashMap<RequestId, Batch<T>>, batches: FnvHashMap<usize, Batch<T>>,
/// A mapping of peers to the number of pending requests. /// A mapping of peers to the number of pending requests.
peer_requests: HashMap<PeerId, HashSet<RequestId>>, peer_requests: HashMap<PeerId, HashSet<usize>>,
} }
impl<T: EthSpec> PendingBatches<T> { impl<T: EthSpec> PendingBatches<T> {
@ -125,7 +124,7 @@ impl<T: EthSpec> PendingBatches<T> {
} }
} }
pub fn insert(&mut self, request_id: RequestId, batch: Batch<T>) -> Option<Batch<T>> { pub fn insert(&mut self, request_id: usize, batch: Batch<T>) -> Option<Batch<T>> {
let peer_request = batch.current_peer.clone(); let peer_request = batch.current_peer.clone();
self.peer_requests self.peer_requests
.entry(peer_request) .entry(peer_request)
@ -134,7 +133,7 @@ impl<T: EthSpec> PendingBatches<T> {
self.batches.insert(request_id, batch) self.batches.insert(request_id, batch)
} }
pub fn remove(&mut self, request_id: RequestId) -> Option<Batch<T>> { pub fn remove(&mut self, request_id: usize) -> Option<Batch<T>> {
if let Some(batch) = self.batches.remove(&request_id) { if let Some(batch) = self.batches.remove(&request_id) {
if let Entry::Occupied(mut entry) = self.peer_requests.entry(batch.current_peer.clone()) if let Entry::Occupied(mut entry) = self.peer_requests.entry(batch.current_peer.clone())
{ {
@ -157,7 +156,7 @@ impl<T: EthSpec> PendingBatches<T> {
/// Adds a block to the batches if the request id exists. Returns None if there is no batch /// Adds a block to the batches if the request id exists. Returns None if there is no batch
/// matching the request id. /// matching the request id.
pub fn add_block(&mut self, request_id: RequestId, block: SignedBeaconBlock<T>) -> Option<()> { pub fn add_block(&mut self, request_id: usize, block: SignedBeaconBlock<T>) -> Option<()> {
let batch = self.batches.get_mut(&request_id)?; let batch = self.batches.get_mut(&request_id)?;
batch.downloaded_blocks.push(block); batch.downloaded_blocks.push(block);
Some(()) Some(())

View File

@ -1,9 +1,8 @@
use super::batch::{Batch, BatchId, PendingBatches}; use super::batch::{Batch, BatchId, PendingBatches};
use crate::sync::block_processor::{spawn_block_processor, BatchProcessResult, ProcessId}; use crate::sync::block_processor::{spawn_block_processor, BatchProcessResult, ProcessId};
use crate::sync::network_context::SyncNetworkContext; use crate::sync::network_context::SyncNetworkContext;
use crate::sync::SyncMessage; use crate::sync::{RequestId, SyncMessage};
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::RequestId;
use eth2_libp2p::PeerId; use eth2_libp2p::PeerId;
use rand::prelude::*; use rand::prelude::*;
use slog::{crit, debug, warn}; use slog::{crit, debug, warn};

View File

@ -47,8 +47,8 @@ use crate::sync::block_processor::BatchProcessResult;
use crate::sync::manager::SyncMessage; use crate::sync::manager::SyncMessage;
use crate::sync::network_context::SyncNetworkContext; use crate::sync::network_context::SyncNetworkContext;
use crate::sync::PeerSyncInfo; use crate::sync::PeerSyncInfo;
use crate::sync::RequestId;
use beacon_chain::{BeaconChain, BeaconChainTypes}; use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::rpc::RequestId;
use eth2_libp2p::{NetworkGlobals, PeerId}; use eth2_libp2p::{NetworkGlobals, PeerId};
use slog::{debug, error, trace}; use slog::{debug, error, trace};
use std::collections::HashSet; use std::collections::HashSet;