lighthouse-pulse/beacon_node/network/src/router/mod.rs
Age Manning 95c8e476bc
Initial work towards v0.2.0 (#924)
* Remove ping protocol

* Initial renaming of network services

* Correct rebasing relative to latest master

* Start updating types

* Adds HashMapDelay struct to utils

* Initial network restructure

* Network restructure. Adds new types for v0.2.0

* Removes build artefacts

* Shift validation to beacon chain

* Temporarily remove gossip validation

This is to be updated to match current optimisation efforts.

* Adds AggregateAndProof

* Begin rebuilding pubsub encoding/decoding

* Signature hacking

* Shift gossipsup decoding into eth2_libp2p

* Existing EF tests passing with fake_crypto

* Shifts block encoding/decoding into RPC

* Delete outdated API spec

* All release tests passing bar genesis state parsing

* Update and test YamlConfig

* Update to spec v0.10 compatible BLS

* Updates to BLS EF tests

* Add EF test for AggregateVerify

And delete unused hash2curve tests for uncompressed points

* Update EF tests to v0.10.1

* Use optional block root correctly in block proc

* Use genesis fork in deposit domain. All tests pass

* Fast aggregate verify test

* Update REST API docs

* Fix unused import

* Bump spec tags to v0.10.1

* Add `seconds_per_eth1_block` to chainspec

* Update to timestamp based eth1 voting scheme

* Return None from `get_votes_to_consider` if block cache is empty

* Handle overflows in `is_candidate_block`

* Revert to failing tests

* Fix eth1 data sets test

* Choose default vote according to spec

* Fix collect_valid_votes tests

* Fix `get_votes_to_consider` to choose all eligible blocks

* Uncomment winning_vote tests

* Add comments; remove unused code

* Reduce seconds_per_eth1_block for simulation

* Addressed review comments

* Add test for default vote case

* Fix logs

* Remove unused functions

* Meter default eth1 votes

* Fix comments

* Progress on attestation service

* Address review comments; remove unused dependency

* Initial work on removing libp2p lock

* Add LRU caches to store (rollup)

* Update attestation validation for DB changes (WIP)

* Initial version of should_forward_block

* Scaffold

* Progress on attestation validation

Also, consolidate prod+testing slot clocks so that they share much
of the same implementation and can both handle sub-slot time changes.

* Removes lock from libp2p service

* Completed network lock removal

* Finish(?) attestation processing

* Correct network termination future

* Add slot check to block check

* Correct fmt issues

* Remove Drop implementation for network service

* Add first attempt at attestation proc. re-write

* Add version 2 of attestation processing

* Minor fixes

* Add validator pubkey cache

* Make get_indexed_attestation take a committee

* Link signature processing into new attn verification

* First working version

* Ensure pubkey cache is updated

* Add more metrics, slight optimizations

* Clone committee cache during attestation processing

* Update shuffling cache during block processing

* Remove old commented-out code

* Fix shuffling cache insert bug

* Used indexed attestation in fork choice

* Restructure attn processing, add metrics

* Add more detailed metrics

* Tidy, fix failing tests

* Fix failing tests, tidy

* Address reviewers suggestions

* Disable/delete two outdated tests

* Modification of validator for subscriptions

* Add slot signing to validator client

* Further progress on validation subscription

* Adds necessary validator subscription functionality

* Add new Pubkeys struct to signature_sets

* Refactor with functional approach

* Update beacon chain

* Clean up validator <-> beacon node http types

* Add aggregator status to ValidatorDuty

* Impl Clone for manual slot clock

* Fix minor errors

* Further progress validator client subscription

* Initial subscription and aggregation handling

* Remove decompressed member from pubkey bytes

* Progress to modifying val client for attestation aggregation

* First draft of validator client upgrade for aggregate attestations

* Add hashmap for indices lookup

* Add state cache, remove store cache

* Only build the head committee cache

* Removes lock on a network channel

* Partially implement beacon node subscription http api

* Correct compilation issues

* Change `get_attesting_indices` to use Vec

* Fix failing test

* Partial implementation of timer

* Adds timer, removes exit_future, http api to op pool

* Partial multiple aggregate attestation handling

* Permits bulk messages accross gossipsub network channel

* Correct compile issues

* Improve gosispsub messaging and correct rest api helpers

* Added global gossipsub subscriptions

* Update validator subscriptions data structs

* Tidy

* Re-structure validator subscriptions

* Initial handling of subscriptions

* Re-structure network service

* Add pubkey cache persistence file

* Add more comments

* Integrate persistence file into builder

* Add pubkey cache tests

* Add HashSetDelay and introduce into attestation service

* Handles validator subscriptions

* Add data_dir to beacon chain builder

* Remove Option in pubkey cache persistence file

* Ensure consistency between datadir/data_dir

* Fix failing network test

* Peer subnet discovery gets queued for future subscriptions

* Reorganise attestation service functions

* Initial wiring of attestation service

* First draft of attestation service timing logic

* Correct minor typos

* Tidy

* Fix todos

* Improve tests

* Add PeerInfo to connected peers mapping

* Fix compile error

* Fix compile error from merge

* Split up block processing metrics

* Tidy

* Refactor get_pubkey_from_state

* Remove commented-out code

* Rename state_cache -> checkpoint_cache

* Rename Checkpoint -> Snapshot

* Tidy, add comments

* Tidy up find_head function

* Change some checkpoint -> snapshot

* Add tests

* Expose max_len

* Remove dead code

* Tidy

* Fix bug

* Add sync-speed metric

* Add first attempt at VerifiableBlock

* Start integrating into beacon chain

* Integrate VerifiableBlock

* Rename VerifableBlock -> PartialBlockVerification

* Add start of typed methods

* Add progress

* Add further progress

* Rename structs

* Add full block verification to block_processing.rs

* Further beacon chain integration

* Update checks for gossip

* Add todo

* Start adding segement verification

* Add passing chain segement test

* Initial integration with batch sync

* Minor changes

* Tidy, add more error checking

* Start adding chain_segment tests

* Finish invalid signature tests

* Include single and gossip verified blocks in tests

* Add gossip verification tests

* Start adding docs

* Finish adding comments to block_processing.rs

* Rename block_processing.rs -> block_verification

* Start removing old block processing code

* Fixes beacon_chain compilation

* Fix project-wide compile errors

* Remove old code

* Correct code to pass all tests

* Fix bug with beacon proposer index

* Fix shim for BlockProcessingError

* Only process one epoch at a time

* Fix loop in chain segment processing

* Correct tests from master merge

* Add caching for state.eth1_data_votes

* Add BeaconChain::validator_pubkey

* Revert "Add caching for state.eth1_data_votes"

This reverts commit cd73dcd6434fb8d8e6bf30c5356355598ea7b78e.

Co-authored-by: Grant Wuerker <gwuerker@gmail.com>
Co-authored-by: Michael Sproul <michael@sigmaprime.io>
Co-authored-by: Michael Sproul <micsproul@gmail.com>
Co-authored-by: pawan <pawandhananjay@gmail.com>
Co-authored-by: Paul Hauner <paul@paulhauner.com>
2020-03-17 17:24:44 +11:00

276 lines
11 KiB
Rust

//! This module handles incoming network messages.
//!
//! It routes the messages to appropriate services, such as the Sync
//! and processes those that are
#![allow(clippy::unit_arg)]
pub mod processor;
use crate::error;
use crate::service::NetworkMessage;
use beacon_chain::{BeaconChain, BeaconChainTypes};
use eth2_libp2p::{
rpc::{RPCError, RPCErrorResponse, RPCRequest, RPCResponse, RequestId, ResponseTermination},
MessageId, PeerId, PubsubData, PubsubMessage, RPCEvent,
};
use futures::future::Future;
use futures::stream::Stream;
use processor::Processor;
use slog::{debug, o, trace, warn};
use std::sync::Arc;
use tokio::sync::mpsc;
use types::EthSpec;
/// Handles messages received from the network and client and organises syncing. This
/// functionality of this struct is to validate an decode messages from the network before
/// passing them to the internal message processor. The message processor spawns a syncing thread
/// which manages which blocks need to be requested and processed.
pub struct Router<T: BeaconChainTypes> {
/// A channel to the network service to allow for gossip propagation.
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
/// Processes validated and decoded messages from the network. Has direct access to the
/// sync manager.
processor: Processor<T>,
/// The `Router` logger.
log: slog::Logger,
}
/// Types of messages the handler can receive.
#[derive(Debug)]
pub enum RouterMessage<T: EthSpec> {
/// We have initiated a connection to a new peer.
PeerDialed(PeerId),
/// Peer has disconnected,
PeerDisconnected(PeerId),
/// An RPC response/request has been received.
RPC(PeerId, RPCEvent<T>),
/// A gossip message has been received. The fields are: message id, the peer that sent us this
/// message and the message itself.
PubsubMessage(MessageId, PeerId, PubsubMessage<T>),
}
impl<T: BeaconChainTypes> Router<T> {
/// Initializes and runs the Router.
pub fn spawn(
beacon_chain: Arc<BeaconChain<T>>,
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
executor: &tokio::runtime::TaskExecutor,
log: slog::Logger,
) -> error::Result<mpsc::UnboundedSender<RouterMessage<T::EthSpec>>> {
let message_handler_log = log.new(o!("service"=> "msg_handler"));
trace!(message_handler_log, "Service starting");
let (handler_send, handler_recv) = mpsc::unbounded_channel();
// Initialise a message instance, which itself spawns the syncing thread.
let processor = Processor::new(executor, beacon_chain, network_send.clone(), &log);
// generate the Message handler
let mut handler = Router {
network_send,
processor,
log: message_handler_log,
};
// spawn handler task and move the message handler instance into the spawned thread
executor.spawn(
handler_recv
.for_each(move |msg| Ok(handler.handle_message(msg)))
.map_err(move |_| {
debug!(log, "Network message handler terminated.");
}),
);
Ok(handler_send)
}
/// Handle all messages incoming from the network service.
fn handle_message(&mut self, message: RouterMessage<T::EthSpec>) {
match message {
// we have initiated a connection to a peer
RouterMessage::PeerDialed(peer_id) => {
self.processor.on_connect(peer_id);
}
// A peer has disconnected
RouterMessage::PeerDisconnected(peer_id) => {
self.processor.on_disconnect(peer_id);
}
// An RPC message request/response has been received
RouterMessage::RPC(peer_id, rpc_event) => {
self.handle_rpc_message(peer_id, rpc_event);
}
// An RPC message request/response has been received
RouterMessage::PubsubMessage(id, peer_id, gossip) => {
self.handle_gossip(id, peer_id, gossip);
}
}
}
/* 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, error) => self.handle_rpc_error(peer_id, id, error),
}
}
/// A new RPC request has been received from the network.
fn handle_rpc_request(
&mut self,
peer_id: PeerId,
request_id: RequestId,
request: RPCRequest<T::EthSpec>,
) {
match request {
RPCRequest::Status(status_message) => {
self.processor
.on_status_request(peer_id, request_id, status_message)
}
RPCRequest::Goodbye(goodbye_reason) => {
debug!(
self.log, "PeerGoodbye";
"peer" => format!("{:?}", peer_id),
"reason" => format!("{:?}", goodbye_reason),
);
self.processor.on_disconnect(peer_id);
}
RPCRequest::BlocksByRange(request) => self
.processor
.on_blocks_by_range_request(peer_id, request_id, request),
RPCRequest::BlocksByRoot(request) => self
.processor
.on_blocks_by_root_request(peer_id, request_id, request),
RPCRequest::Phantom(_) => unreachable!("Phantom never initialised"),
}
}
/// An RPC response has been received from the network.
// we match on id and ignore responses past the timeout.
fn handle_rpc_response(
&mut self,
peer_id: PeerId,
request_id: RequestId,
error_response: RPCErrorResponse<T::EthSpec>,
) {
// an error could have occurred.
match error_response {
RPCErrorResponse::InvalidRequest(error) => {
warn!(self.log, "Peer indicated invalid request";"peer_id" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::ServerError(error) => {
warn!(self.log, "Peer internal server error";"peer_id" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::Unknown(error) => {
warn!(self.log, "Unknown peer error";"peer" => format!("{:?}", peer_id), "error" => error.as_string());
self.handle_rpc_error(peer_id, request_id, RPCError::RPCErrorResponse);
}
RPCErrorResponse::Success(response) => match response {
RPCResponse::Status(status_message) => {
self.processor.on_status_response(peer_id, status_message);
}
RPCResponse::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),
);
}
},
RPCErrorResponse::StreamTermination(response_type) => {
// have received a stream termination, notify the processing functions
match response_type {
ResponseTermination::BlocksByRange => {
self.processor
.on_blocks_by_range_response(peer_id, request_id, None);
}
ResponseTermination::BlocksByRoot => {
self.processor
.on_blocks_by_root_response(peer_id, request_id, None);
}
}
}
}
}
/// Handle various RPC errors
fn handle_rpc_error(&mut self, peer_id: PeerId, request_id: RequestId, error: RPCError) {
warn!(self.log, "RPC Error"; "Peer" => format!("{:?}", peer_id), "request_id" => format!("{}", request_id), "Error" => format!("{:?}", error));
self.processor.on_rpc_error(peer_id, request_id);
}
/// Handle RPC messages
fn handle_gossip(
&mut self,
id: MessageId,
peer_id: PeerId,
gossip_message: PubsubMessage<T::EthSpec>,
) {
match gossip_message.data {
PubsubData::BeaconBlock(block) => {
if self.processor.should_forward_block(&block) {
self.propagate_message(id, peer_id.clone());
}
self.processor.on_block_gossip(peer_id, block);
}
PubsubData::AggregateAndProofAttestation(_agg_attestation) => {
// TODO: Handle propagation conditions
self.propagate_message(id, peer_id);
// TODO Handle aggregate attestion
// self.processor
// .on_attestation_gossip(peer_id.clone(), &agg_attestation);
}
PubsubData::Attestation(boxed_shard_attestation) => {
// TODO: Handle propagation conditions
self.propagate_message(id, peer_id.clone());
self.processor
.on_attestation_gossip(peer_id, boxed_shard_attestation.1);
}
PubsubData::VoluntaryExit(_exit) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle exits
debug!(self.log, "Received a voluntary exit"; "peer_id" => format!("{}", peer_id) );
}
PubsubData::ProposerSlashing(_proposer_slashing) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle proposer slashings
debug!(self.log, "Received a proposer slashing"; "peer_id" => format!("{}", peer_id) );
}
PubsubData::AttesterSlashing(_attester_slashing) => {
// TODO: Apply more sophisticated validation
self.propagate_message(id, peer_id.clone());
// TODO: Handle attester slashings
debug!(self.log, "Received an attester slashing"; "peer_id" => format!("{}", peer_id) );
}
}
}
/// Informs the network service that the message should be forwarded to other peers.
fn propagate_message(&mut self, message_id: MessageId, propagation_source: PeerId) {
self.network_send
.try_send(NetworkMessage::Propagate {
propagation_source,
message_id,
})
.unwrap_or_else(|_| {
warn!(
self.log,
"Could not send propagation request to the network service"
)
});
}
}