Custom RPC request management for sync (#3029)
## Proposed Changes Make `lighthouse_network` generic over request ids, now usable by sync
This commit is contained in:
parent
e88b18be09
commit
4bf1af4e85
@ -70,9 +70,16 @@ pub type PeerRequestId = (ConnectionId, SubstreamId);
|
||||
pub type SubscriptionFilter = MaxCountSubscriptionFilter<WhitelistSubscriptionFilter>;
|
||||
pub type Gossipsub = BaseGossipsub<SnappyTransform, SubscriptionFilter>;
|
||||
|
||||
/// Identifier of a request.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
|
||||
pub enum RequestId<AppReqId> {
|
||||
Application(AppReqId),
|
||||
Behaviour,
|
||||
}
|
||||
|
||||
/// The types of events than can be obtained from polling the behaviour.
|
||||
#[derive(Debug)]
|
||||
pub enum BehaviourEvent<TSpec: EthSpec> {
|
||||
pub enum BehaviourEvent<AppReqId: ReqId, TSpec: EthSpec> {
|
||||
/// We have successfully dialed and connected to a peer.
|
||||
PeerConnectedOutgoing(PeerId),
|
||||
/// A peer has successfully dialed and connected to us.
|
||||
@ -86,7 +93,7 @@ pub enum BehaviourEvent<TSpec: EthSpec> {
|
||||
/// An RPC Request that was sent failed.
|
||||
RPCFailed {
|
||||
/// The id of the failed request.
|
||||
id: RequestId,
|
||||
id: AppReqId,
|
||||
/// The peer to which this request was sent.
|
||||
peer_id: PeerId,
|
||||
},
|
||||
@ -102,7 +109,7 @@ pub enum BehaviourEvent<TSpec: EthSpec> {
|
||||
/// Peer that sent the response.
|
||||
peer_id: PeerId,
|
||||
/// Id of the request to which the peer is responding.
|
||||
id: RequestId,
|
||||
id: AppReqId,
|
||||
/// Response the peer sent.
|
||||
response: Response<TSpec>,
|
||||
},
|
||||
@ -134,16 +141,16 @@ enum InternalBehaviourMessage {
|
||||
/// behaviours.
|
||||
#[derive(NetworkBehaviour)]
|
||||
#[behaviour(
|
||||
out_event = "BehaviourEvent<TSpec>",
|
||||
out_event = "BehaviourEvent<AppReqId, TSpec>",
|
||||
poll_method = "poll",
|
||||
event_process = true
|
||||
)]
|
||||
pub struct Behaviour<TSpec: EthSpec> {
|
||||
pub struct Behaviour<AppReqId: ReqId, TSpec: EthSpec> {
|
||||
/* Sub-Behaviours */
|
||||
/// The routing pub-sub mechanism for eth2.
|
||||
gossipsub: Gossipsub,
|
||||
/// The Eth2 RPC specified in the wire-0 protocol.
|
||||
eth2_rpc: RPC<TSpec>,
|
||||
eth2_rpc: RPC<RequestId<AppReqId>, TSpec>,
|
||||
/// Discv5 Discovery protocol.
|
||||
discovery: Discovery<TSpec>,
|
||||
/// Keep regular connection to peers and disconnect if absent.
|
||||
@ -156,7 +163,7 @@ pub struct Behaviour<TSpec: EthSpec> {
|
||||
/* Auxiliary Fields */
|
||||
/// The output events generated by this behaviour to be consumed in the swarm poll.
|
||||
#[behaviour(ignore)]
|
||||
events: VecDeque<BehaviourEvent<TSpec>>,
|
||||
events: VecDeque<BehaviourEvent<AppReqId, TSpec>>,
|
||||
/// Internal behaviour events, the NBAction type is composed of sub-behaviours, so we use a
|
||||
/// custom type here to avoid having to specify the concrete type.
|
||||
#[behaviour(ignore)]
|
||||
@ -192,7 +199,7 @@ pub struct Behaviour<TSpec: EthSpec> {
|
||||
}
|
||||
|
||||
/// Implements the combined behaviour for the libp2p service.
|
||||
impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
impl<AppReqId: ReqId, TSpec: EthSpec> Behaviour<AppReqId, TSpec> {
|
||||
pub async fn new(
|
||||
local_key: &Keypair,
|
||||
ctx: ServiceContext<'_>,
|
||||
@ -562,9 +569,9 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
/* 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) {
|
||||
pub fn send_request(&mut self, peer_id: PeerId, request_id: AppReqId, request: Request) {
|
||||
self.eth2_rpc
|
||||
.send_request(peer_id, request_id, request.into())
|
||||
.send_request(peer_id, RequestId::Application(request_id), request.into())
|
||||
}
|
||||
|
||||
/// Send a successful response to a peer over RPC.
|
||||
@ -718,12 +725,12 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
}
|
||||
|
||||
/// Sends a Ping request to the peer.
|
||||
fn ping(&mut self, id: RequestId, peer_id: PeerId) {
|
||||
fn ping(&mut self, peer_id: PeerId) {
|
||||
let ping = crate::rpc::Ping {
|
||||
data: *self.network_globals.local_metadata.read().seq_number(),
|
||||
};
|
||||
trace!(self.log, "Sending Ping"; "request_id" => id, "peer_id" => %peer_id);
|
||||
|
||||
trace!(self.log, "Sending Ping"; "peer_id" => %peer_id);
|
||||
let id = RequestId::Behaviour;
|
||||
self.eth2_rpc
|
||||
.send_request(peer_id, id, OutboundRequest::Ping(ping));
|
||||
}
|
||||
@ -761,13 +768,19 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
|
||||
// RPC Propagation methods
|
||||
/// 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.add_event(BehaviourEvent::ResponseReceived {
|
||||
fn propagate_response(
|
||||
&mut self,
|
||||
id: RequestId<AppReqId>,
|
||||
peer_id: PeerId,
|
||||
response: Response<TSpec>,
|
||||
) {
|
||||
match id {
|
||||
RequestId::Application(id) => self.add_event(BehaviourEvent::ResponseReceived {
|
||||
peer_id,
|
||||
id,
|
||||
response,
|
||||
});
|
||||
}),
|
||||
RequestId::Behaviour => {}
|
||||
}
|
||||
}
|
||||
|
||||
@ -793,7 +806,7 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
}
|
||||
|
||||
/// Adds an event to the queue waking the current task to process it.
|
||||
fn add_event(&mut self, event: BehaviourEvent<TSpec>) {
|
||||
fn add_event(&mut self, event: BehaviourEvent<AppReqId, TSpec>) {
|
||||
self.events.push_back(event);
|
||||
if let Some(waker) = &self.waker {
|
||||
waker.wake_by_ref();
|
||||
@ -869,7 +882,11 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
*/
|
||||
|
||||
// Gossipsub
|
||||
impl<TSpec: EthSpec> NetworkBehaviourEventProcess<GossipsubEvent> for Behaviour<TSpec> {
|
||||
impl<AppReqId, TSpec> NetworkBehaviourEventProcess<GossipsubEvent> for Behaviour<AppReqId, TSpec>
|
||||
where
|
||||
AppReqId: ReqId,
|
||||
TSpec: EthSpec,
|
||||
{
|
||||
fn inject_event(&mut self, event: GossipsubEvent) {
|
||||
match event {
|
||||
GossipsubEvent::Message {
|
||||
@ -961,8 +978,13 @@ impl<TSpec: EthSpec> NetworkBehaviourEventProcess<GossipsubEvent> for Behaviour<
|
||||
}
|
||||
|
||||
// RPC
|
||||
impl<TSpec: EthSpec> NetworkBehaviourEventProcess<RPCMessage<TSpec>> for Behaviour<TSpec> {
|
||||
fn inject_event(&mut self, event: RPCMessage<TSpec>) {
|
||||
impl<AppReqId, TSpec> NetworkBehaviourEventProcess<RPCMessage<RequestId<AppReqId>, TSpec>>
|
||||
for Behaviour<AppReqId, TSpec>
|
||||
where
|
||||
AppReqId: ReqId,
|
||||
TSpec: EthSpec,
|
||||
{
|
||||
fn inject_event(&mut self, event: RPCMessage<RequestId<AppReqId>, TSpec>) {
|
||||
let peer_id = event.peer_id;
|
||||
|
||||
if !self.peer_manager.is_connected(&peer_id) {
|
||||
@ -1006,7 +1028,7 @@ impl<TSpec: EthSpec> NetworkBehaviourEventProcess<RPCMessage<TSpec>> for Behavio
|
||||
ConnectionDirection::Outgoing,
|
||||
);
|
||||
// inform failures of requests comming outside the behaviour
|
||||
if !matches!(id, RequestId::Behaviour) {
|
||||
if let RequestId::Application(id) = id {
|
||||
self.add_event(BehaviourEvent::RPCFailed { peer_id, id });
|
||||
}
|
||||
}
|
||||
@ -1090,7 +1112,11 @@ impl<TSpec: EthSpec> NetworkBehaviourEventProcess<RPCMessage<TSpec>> for Behavio
|
||||
}
|
||||
|
||||
// Discovery
|
||||
impl<TSpec: EthSpec> NetworkBehaviourEventProcess<DiscoveryEvent> for Behaviour<TSpec> {
|
||||
impl<AppReqId, TSpec> NetworkBehaviourEventProcess<DiscoveryEvent> for Behaviour<AppReqId, TSpec>
|
||||
where
|
||||
AppReqId: ReqId,
|
||||
TSpec: EthSpec,
|
||||
{
|
||||
fn inject_event(&mut self, event: DiscoveryEvent) {
|
||||
match event {
|
||||
DiscoveryEvent::SocketUpdated(socket_addr) => {
|
||||
@ -1119,7 +1145,11 @@ impl<TSpec: EthSpec> NetworkBehaviourEventProcess<DiscoveryEvent> for Behaviour<
|
||||
}
|
||||
|
||||
// Identify
|
||||
impl<TSpec: EthSpec> NetworkBehaviourEventProcess<IdentifyEvent> for Behaviour<TSpec> {
|
||||
impl<AppReqId, TSpec> NetworkBehaviourEventProcess<IdentifyEvent> for Behaviour<AppReqId, TSpec>
|
||||
where
|
||||
TSpec: EthSpec,
|
||||
AppReqId: ReqId,
|
||||
{
|
||||
fn inject_event(&mut self, event: IdentifyEvent) {
|
||||
match event {
|
||||
IdentifyEvent::Received { peer_id, mut info } => {
|
||||
@ -1140,15 +1170,20 @@ impl<TSpec: EthSpec> NetworkBehaviourEventProcess<IdentifyEvent> for Behaviour<T
|
||||
}
|
||||
}
|
||||
|
||||
impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
type BehaviourHandler<AppReqId, TSpec> =
|
||||
<Behaviour<AppReqId, TSpec> as NetworkBehaviour>::ConnectionHandler;
|
||||
|
||||
impl<AppReqId, TSpec> Behaviour<AppReqId, TSpec>
|
||||
where
|
||||
TSpec: EthSpec,
|
||||
AppReqId: ReqId,
|
||||
{
|
||||
/// Consumes the events list and drives the Lighthouse global NetworkBehaviour.
|
||||
fn poll(
|
||||
&mut self,
|
||||
cx: &mut Context,
|
||||
_: &mut impl PollParameters,
|
||||
) -> Poll<
|
||||
NBAction<BehaviourEvent<TSpec>, <Behaviour<TSpec> as NetworkBehaviour>::ConnectionHandler>,
|
||||
> {
|
||||
) -> Poll<NBAction<BehaviourEvent<AppReqId, TSpec>, BehaviourHandler<AppReqId, TSpec>>> {
|
||||
if let Some(waker) = &self.waker {
|
||||
if waker.will_wake(cx.waker()) {
|
||||
self.waker = Some(cx.waker().clone());
|
||||
@ -1207,7 +1242,9 @@ impl<TSpec: EthSpec> Behaviour<TSpec> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<TSpec: EthSpec> NetworkBehaviourEventProcess<PeerManagerEvent> for Behaviour<TSpec> {
|
||||
impl<AppReqId: ReqId, TSpec: EthSpec> NetworkBehaviourEventProcess<PeerManagerEvent>
|
||||
for Behaviour<AppReqId, TSpec>
|
||||
{
|
||||
fn inject_event(&mut self, event: PeerManagerEvent) {
|
||||
match event {
|
||||
PeerManagerEvent::PeerConnectedIncoming(peer_id) => {
|
||||
@ -1242,7 +1279,7 @@ impl<TSpec: EthSpec> NetworkBehaviourEventProcess<PeerManagerEvent> for Behaviou
|
||||
}
|
||||
PeerManagerEvent::Ping(peer_id) => {
|
||||
// send a ping request to this peer
|
||||
self.ping(RequestId::Behaviour, peer_id);
|
||||
self.ping(peer_id);
|
||||
}
|
||||
PeerManagerEvent::MetaData(peer_id) => {
|
||||
self.send_meta_data_request(peer_id);
|
||||
@ -1251,7 +1288,8 @@ impl<TSpec: EthSpec> NetworkBehaviourEventProcess<PeerManagerEvent> for Behaviou
|
||||
debug!(self.log, "Peer Manager disconnecting peer";
|
||||
"peer_id" => %peer_id, "reason" => %reason);
|
||||
// send one goodbye
|
||||
self.eth2_rpc.shutdown(peer_id, reason);
|
||||
self.eth2_rpc
|
||||
.shutdown(peer_id, RequestId::Behaviour, reason);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1335,3 +1373,19 @@ pub fn save_metadata_to_disk<E: EthSpec>(dir: &Path, metadata: MetaData<E>, log:
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<AppReqId: std::fmt::Debug> slog::Value for RequestId<AppReqId> {
|
||||
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::Application(ref id) => {
|
||||
slog::Value::serialize(&format_args!("{:?}", id), record, key, serializer)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -110,9 +110,13 @@ impl<TSpec: EthSpec> NetworkBehaviour for PeerManager<TSpec> {
|
||||
_connection_id: &ConnectionId,
|
||||
endpoint: &ConnectedPoint,
|
||||
_failed_addresses: Option<&Vec<Multiaddr>>,
|
||||
_other_established: usize,
|
||||
other_established: usize,
|
||||
) {
|
||||
debug!(self.log, "Connection established"; "peer_id" => %peer_id, "connection" => ?endpoint.to_endpoint());
|
||||
if other_established == 0 {
|
||||
self.events.push(PeerManagerEvent::MetaData(*peer_id));
|
||||
}
|
||||
|
||||
// Check NAT if metrics are enabled
|
||||
if self.network_globals.local_enr.read().udp().is_some() {
|
||||
metrics::check_nat();
|
||||
|
@ -1,12 +1,10 @@
|
||||
#![allow(clippy::type_complexity)]
|
||||
#![allow(clippy::cognitive_complexity)]
|
||||
|
||||
use super::methods::{
|
||||
GoodbyeReason, RPCCodedResponse, RPCResponseErrorCode, RequestId, ResponseTermination,
|
||||
};
|
||||
use super::methods::{GoodbyeReason, RPCCodedResponse, RPCResponseErrorCode, ResponseTermination};
|
||||
use super::outbound::OutboundRequestContainer;
|
||||
use super::protocol::{max_rpc_size, InboundRequest, Protocol, RPCError, RPCProtocol};
|
||||
use super::{RPCReceived, RPCSend};
|
||||
use super::{RPCReceived, RPCSend, ReqId};
|
||||
use crate::rpc::outbound::{OutboundFramed, OutboundRequest};
|
||||
use crate::rpc::protocol::InboundFramed;
|
||||
use fnv::FnvHashMap;
|
||||
@ -49,11 +47,11 @@ pub struct SubstreamId(usize);
|
||||
type InboundSubstream<TSpec> = InboundFramed<NegotiatedSubstream, TSpec>;
|
||||
|
||||
/// Events the handler emits to the behaviour.
|
||||
type HandlerEvent<T> = Result<RPCReceived<T>, HandlerErr>;
|
||||
pub type HandlerEvent<Id, T> = Result<RPCReceived<Id, T>, HandlerErr<Id>>;
|
||||
|
||||
/// An error encountered by the handler.
|
||||
#[derive(Debug)]
|
||||
pub enum HandlerErr {
|
||||
pub enum HandlerErr<Id> {
|
||||
/// An error occurred for this peer's request. This can occur during protocol negotiation,
|
||||
/// message passing, or if the handler identifies that we are sending an error response to the peer.
|
||||
Inbound {
|
||||
@ -69,7 +67,7 @@ pub enum HandlerErr {
|
||||
/// indicates an error.
|
||||
Outbound {
|
||||
/// Application-given Id of the request for which an error occurred.
|
||||
id: RequestId,
|
||||
id: Id,
|
||||
/// Information of the protocol.
|
||||
proto: Protocol,
|
||||
/// The error that occurred.
|
||||
@ -78,7 +76,7 @@ pub enum HandlerErr {
|
||||
}
|
||||
|
||||
/// Implementation of `ConnectionHandler` for the RPC protocol.
|
||||
pub struct RPCHandler<TSpec>
|
||||
pub struct RPCHandler<Id, TSpec>
|
||||
where
|
||||
TSpec: EthSpec,
|
||||
{
|
||||
@ -86,10 +84,10 @@ where
|
||||
listen_protocol: SubstreamProtocol<RPCProtocol<TSpec>, ()>,
|
||||
|
||||
/// Queue of events to produce in `poll()`.
|
||||
events_out: SmallVec<[HandlerEvent<TSpec>; 4]>,
|
||||
events_out: SmallVec<[HandlerEvent<Id, TSpec>; 4]>,
|
||||
|
||||
/// Queue of outbound substreams to open.
|
||||
dial_queue: SmallVec<[(RequestId, OutboundRequest<TSpec>); 4]>,
|
||||
dial_queue: SmallVec<[(Id, OutboundRequest<TSpec>); 4]>,
|
||||
|
||||
/// Current number of concurrent outbound substreams being opened.
|
||||
dial_negotiated: u32,
|
||||
@ -101,7 +99,7 @@ where
|
||||
inbound_substreams_delay: DelayQueue<SubstreamId>,
|
||||
|
||||
/// Map of outbound substreams that need to be driven to completion.
|
||||
outbound_substreams: FnvHashMap<SubstreamId, OutboundInfo<TSpec>>,
|
||||
outbound_substreams: FnvHashMap<SubstreamId, OutboundInfo<Id, TSpec>>,
|
||||
|
||||
/// Inbound substream `DelayQueue` which keeps track of when an inbound substream will timeout.
|
||||
outbound_substreams_delay: DelayQueue<SubstreamId>,
|
||||
@ -163,7 +161,7 @@ struct InboundInfo<TSpec: EthSpec> {
|
||||
}
|
||||
|
||||
/// Contains the information the handler keeps on established outbound substreams.
|
||||
struct OutboundInfo<TSpec: EthSpec> {
|
||||
struct OutboundInfo<Id, TSpec: EthSpec> {
|
||||
/// State of the substream.
|
||||
state: OutboundSubstreamState<TSpec>,
|
||||
/// Key to keep track of the substream's timeout via `self.outbound_substreams_delay`.
|
||||
@ -172,8 +170,8 @@ struct OutboundInfo<TSpec: EthSpec> {
|
||||
proto: Protocol,
|
||||
/// Number of chunks to be seen from the peer's response.
|
||||
remaining_chunks: Option<u64>,
|
||||
/// `RequestId` as given by the application that sent the request.
|
||||
req_id: RequestId,
|
||||
/// `Id` as given by the application that sent the request.
|
||||
req_id: Id,
|
||||
}
|
||||
|
||||
/// State of an inbound substream connection.
|
||||
@ -204,7 +202,7 @@ pub enum OutboundSubstreamState<TSpec: EthSpec> {
|
||||
Poisoned,
|
||||
}
|
||||
|
||||
impl<TSpec> RPCHandler<TSpec>
|
||||
impl<Id, TSpec> RPCHandler<Id, TSpec>
|
||||
where
|
||||
TSpec: EthSpec,
|
||||
{
|
||||
@ -235,7 +233,7 @@ where
|
||||
|
||||
/// Initiates the handler's shutdown process, sending an optional Goodbye message to the
|
||||
/// peer.
|
||||
fn shutdown(&mut self, goodbye_reason: Option<GoodbyeReason>) {
|
||||
fn shutdown(&mut self, goodbye_reason: Option<(Id, GoodbyeReason)>) {
|
||||
if matches!(self.state, HandlerState::Active) {
|
||||
if !self.dial_queue.is_empty() {
|
||||
debug!(self.log, "Starting handler shutdown"; "unsent_queued_requests" => self.dial_queue.len());
|
||||
@ -250,9 +248,8 @@ where
|
||||
}
|
||||
|
||||
// Queue our goodbye message.
|
||||
if let Some(reason) = goodbye_reason {
|
||||
self.dial_queue
|
||||
.push((RequestId::Router, OutboundRequest::Goodbye(reason)));
|
||||
if let Some((id, reason)) = goodbye_reason {
|
||||
self.dial_queue.push((id, OutboundRequest::Goodbye(reason)));
|
||||
}
|
||||
|
||||
self.state = HandlerState::ShuttingDown(Box::new(sleep_until(
|
||||
@ -262,7 +259,7 @@ where
|
||||
}
|
||||
|
||||
/// Opens an outbound substream with a request.
|
||||
fn send_request(&mut self, id: RequestId, req: OutboundRequest<TSpec>) {
|
||||
fn send_request(&mut self, id: Id, req: OutboundRequest<TSpec>) {
|
||||
match self.state {
|
||||
HandlerState::Active => {
|
||||
self.dial_queue.push((id, req));
|
||||
@ -310,16 +307,17 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
impl<TSpec> ConnectionHandler for RPCHandler<TSpec>
|
||||
impl<Id, TSpec> ConnectionHandler for RPCHandler<Id, TSpec>
|
||||
where
|
||||
TSpec: EthSpec,
|
||||
Id: ReqId,
|
||||
{
|
||||
type InEvent = RPCSend<TSpec>;
|
||||
type OutEvent = HandlerEvent<TSpec>;
|
||||
type InEvent = RPCSend<Id, TSpec>;
|
||||
type OutEvent = HandlerEvent<Id, TSpec>;
|
||||
type Error = RPCError;
|
||||
type InboundProtocol = RPCProtocol<TSpec>;
|
||||
type OutboundProtocol = OutboundRequestContainer<TSpec>;
|
||||
type OutboundOpenInfo = (RequestId, OutboundRequest<TSpec>); // Keep track of the id and the request
|
||||
type OutboundOpenInfo = (Id, OutboundRequest<TSpec>); // Keep track of the id and the request
|
||||
type InboundOpenInfo = ();
|
||||
|
||||
fn listen_protocol(&self) -> SubstreamProtocol<Self::InboundProtocol, ()> {
|
||||
@ -432,7 +430,7 @@ where
|
||||
match rpc_event {
|
||||
RPCSend::Request(id, req) => self.send_request(id, req),
|
||||
RPCSend::Response(inbound_id, response) => self.send_response(inbound_id, response),
|
||||
RPCSend::Shutdown(reason) => self.shutdown(Some(reason)),
|
||||
RPCSend::Shutdown(id, reason) => self.shutdown(Some((id, reason))),
|
||||
}
|
||||
// In any case, we need the handler to process the event.
|
||||
if let Some(waker) = &self.waker {
|
||||
|
@ -56,17 +56,6 @@ impl ToString for ErrorType {
|
||||
|
||||
/* Requests */
|
||||
|
||||
/// 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.
|
||||
#[derive(Encode, Decode, Clone, Debug, PartialEq)]
|
||||
pub struct StatusMessage {
|
||||
@ -432,18 +421,3 @@ impl slog::KV for StatusMessage {
|
||||
slog::Result::Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
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),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -5,13 +5,13 @@
|
||||
//! syncing.
|
||||
|
||||
use futures::future::FutureExt;
|
||||
use handler::RPCHandler;
|
||||
use libp2p::core::{connection::ConnectionId, ConnectedPoint};
|
||||
use handler::{HandlerEvent, RPCHandler};
|
||||
use libp2p::core::connection::ConnectionId;
|
||||
use libp2p::swarm::{
|
||||
handler::ConnectionHandler, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler,
|
||||
PollParameters, SubstreamProtocol,
|
||||
};
|
||||
use libp2p::{Multiaddr, PeerId};
|
||||
use libp2p::PeerId;
|
||||
use rate_limiter::{RPCRateLimiter as RateLimiter, RPCRateLimiterBuilder, RateLimitedErr};
|
||||
use slog::{crit, debug, o};
|
||||
use std::marker::PhantomData;
|
||||
@ -27,7 +27,7 @@ pub(crate) use protocol::{InboundRequest, RPCProtocol};
|
||||
pub use handler::SubstreamId;
|
||||
pub use methods::{
|
||||
BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, MaxRequestBlocks,
|
||||
RPCResponseErrorCode, RequestId, ResponseTermination, StatusMessage, MAX_REQUEST_BLOCKS,
|
||||
RPCResponseErrorCode, ResponseTermination, StatusMessage, MAX_REQUEST_BLOCKS,
|
||||
};
|
||||
pub(crate) use outbound::OutboundRequest;
|
||||
pub use protocol::{max_rpc_size, Protocol, RPCError};
|
||||
@ -39,14 +39,18 @@ mod outbound;
|
||||
mod protocol;
|
||||
mod rate_limiter;
|
||||
|
||||
/// Composite trait for a request id.
|
||||
pub trait ReqId: Send + 'static + std::fmt::Debug + Copy + Clone {}
|
||||
impl<T> ReqId for T where T: Send + 'static + std::fmt::Debug + Copy + Clone {}
|
||||
|
||||
/// RPC events sent from Lighthouse.
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum RPCSend<TSpec: EthSpec> {
|
||||
pub enum RPCSend<Id, TSpec: EthSpec> {
|
||||
/// A request sent from Lighthouse.
|
||||
///
|
||||
/// The `RequestId` is given by the application making the request. These
|
||||
/// The `Id` is given by the application making the request. These
|
||||
/// go over *outbound* connections.
|
||||
Request(RequestId, OutboundRequest<TSpec>),
|
||||
Request(Id, OutboundRequest<TSpec>),
|
||||
/// A response sent from Lighthouse.
|
||||
///
|
||||
/// The `SubstreamId` must correspond to the RPC-given ID of the original request received from the
|
||||
@ -54,12 +58,12 @@ pub enum RPCSend<TSpec: EthSpec> {
|
||||
/// connections.
|
||||
Response(SubstreamId, RPCCodedResponse<TSpec>),
|
||||
/// Lighthouse has requested to terminate the connection with a goodbye message.
|
||||
Shutdown(GoodbyeReason),
|
||||
Shutdown(Id, GoodbyeReason),
|
||||
}
|
||||
|
||||
/// RPC events received from outside Lighthouse.
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum RPCReceived<T: EthSpec> {
|
||||
pub enum RPCReceived<Id, T: EthSpec> {
|
||||
/// A request received from the outside.
|
||||
///
|
||||
/// The `SubstreamId` is given by the `RPCHandler` as it identifies this request with the
|
||||
@ -67,47 +71,47 @@ pub enum RPCReceived<T: EthSpec> {
|
||||
Request(SubstreamId, InboundRequest<T>),
|
||||
/// A response received from the outside.
|
||||
///
|
||||
/// The `RequestId` corresponds to the application given ID of the original request sent to the
|
||||
/// The `Id` 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>),
|
||||
Response(Id, RPCResponse<T>),
|
||||
/// Marks a request as completed
|
||||
EndOfStream(RequestId, ResponseTermination),
|
||||
EndOfStream(Id, ResponseTermination),
|
||||
}
|
||||
|
||||
impl<T: EthSpec> std::fmt::Display for RPCSend<T> {
|
||||
impl<T: EthSpec, Id: std::fmt::Debug> std::fmt::Display for RPCSend<Id, 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),
|
||||
RPCSend::Shutdown(reason) => write!(f, "Sending Goodbye: {}", reason),
|
||||
RPCSend::Shutdown(_id, reason) => write!(f, "Sending Goodbye: {}", reason),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Messages sent to the user from the RPC protocol.
|
||||
pub struct RPCMessage<TSpec: EthSpec> {
|
||||
pub struct RPCMessage<Id, TSpec: EthSpec> {
|
||||
/// The peer that sent the message.
|
||||
pub peer_id: PeerId,
|
||||
/// Handler managing this message.
|
||||
pub conn_id: ConnectionId,
|
||||
/// The message that was sent.
|
||||
pub event: <RPCHandler<TSpec> as ConnectionHandler>::OutEvent,
|
||||
pub event: HandlerEvent<Id, TSpec>,
|
||||
}
|
||||
|
||||
/// Implements the libp2p `NetworkBehaviour` trait and therefore manages network-level
|
||||
/// logic.
|
||||
pub struct RPC<TSpec: EthSpec> {
|
||||
pub struct RPC<Id: ReqId, TSpec: EthSpec> {
|
||||
/// Rate limiter
|
||||
limiter: RateLimiter,
|
||||
/// Queue of events to be processed.
|
||||
events: Vec<NetworkBehaviourAction<RPCMessage<TSpec>, RPCHandler<TSpec>>>,
|
||||
events: Vec<NetworkBehaviourAction<RPCMessage<Id, TSpec>, RPCHandler<Id, TSpec>>>,
|
||||
fork_context: Arc<ForkContext>,
|
||||
/// Slog logger for RPC behaviour.
|
||||
log: slog::Logger,
|
||||
}
|
||||
|
||||
impl<TSpec: EthSpec> RPC<TSpec> {
|
||||
impl<Id: ReqId, TSpec: EthSpec> RPC<Id, TSpec> {
|
||||
pub fn new(fork_context: Arc<ForkContext>, log: slog::Logger) -> Self {
|
||||
let log = log.new(o!("service" => "libp2p_rpc"));
|
||||
let limiter = RPCRateLimiterBuilder::new()
|
||||
@ -150,12 +154,7 @@ impl<TSpec: EthSpec> RPC<TSpec> {
|
||||
/// Submits an RPC request.
|
||||
///
|
||||
/// The peer must be connected for this to succeed.
|
||||
pub fn send_request(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
request_id: RequestId,
|
||||
event: OutboundRequest<TSpec>,
|
||||
) {
|
||||
pub fn send_request(&mut self, peer_id: PeerId, request_id: Id, event: OutboundRequest<TSpec>) {
|
||||
self.events.push(NetworkBehaviourAction::NotifyHandler {
|
||||
peer_id,
|
||||
handler: NotifyHandler::Any,
|
||||
@ -165,21 +164,22 @@ impl<TSpec: EthSpec> RPC<TSpec> {
|
||||
|
||||
/// Lighthouse wishes to disconnect from this peer by sending a Goodbye message. This
|
||||
/// gracefully terminates the RPC behaviour with a goodbye message.
|
||||
pub fn shutdown(&mut self, peer_id: PeerId, reason: GoodbyeReason) {
|
||||
pub fn shutdown(&mut self, peer_id: PeerId, id: Id, reason: GoodbyeReason) {
|
||||
self.events.push(NetworkBehaviourAction::NotifyHandler {
|
||||
peer_id,
|
||||
handler: NotifyHandler::Any,
|
||||
event: RPCSend::Shutdown(reason),
|
||||
event: RPCSend::Shutdown(id, reason),
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
impl<TSpec> NetworkBehaviour for RPC<TSpec>
|
||||
impl<Id, TSpec> NetworkBehaviour for RPC<Id, TSpec>
|
||||
where
|
||||
TSpec: EthSpec,
|
||||
Id: ReqId,
|
||||
{
|
||||
type ConnectionHandler = RPCHandler<TSpec>;
|
||||
type OutEvent = RPCMessage<TSpec>;
|
||||
type ConnectionHandler = RPCHandler<Id, TSpec>;
|
||||
type OutEvent = RPCMessage<Id, TSpec>;
|
||||
|
||||
fn new_handler(&mut self) -> Self::ConnectionHandler {
|
||||
RPCHandler::new(
|
||||
@ -196,33 +196,6 @@ where
|
||||
)
|
||||
}
|
||||
|
||||
// handled by discovery
|
||||
fn addresses_of_peer(&mut self, _peer_id: &PeerId) -> Vec<Multiaddr> {
|
||||
Vec::new()
|
||||
}
|
||||
|
||||
// Use connection established/closed instead of these currently
|
||||
fn inject_connection_established(
|
||||
&mut self,
|
||||
peer_id: &PeerId,
|
||||
_connection_id: &ConnectionId,
|
||||
_endpoint: &ConnectedPoint,
|
||||
_failed_addresses: Option<&Vec<Multiaddr>>,
|
||||
other_established: usize,
|
||||
) {
|
||||
if other_established == 0 {
|
||||
// find the peer's meta-data
|
||||
debug!(self.log, "Requesting new peer's metadata"; "peer_id" => %peer_id);
|
||||
let rpc_event =
|
||||
RPCSend::Request(RequestId::Behaviour, OutboundRequest::MetaData(PhantomData));
|
||||
self.events.push(NetworkBehaviourAction::NotifyHandler {
|
||||
peer_id: *peer_id,
|
||||
handler: NotifyHandler::Any,
|
||||
event: rpc_event,
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
fn inject_event(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
|
@ -4,9 +4,7 @@ use crate::behaviour::{
|
||||
use crate::config::NetworkLoad;
|
||||
use crate::discovery::enr;
|
||||
use crate::multiaddr::Protocol;
|
||||
use crate::rpc::{
|
||||
GoodbyeReason, MetaData, MetaDataV1, MetaDataV2, RPCResponseErrorCode, RequestId,
|
||||
};
|
||||
use crate::rpc::{GoodbyeReason, MetaData, MetaDataV1, MetaDataV2, RPCResponseErrorCode, ReqId};
|
||||
use crate::types::{error, EnrAttestationBitfield, EnrSyncCommitteeBitfield, GossipKind};
|
||||
use crate::EnrExt;
|
||||
use crate::{NetworkConfig, NetworkGlobals, PeerAction, ReportSource};
|
||||
@ -42,9 +40,9 @@ pub const METADATA_FILENAME: &str = "metadata";
|
||||
///
|
||||
/// This is a subset of the events that a libp2p swarm emits.
|
||||
#[derive(Debug)]
|
||||
pub enum Libp2pEvent<TSpec: EthSpec> {
|
||||
pub enum Libp2pEvent<AppReqId: ReqId, TSpec: EthSpec> {
|
||||
/// A behaviour event
|
||||
Behaviour(BehaviourEvent<TSpec>),
|
||||
Behaviour(BehaviourEvent<AppReqId, TSpec>),
|
||||
/// A new listening address has been established.
|
||||
NewListenAddr(Multiaddr),
|
||||
/// We reached zero listening addresses.
|
||||
@ -52,9 +50,9 @@ pub enum Libp2pEvent<TSpec: EthSpec> {
|
||||
}
|
||||
|
||||
/// The configuration and state of the libp2p components for the beacon node.
|
||||
pub struct Service<TSpec: EthSpec> {
|
||||
pub struct Service<AppReqId: ReqId, TSpec: EthSpec> {
|
||||
/// The libp2p Swarm handler.
|
||||
pub swarm: Swarm<Behaviour<TSpec>>,
|
||||
pub swarm: Swarm<Behaviour<AppReqId, TSpec>>,
|
||||
/// The bandwidth logger for the underlying libp2p transport.
|
||||
pub bandwidth: Arc<BandwidthSinks>,
|
||||
/// This node's PeerId.
|
||||
@ -71,7 +69,7 @@ pub struct Context<'a> {
|
||||
pub gossipsub_registry: Option<&'a mut Registry>,
|
||||
}
|
||||
|
||||
impl<TSpec: EthSpec> Service<TSpec> {
|
||||
impl<AppReqId: ReqId, TSpec: EthSpec> Service<AppReqId, TSpec> {
|
||||
pub async fn new(
|
||||
executor: task_executor::TaskExecutor,
|
||||
ctx: Context<'_>,
|
||||
@ -260,7 +258,7 @@ impl<TSpec: EthSpec> Service<TSpec> {
|
||||
}
|
||||
|
||||
/// Sends a request to a peer, with a given Id.
|
||||
pub fn send_request(&mut self, peer_id: PeerId, request_id: RequestId, request: Request) {
|
||||
pub fn send_request(&mut self, peer_id: PeerId, request_id: AppReqId, request: Request) {
|
||||
self.swarm
|
||||
.behaviour_mut()
|
||||
.send_request(peer_id, request_id, request);
|
||||
@ -307,7 +305,7 @@ impl<TSpec: EthSpec> Service<TSpec> {
|
||||
.send_successful_response(peer_id, id, response);
|
||||
}
|
||||
|
||||
pub async fn next_event(&mut self) -> Libp2pEvent<TSpec> {
|
||||
pub async fn next_event(&mut self) -> Libp2pEvent<AppReqId, TSpec> {
|
||||
loop {
|
||||
match self.swarm.select_next_some().await {
|
||||
SwarmEvent::Behaviour(behaviour) => {
|
||||
|
@ -21,6 +21,8 @@ pub mod behaviour;
|
||||
pub mod swarm;
|
||||
|
||||
type E = MinimalEthSpec;
|
||||
type ReqId = usize;
|
||||
|
||||
use tempfile::Builder as TempBuilder;
|
||||
|
||||
/// Returns a dummy fork context
|
||||
@ -33,10 +35,10 @@ pub fn fork_context() -> ForkContext {
|
||||
ForkContext::new::<E>(types::Slot::new(0), Hash256::zero(), &chain_spec)
|
||||
}
|
||||
|
||||
pub struct Libp2pInstance(LibP2PService<E>, exit_future::Signal);
|
||||
pub struct Libp2pInstance(LibP2PService<ReqId, E>, exit_future::Signal);
|
||||
|
||||
impl std::ops::Deref for Libp2pInstance {
|
||||
type Target = LibP2PService<E>;
|
||||
type Target = LibP2PService<ReqId, E>;
|
||||
fn deref(&self) -> &Self::Target {
|
||||
&self.0
|
||||
}
|
||||
@ -113,7 +115,7 @@ pub async fn build_libp2p_instance(
|
||||
}
|
||||
|
||||
#[allow(dead_code)]
|
||||
pub fn get_enr(node: &LibP2PService<E>) -> Enr {
|
||||
pub fn get_enr(node: &LibP2PService<ReqId, E>) -> Enr {
|
||||
node.swarm.behaviour().local_enr()
|
||||
}
|
||||
|
||||
|
@ -88,15 +88,14 @@ fn test_status_rpc() {
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::PeerConnectedOutgoing(peer_id)) => {
|
||||
// Send a STATUS message
|
||||
debug!(log, "Sending RPC");
|
||||
sender.swarm.behaviour_mut().send_request(
|
||||
peer_id,
|
||||
RequestId::Sync(10),
|
||||
rpc_request.clone(),
|
||||
);
|
||||
sender
|
||||
.swarm
|
||||
.behaviour_mut()
|
||||
.send_request(peer_id, 10, rpc_request.clone());
|
||||
}
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
|
||||
peer_id: _,
|
||||
id: RequestId::Sync(10),
|
||||
id: 10,
|
||||
response,
|
||||
}) => {
|
||||
// Should receive the RPC response
|
||||
@ -186,7 +185,7 @@ fn test_blocks_by_range_chunked_rpc() {
|
||||
|
||||
// keep count of the number of messages received
|
||||
let mut messages_received = 0;
|
||||
let request_id = RequestId::Sync(messages_to_send as usize);
|
||||
let request_id = messages_to_send as usize;
|
||||
// build the sender future
|
||||
let sender_future = async {
|
||||
loop {
|
||||
@ -313,7 +312,7 @@ fn test_blocks_by_range_over_limit() {
|
||||
let signed_full_block = SignedBeaconBlock::from_block(full_block, Signature::empty());
|
||||
let rpc_response_merge_large = Response::BlocksByRange(Some(Box::new(signed_full_block)));
|
||||
|
||||
let request_id = RequestId::Sync(messages_to_send as usize);
|
||||
let request_id = messages_to_send as usize;
|
||||
// build the sender future
|
||||
let sender_future = async {
|
||||
loop {
|
||||
@ -413,7 +412,7 @@ fn test_blocks_by_range_chunked_rpc_terminates_correctly() {
|
||||
|
||||
// keep count of the number of messages received
|
||||
let mut messages_received: u64 = 0;
|
||||
let request_id = RequestId::Sync(messages_to_send as usize);
|
||||
let request_id = messages_to_send as usize;
|
||||
// build the sender future
|
||||
let sender_future = async {
|
||||
loop {
|
||||
@ -553,15 +552,14 @@ fn test_blocks_by_range_single_empty_rpc() {
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::PeerConnectedOutgoing(peer_id)) => {
|
||||
// Send a STATUS message
|
||||
debug!(log, "Sending RPC");
|
||||
sender.swarm.behaviour_mut().send_request(
|
||||
peer_id,
|
||||
RequestId::Sync(10),
|
||||
rpc_request.clone(),
|
||||
);
|
||||
sender
|
||||
.swarm
|
||||
.behaviour_mut()
|
||||
.send_request(peer_id, 10, rpc_request.clone());
|
||||
}
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
|
||||
peer_id: _,
|
||||
id: RequestId::Sync(10),
|
||||
id: 10,
|
||||
response,
|
||||
}) => match response {
|
||||
Response::BlocksByRange(Some(_)) => {
|
||||
@ -679,15 +677,14 @@ fn test_blocks_by_root_chunked_rpc() {
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::PeerConnectedOutgoing(peer_id)) => {
|
||||
// Send a STATUS message
|
||||
debug!(log, "Sending RPC");
|
||||
sender.swarm.behaviour_mut().send_request(
|
||||
peer_id,
|
||||
RequestId::Sync(6),
|
||||
rpc_request.clone(),
|
||||
);
|
||||
sender
|
||||
.swarm
|
||||
.behaviour_mut()
|
||||
.send_request(peer_id, 6, rpc_request.clone());
|
||||
}
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
|
||||
peer_id: _,
|
||||
id: RequestId::Sync(6),
|
||||
id: 6,
|
||||
response,
|
||||
}) => match response {
|
||||
Response::BlocksByRoot(Some(_)) => {
|
||||
@ -814,15 +811,14 @@ fn test_blocks_by_root_chunked_rpc_terminates_correctly() {
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::PeerConnectedOutgoing(peer_id)) => {
|
||||
// Send a STATUS message
|
||||
debug!(log, "Sending RPC");
|
||||
sender.swarm.behaviour_mut().send_request(
|
||||
peer_id,
|
||||
RequestId::Sync(10),
|
||||
rpc_request.clone(),
|
||||
);
|
||||
sender
|
||||
.swarm
|
||||
.behaviour_mut()
|
||||
.send_request(peer_id, 10, rpc_request.clone());
|
||||
}
|
||||
Libp2pEvent::Behaviour(BehaviourEvent::ResponseReceived {
|
||||
peer_id: _,
|
||||
id: RequestId::Sync(10),
|
||||
id: 10,
|
||||
response,
|
||||
}) => {
|
||||
debug!(log, "Sender received a response");
|
||||
|
@ -8,12 +8,11 @@
|
||||
mod processor;
|
||||
|
||||
use crate::error;
|
||||
use crate::service::NetworkMessage;
|
||||
use crate::service::{NetworkMessage, RequestId};
|
||||
use beacon_chain::{BeaconChain, BeaconChainTypes};
|
||||
use futures::prelude::*;
|
||||
use lighthouse_network::{
|
||||
rpc::RequestId, MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, Request,
|
||||
Response,
|
||||
MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, Request, Response,
|
||||
};
|
||||
use processor::Processor;
|
||||
use slog::{debug, o, trace};
|
||||
|
@ -1,7 +1,8 @@
|
||||
use crate::beacon_processor::{
|
||||
BeaconProcessor, WorkEvent as BeaconWorkEvent, MAX_WORK_EVENT_QUEUE_LEN,
|
||||
};
|
||||
use crate::service::NetworkMessage;
|
||||
use crate::service::{NetworkMessage, RequestId};
|
||||
use crate::sync::manager::RequestId as SyncId;
|
||||
use crate::sync::SyncMessage;
|
||||
use beacon_chain::{BeaconChain, BeaconChainError, BeaconChainTypes};
|
||||
use lighthouse_network::rpc::*;
|
||||
@ -100,8 +101,11 @@ impl<T: BeaconChainTypes> Processor<T> {
|
||||
/// this function notifies the sync manager of the error.
|
||||
pub fn on_rpc_error(&mut self, peer_id: PeerId, request_id: RequestId) {
|
||||
// Check if the failed RPC belongs to sync
|
||||
if let RequestId::Sync(id) = request_id {
|
||||
self.send_to_sync(SyncMessage::RPCError(peer_id, id));
|
||||
if let RequestId::Sync(request_id) = request_id {
|
||||
self.send_to_sync(SyncMessage::RpcError {
|
||||
peer_id,
|
||||
request_id,
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
@ -176,24 +180,28 @@ impl<T: BeaconChainTypes> Processor<T> {
|
||||
request_id: RequestId,
|
||||
beacon_block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
|
||||
) {
|
||||
let request_id = match request_id {
|
||||
RequestId::Sync(sync_id) => match sync_id {
|
||||
SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. } => {
|
||||
unreachable!("Block lookups do not request BBRange requests")
|
||||
}
|
||||
id @ (SyncId::BackFillSync { .. } | SyncId::RangeSync { .. }) => id,
|
||||
},
|
||||
RequestId::Router => unreachable!("All BBRange requests belong to sync"),
|
||||
};
|
||||
|
||||
trace!(
|
||||
self.log,
|
||||
"Received BlocksByRange Response";
|
||||
"peer" => %peer_id,
|
||||
);
|
||||
|
||||
if let RequestId::Sync(id) = request_id {
|
||||
self.send_to_sync(SyncMessage::BlocksByRangeResponse {
|
||||
self.send_to_sync(SyncMessage::RpcBlock {
|
||||
peer_id,
|
||||
request_id: id,
|
||||
request_id,
|
||||
beacon_block,
|
||||
seen_timestamp: timestamp_now(),
|
||||
});
|
||||
} else {
|
||||
debug!(
|
||||
self.log,
|
||||
"All blocks by range responses should belong to sync"
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/// Handle a `BlocksByRoot` response from the peer.
|
||||
@ -203,25 +211,27 @@ impl<T: BeaconChainTypes> Processor<T> {
|
||||
request_id: RequestId,
|
||||
beacon_block: Option<Box<SignedBeaconBlock<T::EthSpec>>>,
|
||||
) {
|
||||
let request_id = match request_id {
|
||||
RequestId::Sync(sync_id) => match sync_id {
|
||||
id @ (SyncId::SingleBlock { .. } | SyncId::ParentLookup { .. }) => id,
|
||||
SyncId::BackFillSync { .. } | SyncId::RangeSync { .. } => {
|
||||
unreachable!("Batch syncing do not request BBRoot requests")
|
||||
}
|
||||
},
|
||||
RequestId::Router => unreachable!("All BBRoot requests belong to sync"),
|
||||
};
|
||||
|
||||
trace!(
|
||||
self.log,
|
||||
"Received BlocksByRoot Response";
|
||||
"peer" => %peer_id,
|
||||
);
|
||||
|
||||
if let RequestId::Sync(id) = request_id {
|
||||
self.send_to_sync(SyncMessage::BlocksByRootResponse {
|
||||
self.send_to_sync(SyncMessage::RpcBlock {
|
||||
peer_id,
|
||||
request_id: id,
|
||||
request_id,
|
||||
beacon_block,
|
||||
seen_timestamp: timestamp_now(),
|
||||
});
|
||||
} else {
|
||||
debug!(
|
||||
self.log,
|
||||
"All Blocks by Root responses should belong to sync"
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/// Process a gossip message declaring a new block.
|
||||
|
@ -1,3 +1,4 @@
|
||||
use super::sync::manager::RequestId as SyncId;
|
||||
use crate::persisted_dht::{clear_dht, load_dht, persist_dht};
|
||||
use crate::router::{Router, RouterMessage};
|
||||
use crate::subnet_service::SyncCommitteeService;
|
||||
@ -14,7 +15,7 @@ use lighthouse_network::{
|
||||
prometheus_client::registry::Registry, MessageAcceptance, Service as LibP2PService,
|
||||
};
|
||||
use lighthouse_network::{
|
||||
rpc::{GoodbyeReason, RPCResponseErrorCode, RequestId},
|
||||
rpc::{GoodbyeReason, RPCResponseErrorCode},
|
||||
Context, Libp2pEvent, PeerAction, PeerRequestId, PubsubMessage, ReportSource, Request,
|
||||
Response, Subnet,
|
||||
};
|
||||
@ -42,6 +43,13 @@ const SUBSCRIBE_DELAY_SLOTS: u64 = 2;
|
||||
/// Delay after a fork where we unsubscribe from pre-fork topics.
|
||||
const UNSUBSCRIBE_DELAY_EPOCHS: u64 = 2;
|
||||
|
||||
/// Application level requests sent to the network.
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
pub enum RequestId {
|
||||
Sync(SyncId),
|
||||
Router,
|
||||
}
|
||||
|
||||
/// Types of messages that the network service can receive.
|
||||
#[derive(Debug)]
|
||||
pub enum NetworkMessage<T: EthSpec> {
|
||||
@ -112,7 +120,7 @@ pub struct NetworkService<T: BeaconChainTypes> {
|
||||
/// A reference to the underlying beacon chain.
|
||||
beacon_chain: Arc<BeaconChain<T>>,
|
||||
/// The underlying libp2p service that drives all the network interactions.
|
||||
libp2p: LibP2PService<T::EthSpec>,
|
||||
libp2p: LibP2PService<RequestId, T::EthSpec>,
|
||||
/// An attestation and subnet manager service.
|
||||
attestation_service: AttestationService<T>,
|
||||
/// A sync committeee subnet manager service.
|
||||
@ -389,7 +397,7 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
||||
/// Handle an event received from the network.
|
||||
async fn on_libp2p_event(
|
||||
&mut self,
|
||||
ev: Libp2pEvent<T::EthSpec>,
|
||||
ev: Libp2pEvent<RequestId, T::EthSpec>,
|
||||
shutdown_sender: &mut Sender<ShutdownReason>,
|
||||
) {
|
||||
match ev {
|
||||
|
@ -8,9 +8,8 @@
|
||||
//! If a batch fails, the backfill sync cannot progress. In this scenario, we mark the backfill
|
||||
//! sync as failed, log an error and attempt to retry once a new peer joins the node.
|
||||
|
||||
use super::RequestId;
|
||||
use crate::beacon_processor::{ProcessId, WorkEvent as BeaconWorkEvent};
|
||||
use crate::sync::manager::BatchProcessResult;
|
||||
use crate::sync::manager::{BatchProcessResult, Id};
|
||||
use crate::sync::network_context::SyncNetworkContext;
|
||||
use crate::sync::range_sync::{BatchConfig, BatchId, BatchInfo, BatchState};
|
||||
use beacon_chain::{BeaconChain, BeaconChainTypes};
|
||||
@ -357,7 +356,7 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
|
||||
network: &mut SyncNetworkContext<T::EthSpec>,
|
||||
batch_id: BatchId,
|
||||
peer_id: &PeerId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
) -> Result<(), BackFillError> {
|
||||
if let Some(batch) = self.batches.get_mut(&batch_id) {
|
||||
// A batch could be retried without the peer failing the request (disconnecting/
|
||||
@ -392,7 +391,7 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
|
||||
network: &mut SyncNetworkContext<T::EthSpec>,
|
||||
batch_id: BatchId,
|
||||
peer_id: &PeerId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
|
||||
) -> Result<ProcessResult, BackFillError> {
|
||||
// check if we have this batch
|
||||
|
@ -37,7 +37,6 @@ use super::backfill_sync::{BackFillSync, ProcessResult, SyncStart};
|
||||
use super::network_context::SyncNetworkContext;
|
||||
use super::peer_sync_info::{remote_sync_type, PeerSyncType};
|
||||
use super::range_sync::{ChainId, RangeSync, RangeSyncType, EPOCHS_PER_BATCH};
|
||||
use super::RequestId;
|
||||
use crate::beacon_processor::{ProcessId, WorkEvent as BeaconWorkEvent};
|
||||
use crate::service::NetworkMessage;
|
||||
use crate::status::ToStatusMessage;
|
||||
@ -52,6 +51,7 @@ use slog::{crit, debug, error, info, trace, warn, Logger};
|
||||
use smallvec::SmallVec;
|
||||
use ssz_types::VariableList;
|
||||
use std::boxed::Box;
|
||||
use std::collections::hash_map::Entry;
|
||||
use std::ops::Sub;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
@ -73,23 +73,31 @@ const PARENT_FAIL_TOLERANCE: usize = 5;
|
||||
/// is further back than the most recent head slot.
|
||||
const PARENT_DEPTH_TOLERANCE: usize = SLOT_IMPORT_TOLERANCE * 2;
|
||||
|
||||
pub type Id = u32;
|
||||
|
||||
/// Id of rpc requests sent by sync to the network.
|
||||
#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)]
|
||||
pub enum RequestId {
|
||||
/// Request searching for a block given a hash.
|
||||
SingleBlock { id: Id },
|
||||
/// Request searching for a block's parent. The id is the chain
|
||||
ParentLookup { id: Id },
|
||||
/// Request was from the backfill sync algorithm.
|
||||
BackFillSync { id: Id },
|
||||
/// The request was from a chain in the range sync algorithm.
|
||||
RangeSync { id: Id },
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
/// A message than can be sent to the sync manager thread.
|
||||
pub enum SyncMessage<T: EthSpec> {
|
||||
/// A useful peer has been discovered.
|
||||
AddPeer(PeerId, SyncInfo),
|
||||
|
||||
/// A [`BlocksByRange`] response has been received.
|
||||
BlocksByRangeResponse {
|
||||
peer_id: PeerId,
|
||||
/// A block has been received from the RPC.
|
||||
RpcBlock {
|
||||
request_id: RequestId,
|
||||
beacon_block: Option<Box<SignedBeaconBlock<T>>>,
|
||||
},
|
||||
|
||||
/// A [`BlocksByRoot`] response has been received.
|
||||
BlocksByRootResponse {
|
||||
peer_id: PeerId,
|
||||
request_id: RequestId,
|
||||
beacon_block: Option<Box<SignedBeaconBlock<T>>>,
|
||||
seen_timestamp: Duration,
|
||||
},
|
||||
@ -105,7 +113,10 @@ pub enum SyncMessage<T: EthSpec> {
|
||||
Disconnect(PeerId),
|
||||
|
||||
/// An RPC Error has occurred on a request.
|
||||
RPCError(PeerId, RequestId),
|
||||
RpcError {
|
||||
peer_id: PeerId,
|
||||
request_id: RequestId,
|
||||
},
|
||||
|
||||
/// A batch has been processed by the block processor thread.
|
||||
BatchProcessed {
|
||||
@ -157,7 +168,7 @@ struct ParentRequests<T: EthSpec> {
|
||||
last_submitted_peer: PeerId,
|
||||
|
||||
/// The request ID of this lookup is in progress.
|
||||
pending: Option<RequestId>,
|
||||
pending: Option<Id>,
|
||||
}
|
||||
|
||||
/// The primary object for handling and driving all the current syncing logic. It maintains the
|
||||
@ -193,7 +204,7 @@ pub struct SyncManager<T: BeaconChainTypes> {
|
||||
/// received or not.
|
||||
///
|
||||
/// The flag allows us to determine if the peer returned data or sent us nothing.
|
||||
single_block_lookups: FnvHashMap<RequestId, SingleBlockRequest>,
|
||||
single_block_lookups: FnvHashMap<Id, SingleBlockRequest>,
|
||||
|
||||
/// A multi-threaded, non-blocking processor for applying messages to the beacon chain.
|
||||
beacon_processor_send: mpsc::Sender<BeaconWorkEvent<T>>,
|
||||
@ -313,46 +324,31 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
/// There are two reasons we could have received a BlocksByRoot response
|
||||
/// - We requested a single hash and have received a response for the single_block_lookup
|
||||
/// - We are looking up parent blocks in parent lookup search
|
||||
async fn blocks_by_root_response(
|
||||
async fn parent_lookup_response(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
block: Option<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
_seen_timestamp: Duration,
|
||||
) {
|
||||
match block {
|
||||
Some(block) => {
|
||||
// data was returned, not just a stream termination
|
||||
|
||||
// check if this is a single block lookup - i.e we were searching for a specific hash
|
||||
let mut single_block_hash = None;
|
||||
if let Some(block_request) = self.single_block_lookups.get_mut(&request_id) {
|
||||
// update the state of the lookup indicating a block was received from the peer
|
||||
block_request.block_returned = true;
|
||||
single_block_hash = Some(block_request.hash);
|
||||
}
|
||||
if let Some(block_hash) = single_block_hash {
|
||||
self.single_block_lookup_response(peer_id, block, block_hash, seen_timestamp)
|
||||
.await;
|
||||
return;
|
||||
}
|
||||
|
||||
// This wasn't a single block lookup request, it must be a response to a parent request search
|
||||
// find the request
|
||||
let mut parent_request = match self
|
||||
let mut parent_request = if let Some(pos) = self
|
||||
.parent_queue
|
||||
.iter()
|
||||
.position(|request| request.pending == Some(request_id))
|
||||
{
|
||||
// we remove from the queue and process it. It will get re-added if required
|
||||
Some(pos) => self.parent_queue.remove(pos),
|
||||
None => {
|
||||
// No pending request, invalid request_id or coding error
|
||||
warn!(self.log, "BlocksByRoot response unknown"; "request_id" => request_id);
|
||||
return;
|
||||
self.parent_queue.remove(pos)
|
||||
} else {
|
||||
if block.is_some() {
|
||||
debug!(self.log, "Response for a parent lookup request that was not found"; "peer_id" => %peer_id);
|
||||
}
|
||||
return;
|
||||
};
|
||||
|
||||
match block {
|
||||
Some(block) => {
|
||||
// data was returned, not just a stream termination
|
||||
|
||||
// check if the parent of this block isn't in our failed cache. If it is, this
|
||||
// chain should be dropped and the peer downscored.
|
||||
if self.failed_chains.contains(&block.message().parent_root()) {
|
||||
@ -382,38 +378,6 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
self.process_parent_request(parent_request).await;
|
||||
}
|
||||
None => {
|
||||
// this is a stream termination
|
||||
|
||||
// stream termination for a single block lookup, remove the key
|
||||
if let Some(single_block_request) = self.single_block_lookups.remove(&request_id) {
|
||||
// The peer didn't respond with a block that it referenced.
|
||||
// This can be allowed as some clients may implement pruning. We mildly
|
||||
// tolerate this behaviour.
|
||||
if !single_block_request.block_returned {
|
||||
warn!(self.log, "Peer didn't respond with a block it referenced"; "referenced_block_hash" => %single_block_request.hash, "peer_id" => %peer_id);
|
||||
self.network.report_peer(
|
||||
peer_id,
|
||||
PeerAction::MidToleranceError,
|
||||
"bbroot_no_block",
|
||||
);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
// This wasn't a single block lookup request, it must be a response to a parent request search
|
||||
// find the request and remove it
|
||||
let mut parent_request = match self
|
||||
.parent_queue
|
||||
.iter()
|
||||
.position(|request| request.pending == Some(request_id))
|
||||
{
|
||||
Some(pos) => self.parent_queue.remove(pos),
|
||||
None => {
|
||||
// No pending request, the parent request has been processed and this is
|
||||
// the resulting stream termination.
|
||||
return;
|
||||
}
|
||||
};
|
||||
// An empty response has been returned to a parent request
|
||||
// if an empty response is given, the peer didn't have the requested block, try again
|
||||
parent_request.failed_attempts += 1;
|
||||
@ -458,13 +422,34 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
/// lookup search is started.
|
||||
async fn single_block_lookup_response(
|
||||
&mut self,
|
||||
request_id: Id,
|
||||
peer_id: PeerId,
|
||||
block: SignedBeaconBlock<T::EthSpec>,
|
||||
expected_block_hash: Hash256,
|
||||
block: Option<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
if let Entry::Occupied(mut entry) = self.single_block_lookups.entry(request_id) {
|
||||
match block {
|
||||
None => {
|
||||
// Stream termination. Remove the lookup
|
||||
let (_, single_block_request) = entry.remove_entry();
|
||||
// The peer didn't respond with a block that it referenced.
|
||||
// This can be allowed as some clients may implement pruning. We mildly
|
||||
// tolerate this behaviour.
|
||||
if !single_block_request.block_returned {
|
||||
warn!(self.log, "Peer didn't respond with a block it referenced";
|
||||
"referenced_block_hash" => %single_block_request.hash, "peer_id" => %peer_id);
|
||||
self.network.report_peer(
|
||||
peer_id,
|
||||
PeerAction::MidToleranceError,
|
||||
"bbroot_no_block",
|
||||
);
|
||||
}
|
||||
}
|
||||
Some(block) => {
|
||||
// update the state of the lookup indicating a block was received from the peer
|
||||
entry.get_mut().block_returned = true;
|
||||
// verify the hash is correct and try and process the block
|
||||
if expected_block_hash != block.canonical_root() {
|
||||
if entry.get().hash != block.canonical_root() {
|
||||
// The peer that sent this, sent us the wrong block.
|
||||
// We do not tolerate this behaviour. The peer is instantly disconnected and banned.
|
||||
warn!(self.log, "Peer sent incorrect block for single block lookup"; "peer_id" => %peer_id);
|
||||
@ -526,6 +511,9 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A block has been sent to us that has an unknown parent. This begins a parent lookup search
|
||||
/// to find the parent or chain of parents that match our current chain.
|
||||
@ -612,7 +600,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
block_roots: VariableList::from(vec![block_hash]),
|
||||
};
|
||||
|
||||
if let Ok(request_id) = self.network.blocks_by_root_request(peer_id, request) {
|
||||
if let Ok(request_id) = self.network.single_block_lookup_request(peer_id, request) {
|
||||
self.single_block_lookups
|
||||
.insert(request_id, SingleBlockRequest::new(block_hash));
|
||||
}
|
||||
@ -621,27 +609,47 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
/// Handles RPC errors related to requests that were emitted from the sync manager.
|
||||
fn inject_error(&mut self, peer_id: PeerId, request_id: RequestId) {
|
||||
trace!(self.log, "Sync manager received a failed RPC");
|
||||
// remove any single block lookups
|
||||
if self.single_block_lookups.remove(&request_id).is_some() {
|
||||
// this was a single block request lookup, look no further
|
||||
return;
|
||||
match request_id {
|
||||
RequestId::SingleBlock { id } => {
|
||||
self.single_block_lookups.remove(&id);
|
||||
}
|
||||
|
||||
// increment the failure of a parent lookup if the request matches a parent search
|
||||
RequestId::ParentLookup { id } => {
|
||||
if let Some(pos) = self
|
||||
.parent_queue
|
||||
.iter()
|
||||
.position(|request| request.pending == Some(request_id))
|
||||
.position(|request| request.pending == Some(id))
|
||||
{
|
||||
// increment the failure of a parent lookup if the request matches a parent search
|
||||
let mut parent_request = self.parent_queue.remove(pos);
|
||||
parent_request.failed_attempts += 1;
|
||||
parent_request.last_submitted_peer = peer_id;
|
||||
self.request_parent(parent_request);
|
||||
return;
|
||||
}
|
||||
|
||||
// Otherwise this error matches no known request.
|
||||
trace!(self.log, "Response/Error for non registered request"; "request_id" => request_id)
|
||||
}
|
||||
RequestId::BackFillSync { id } => {
|
||||
if let Some(batch_id) = self.network.backfill_sync_response(id, true) {
|
||||
match self
|
||||
.backfill_sync
|
||||
.inject_error(&mut self.network, batch_id, &peer_id, id)
|
||||
{
|
||||
Ok(_) => {}
|
||||
Err(_) => self.update_sync_state(),
|
||||
}
|
||||
}
|
||||
}
|
||||
RequestId::RangeSync { id } => {
|
||||
if let Some((chain_id, batch_id)) = self.network.range_sync_response(id, true) {
|
||||
self.range_sync.inject_error(
|
||||
&mut self.network,
|
||||
peer_id,
|
||||
batch_id,
|
||||
chain_id,
|
||||
id,
|
||||
);
|
||||
self.update_sync_state()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn peer_disconnect(&mut self, peer_id: &PeerId) {
|
||||
@ -978,7 +986,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
// guaranteed to have this chain of blocks.
|
||||
let peer_id = parent_request.last_submitted_peer;
|
||||
|
||||
if let Ok(request_id) = self.network.blocks_by_root_request(peer_id, request) {
|
||||
if let Ok(request_id) = self.network.parent_lookup_request(peer_id, request) {
|
||||
// if the request was successful add the queue back into self
|
||||
parent_request.pending = Some(request_id);
|
||||
self.parent_queue.push(parent_request);
|
||||
@ -994,59 +1002,15 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
SyncMessage::AddPeer(peer_id, info) => {
|
||||
self.add_peer(peer_id, info);
|
||||
}
|
||||
SyncMessage::BlocksByRangeResponse {
|
||||
SyncMessage::RpcBlock {
|
||||
request_id,
|
||||
peer_id,
|
||||
request_id,
|
||||
beacon_block,
|
||||
} => {
|
||||
let beacon_block = beacon_block.map(|b| *b);
|
||||
// Obtain which sync requested these blocks and divert accordingly.
|
||||
match self
|
||||
.network
|
||||
.blocks_by_range_response(request_id, beacon_block.is_none())
|
||||
{
|
||||
Some(SyncRequestType::RangeSync(batch_id, chain_id)) => {
|
||||
self.range_sync.blocks_by_range_response(
|
||||
&mut self.network,
|
||||
peer_id,
|
||||
chain_id,
|
||||
batch_id,
|
||||
request_id,
|
||||
beacon_block,
|
||||
);
|
||||
self.update_sync_state();
|
||||
}
|
||||
Some(SyncRequestType::BackFillSync(batch_id)) => {
|
||||
match self.backfill_sync.on_block_response(
|
||||
&mut self.network,
|
||||
batch_id,
|
||||
&peer_id,
|
||||
request_id,
|
||||
beacon_block,
|
||||
) {
|
||||
Ok(ProcessResult::SyncCompleted) => self.update_sync_state(),
|
||||
Ok(ProcessResult::Successful) => {}
|
||||
Err(_error) => {
|
||||
// The backfill sync has failed, errors are reported
|
||||
// within.
|
||||
self.update_sync_state();
|
||||
}
|
||||
}
|
||||
}
|
||||
None => {
|
||||
trace!(self.log, "Response/Error for non registered request"; "request_id" => request_id)
|
||||
}
|
||||
}
|
||||
}
|
||||
SyncMessage::BlocksByRootResponse {
|
||||
peer_id,
|
||||
request_id,
|
||||
beacon_block,
|
||||
seen_timestamp,
|
||||
} => {
|
||||
self.blocks_by_root_response(
|
||||
peer_id,
|
||||
self.rpc_block_received(
|
||||
request_id,
|
||||
peer_id,
|
||||
beacon_block.map(|b| *b),
|
||||
seen_timestamp,
|
||||
)
|
||||
@ -1061,38 +1025,10 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
SyncMessage::Disconnect(peer_id) => {
|
||||
self.peer_disconnect(&peer_id);
|
||||
}
|
||||
SyncMessage::RPCError(peer_id, request_id) => {
|
||||
// Redirect to a sync mechanism if the error is related to one of their
|
||||
// requests.
|
||||
match self.network.blocks_by_range_response(request_id, true) {
|
||||
Some(SyncRequestType::RangeSync(batch_id, chain_id)) => {
|
||||
self.range_sync.inject_error(
|
||||
&mut self.network,
|
||||
SyncMessage::RpcError {
|
||||
peer_id,
|
||||
batch_id,
|
||||
chain_id,
|
||||
request_id,
|
||||
);
|
||||
self.update_sync_state();
|
||||
}
|
||||
Some(SyncRequestType::BackFillSync(batch_id)) => {
|
||||
match self.backfill_sync.inject_error(
|
||||
&mut self.network,
|
||||
batch_id,
|
||||
&peer_id,
|
||||
request_id,
|
||||
) {
|
||||
Ok(_) => {}
|
||||
Err(_) => self.update_sync_state(),
|
||||
}
|
||||
}
|
||||
None => {
|
||||
// This is a request not belonging to a sync algorithm.
|
||||
// Process internally.
|
||||
self.inject_error(peer_id, request_id);
|
||||
}
|
||||
}
|
||||
}
|
||||
} => self.inject_error(peer_id, request_id),
|
||||
SyncMessage::BatchProcessed { sync_type, result } => match sync_type {
|
||||
SyncRequestType::RangeSync(epoch, chain_id) => {
|
||||
self.range_sync.handle_block_process_result(
|
||||
@ -1136,4 +1072,60 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn rpc_block_received(
|
||||
&mut self,
|
||||
request_id: RequestId,
|
||||
peer_id: PeerId,
|
||||
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
match request_id {
|
||||
RequestId::SingleBlock { id } => {
|
||||
self.single_block_lookup_response(id, peer_id, beacon_block, seen_timestamp)
|
||||
.await;
|
||||
}
|
||||
RequestId::ParentLookup { id } => {
|
||||
self.parent_lookup_response(peer_id, id, beacon_block, seen_timestamp)
|
||||
.await
|
||||
}
|
||||
RequestId::BackFillSync { id } => {
|
||||
if let Some(batch_id) = self
|
||||
.network
|
||||
.backfill_sync_response(id, beacon_block.is_none())
|
||||
{
|
||||
match self.backfill_sync.on_block_response(
|
||||
&mut self.network,
|
||||
batch_id,
|
||||
&peer_id,
|
||||
id,
|
||||
beacon_block,
|
||||
) {
|
||||
Ok(ProcessResult::SyncCompleted) => self.update_sync_state(),
|
||||
Ok(ProcessResult::Successful) => {}
|
||||
Err(_error) => {
|
||||
// The backfill sync has failed, errors are reported
|
||||
// within.
|
||||
self.update_sync_state();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
RequestId::RangeSync { id } => {
|
||||
if let Some((chain_id, batch_id)) =
|
||||
self.network.range_sync_response(id, beacon_block.is_none())
|
||||
{
|
||||
self.range_sync.blocks_by_range_response(
|
||||
&mut self.network,
|
||||
peer_id,
|
||||
chain_id,
|
||||
batch_id,
|
||||
id,
|
||||
beacon_block,
|
||||
);
|
||||
self.update_sync_state();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -9,6 +9,3 @@ mod range_sync;
|
||||
|
||||
pub use manager::{BatchProcessResult, SyncMessage};
|
||||
pub use range_sync::ChainId;
|
||||
|
||||
/// Type of id of rpc requests sent by sync
|
||||
pub type RequestId = usize;
|
||||
|
@ -1,15 +1,12 @@
|
||||
//! Provides network functionality for the Syncing thread. This fundamentally wraps a network
|
||||
//! channel and stores a global RPC ID to perform requests.
|
||||
|
||||
use super::manager::SyncRequestType;
|
||||
use super::manager::{Id, RequestId as SyncRequestId};
|
||||
use super::range_sync::{BatchId, ChainId};
|
||||
use super::RequestId as SyncRequestId;
|
||||
use crate::service::NetworkMessage;
|
||||
use crate::service::{NetworkMessage, RequestId};
|
||||
use crate::status::ToStatusMessage;
|
||||
use fnv::FnvHashMap;
|
||||
use lighthouse_network::rpc::{
|
||||
BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason, RequestId,
|
||||
};
|
||||
use lighthouse_network::rpc::{BlocksByRangeRequest, BlocksByRootRequest, GoodbyeReason};
|
||||
use lighthouse_network::{Client, NetworkGlobals, PeerAction, PeerId, ReportSource, Request};
|
||||
use slog::{debug, trace, warn};
|
||||
use std::sync::Arc;
|
||||
@ -26,10 +23,12 @@ pub struct SyncNetworkContext<T: EthSpec> {
|
||||
network_globals: Arc<NetworkGlobals<T>>,
|
||||
|
||||
/// A sequential ID for all RPC requests.
|
||||
request_id: SyncRequestId,
|
||||
request_id: Id,
|
||||
|
||||
/// BlocksByRange requests made by syncing algorithms.
|
||||
range_requests: FnvHashMap<SyncRequestId, SyncRequestType>,
|
||||
/// BlocksByRange requests made by the range syncing algorithm.
|
||||
range_requests: FnvHashMap<Id, (ChainId, BatchId)>,
|
||||
|
||||
backfill_requests: FnvHashMap<Id, BatchId>,
|
||||
|
||||
/// Logger for the `SyncNetworkContext`.
|
||||
log: slog::Logger,
|
||||
@ -46,6 +45,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
network_globals,
|
||||
request_id: 1,
|
||||
range_requests: FnvHashMap::default(),
|
||||
backfill_requests: FnvHashMap::default(),
|
||||
log,
|
||||
}
|
||||
}
|
||||
@ -78,7 +78,13 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
"head_slot" => %status_message.head_slot,
|
||||
);
|
||||
|
||||
let _ = self.send_rpc_request(peer_id, Request::Status(status_message.clone()));
|
||||
let request = Request::Status(status_message.clone());
|
||||
let request_id = RequestId::Router;
|
||||
let _ = self.send_network_msg(NetworkMessage::SendRequest {
|
||||
peer_id,
|
||||
request,
|
||||
request_id,
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -90,7 +96,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
request: BlocksByRangeRequest,
|
||||
chain_id: ChainId,
|
||||
batch_id: BatchId,
|
||||
) -> Result<SyncRequestId, &'static str> {
|
||||
) -> Result<Id, &'static str> {
|
||||
trace!(
|
||||
self.log,
|
||||
"Sending BlocksByRange Request";
|
||||
@ -98,10 +104,16 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
"count" => request.count,
|
||||
"peer" => %peer_id,
|
||||
);
|
||||
let req_id = self.send_rpc_request(peer_id, Request::BlocksByRange(request))?;
|
||||
self.range_requests
|
||||
.insert(req_id, SyncRequestType::RangeSync(batch_id, chain_id));
|
||||
Ok(req_id)
|
||||
let request = Request::BlocksByRange(request);
|
||||
let id = self.next_id();
|
||||
let request_id = RequestId::Sync(SyncRequestId::RangeSync { id });
|
||||
self.send_network_msg(NetworkMessage::SendRequest {
|
||||
peer_id,
|
||||
request,
|
||||
request_id,
|
||||
})?;
|
||||
self.range_requests.insert(id, (chain_id, batch_id));
|
||||
Ok(id)
|
||||
}
|
||||
|
||||
/// A blocks by range request sent by the backfill sync algorithm
|
||||
@ -110,7 +122,7 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
peer_id: PeerId,
|
||||
request: BlocksByRangeRequest,
|
||||
batch_id: BatchId,
|
||||
) -> Result<SyncRequestId, &'static str> {
|
||||
) -> Result<Id, &'static str> {
|
||||
trace!(
|
||||
self.log,
|
||||
"Sending backfill BlocksByRange Request";
|
||||
@ -118,21 +130,24 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
"count" => request.count,
|
||||
"peer" => %peer_id,
|
||||
);
|
||||
let req_id = self.send_rpc_request(peer_id, Request::BlocksByRange(request))?;
|
||||
self.range_requests
|
||||
.insert(req_id, SyncRequestType::BackFillSync(batch_id));
|
||||
Ok(req_id)
|
||||
let request = Request::BlocksByRange(request);
|
||||
let id = self.next_id();
|
||||
let request_id = RequestId::Sync(SyncRequestId::BackFillSync { id });
|
||||
self.send_network_msg(NetworkMessage::SendRequest {
|
||||
peer_id,
|
||||
request,
|
||||
request_id,
|
||||
})?;
|
||||
self.backfill_requests.insert(id, batch_id);
|
||||
Ok(id)
|
||||
}
|
||||
|
||||
/// Received a blocks by range response.
|
||||
pub fn blocks_by_range_response(
|
||||
pub fn range_sync_response(
|
||||
&mut self,
|
||||
request_id: usize,
|
||||
request_id: Id,
|
||||
remove: bool,
|
||||
) -> Option<SyncRequestType> {
|
||||
// NOTE: we can't guarantee that the request must be registered as it could receive more
|
||||
// than an error, and be removed after receiving the first one.
|
||||
// FIXME: https://github.com/sigp/lighthouse/issues/1634
|
||||
) -> Option<(ChainId, BatchId)> {
|
||||
if remove {
|
||||
self.range_requests.remove(&request_id)
|
||||
} else {
|
||||
@ -140,12 +155,21 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
}
|
||||
}
|
||||
|
||||
/// Sends a blocks by root request.
|
||||
pub fn blocks_by_root_request(
|
||||
/// Received a blocks by range response.
|
||||
pub fn backfill_sync_response(&mut self, request_id: Id, remove: bool) -> Option<BatchId> {
|
||||
if remove {
|
||||
self.backfill_requests.remove(&request_id)
|
||||
} else {
|
||||
self.backfill_requests.get(&request_id).cloned()
|
||||
}
|
||||
}
|
||||
|
||||
/// Sends a blocks by root request for a single block lookup.
|
||||
pub fn single_block_lookup_request(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
request: BlocksByRootRequest,
|
||||
) -> Result<usize, &'static str> {
|
||||
) -> Result<Id, &'static str> {
|
||||
trace!(
|
||||
self.log,
|
||||
"Sending BlocksByRoot Request";
|
||||
@ -153,7 +177,39 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
"count" => request.block_roots.len(),
|
||||
"peer" => %peer_id
|
||||
);
|
||||
self.send_rpc_request(peer_id, Request::BlocksByRoot(request))
|
||||
let request = Request::BlocksByRoot(request);
|
||||
let id = self.next_id();
|
||||
let request_id = RequestId::Sync(SyncRequestId::SingleBlock { id });
|
||||
self.send_network_msg(NetworkMessage::SendRequest {
|
||||
peer_id,
|
||||
request,
|
||||
request_id,
|
||||
})?;
|
||||
Ok(id)
|
||||
}
|
||||
|
||||
/// Sends a blocks by root request for a parent request.
|
||||
pub fn parent_lookup_request(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
request: BlocksByRootRequest,
|
||||
) -> Result<Id, &'static str> {
|
||||
trace!(
|
||||
self.log,
|
||||
"Sending BlocksByRoot Request";
|
||||
"method" => "BlocksByRoot",
|
||||
"count" => request.block_roots.len(),
|
||||
"peer" => %peer_id
|
||||
);
|
||||
let request = Request::BlocksByRoot(request);
|
||||
let id = self.next_id();
|
||||
let request_id = RequestId::Sync(SyncRequestId::ParentLookup { id });
|
||||
self.send_network_msg(NetworkMessage::SendRequest {
|
||||
peer_id,
|
||||
request,
|
||||
request_id,
|
||||
})?;
|
||||
Ok(id)
|
||||
}
|
||||
|
||||
/// Terminates the connection with the peer and bans them.
|
||||
@ -184,22 +240,6 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
});
|
||||
}
|
||||
|
||||
/// Sends an RPC request.
|
||||
fn send_rpc_request(
|
||||
&mut self,
|
||||
peer_id: PeerId,
|
||||
request: Request,
|
||||
) -> Result<usize, &'static str> {
|
||||
let request_id = self.request_id;
|
||||
self.request_id += 1;
|
||||
self.send_network_msg(NetworkMessage::SendRequest {
|
||||
peer_id,
|
||||
request_id: RequestId::Sync(request_id),
|
||||
request,
|
||||
})?;
|
||||
Ok(request_id)
|
||||
}
|
||||
|
||||
/// Subscribes to core topics.
|
||||
pub fn subscribe_core_topics(&mut self) {
|
||||
self.network_send
|
||||
@ -216,4 +256,10 @@ impl<T: EthSpec> SyncNetworkContext<T> {
|
||||
"Network channel send Failed"
|
||||
})
|
||||
}
|
||||
|
||||
fn next_id(&mut self) -> Id {
|
||||
let id = self.request_id;
|
||||
self.request_id += 1;
|
||||
id
|
||||
}
|
||||
}
|
||||
|
@ -1,4 +1,4 @@
|
||||
use crate::sync::RequestId;
|
||||
use crate::sync::manager::Id;
|
||||
use lighthouse_network::rpc::methods::BlocksByRangeRequest;
|
||||
use lighthouse_network::PeerId;
|
||||
use std::collections::HashSet;
|
||||
@ -93,7 +93,7 @@ pub enum BatchState<T: EthSpec> {
|
||||
/// The batch has failed either downloading or processing, but can be requested again.
|
||||
AwaitingDownload,
|
||||
/// The batch is being downloaded.
|
||||
Downloading(PeerId, Vec<SignedBeaconBlock<T>>, RequestId),
|
||||
Downloading(PeerId, Vec<SignedBeaconBlock<T>>, Id),
|
||||
/// The batch has been completely downloaded and is ready for processing.
|
||||
AwaitingProcessing(PeerId, Vec<SignedBeaconBlock<T>>),
|
||||
/// The batch is being processed.
|
||||
@ -167,7 +167,7 @@ impl<T: EthSpec, B: BatchConfig> BatchInfo<T, B> {
|
||||
}
|
||||
|
||||
/// Verifies if an incomming block belongs to this batch.
|
||||
pub fn is_expecting_block(&self, peer_id: &PeerId, request_id: &RequestId) -> bool {
|
||||
pub fn is_expecting_block(&self, peer_id: &PeerId, request_id: &Id) -> bool {
|
||||
if let BatchState::Downloading(expected_peer, _, expected_id) = &self.state {
|
||||
return peer_id == expected_peer && expected_id == request_id;
|
||||
}
|
||||
@ -312,7 +312,7 @@ impl<T: EthSpec, B: BatchConfig> BatchInfo<T, B> {
|
||||
pub fn start_downloading_from_peer(
|
||||
&mut self,
|
||||
peer: PeerId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
) -> Result<(), WrongState> {
|
||||
match self.state.poison() {
|
||||
BatchState::AwaitingDownload => {
|
||||
|
@ -1,7 +1,7 @@
|
||||
use super::batch::{BatchInfo, BatchState};
|
||||
use crate::beacon_processor::ProcessId;
|
||||
use crate::beacon_processor::WorkEvent as BeaconWorkEvent;
|
||||
use crate::sync::{network_context::SyncNetworkContext, BatchProcessResult, RequestId};
|
||||
use crate::sync::{manager::Id, network_context::SyncNetworkContext, BatchProcessResult};
|
||||
use beacon_chain::BeaconChainTypes;
|
||||
use fnv::FnvHashMap;
|
||||
use lighthouse_network::{PeerAction, PeerId};
|
||||
@ -214,7 +214,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
||||
network: &mut SyncNetworkContext<T::EthSpec>,
|
||||
batch_id: BatchId,
|
||||
peer_id: &PeerId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
|
||||
) -> ProcessingResult {
|
||||
// check if we have this batch
|
||||
@ -807,7 +807,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
||||
network: &mut SyncNetworkContext<T::EthSpec>,
|
||||
batch_id: BatchId,
|
||||
peer_id: &PeerId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
) -> ProcessingResult {
|
||||
if let Some(batch) = self.batches.get_mut(&batch_id) {
|
||||
// A batch could be retried without the peer failing the request (disconnecting/
|
||||
|
@ -45,8 +45,9 @@ use super::chain_collection::ChainCollection;
|
||||
use super::sync_type::RangeSyncType;
|
||||
use crate::beacon_processor::WorkEvent as BeaconWorkEvent;
|
||||
use crate::status::ToStatusMessage;
|
||||
use crate::sync::manager::Id;
|
||||
use crate::sync::network_context::SyncNetworkContext;
|
||||
use crate::sync::{BatchProcessResult, RequestId};
|
||||
use crate::sync::BatchProcessResult;
|
||||
use beacon_chain::{BeaconChain, BeaconChainTypes};
|
||||
use lighthouse_network::PeerId;
|
||||
use lighthouse_network::SyncInfo;
|
||||
@ -201,7 +202,7 @@ where
|
||||
peer_id: PeerId,
|
||||
chain_id: ChainId,
|
||||
batch_id: BatchId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
beacon_block: Option<SignedBeaconBlock<T::EthSpec>>,
|
||||
) {
|
||||
// check if this chunk removes the chain
|
||||
@ -300,7 +301,7 @@ where
|
||||
peer_id: PeerId,
|
||||
batch_id: BatchId,
|
||||
chain_id: ChainId,
|
||||
request_id: RequestId,
|
||||
request_id: Id,
|
||||
) {
|
||||
// check that this request is pending
|
||||
match self.chains.call_by_id(chain_id, |chain| {
|
||||
@ -364,6 +365,7 @@ where
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::service::RequestId;
|
||||
use crate::NetworkMessage;
|
||||
|
||||
use super::*;
|
||||
@ -494,10 +496,7 @@ mod tests {
|
||||
}
|
||||
|
||||
/// Reads an BlocksByRange request to a given peer from the network receiver channel.
|
||||
fn grab_request(
|
||||
&mut self,
|
||||
expected_peer: &PeerId,
|
||||
) -> (lighthouse_network::rpc::RequestId, BlocksByRangeRequest) {
|
||||
fn grab_request(&mut self, expected_peer: &PeerId) -> (RequestId, BlocksByRangeRequest) {
|
||||
if let Some(NetworkMessage::SendRequest {
|
||||
peer_id,
|
||||
request: Request::BlocksByRange(request),
|
||||
|
Loading…
Reference in New Issue
Block a user