From ca18d4390aff4bfbf78d361be65916de9b21325f Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Thu, 21 Mar 2019 17:17:01 +1100 Subject: [PATCH 01/24] Add first changes to syncing logic - Adds testing framework - Breaks out new `NetworkContext` object --- beacon_node/beacon_chain/src/lib.rs | 1 + .../beacon_chain/src/test_utils/mod.rs | 3 + .../testing_beacon_chain_builder.rs | 50 +++++ .../test_harness/src/beacon_chain_harness.rs | 11 +- beacon_node/libp2p/Cargo.toml | 1 + beacon_node/libp2p/src/rpc/methods.rs | 28 +++ beacon_node/network/Cargo.toml | 4 + beacon_node/network/src/beacon_chain.rs | 41 ++++ beacon_node/network/src/lib.rs | 4 +- beacon_node/network/src/message_handler.rs | 151 +++++++------- beacon_node/network/src/service.rs | 1 + beacon_node/network/src/sync/simple_sync.rs | 115 ++++++++++- beacon_node/network/tests/tests.rs | 184 ++++++++++++++++++ .../testing_beacon_state_builder.rs | 1 + 14 files changed, 513 insertions(+), 82 deletions(-) create mode 100644 beacon_node/beacon_chain/src/test_utils/mod.rs create mode 100644 beacon_node/beacon_chain/src/test_utils/testing_beacon_chain_builder.rs create mode 100644 beacon_node/network/tests/tests.rs diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 2137c0edf..5fa7e7a77 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -3,6 +3,7 @@ mod beacon_chain; mod checkpoint; mod errors; pub mod initialise; +pub mod test_utils; pub use self::beacon_chain::{BeaconChain, BlockProcessingOutcome, InvalidBlock, ValidBlock}; pub use self::checkpoint::CheckPoint; diff --git a/beacon_node/beacon_chain/src/test_utils/mod.rs b/beacon_node/beacon_chain/src/test_utils/mod.rs new file mode 100644 index 000000000..ad251a3c9 --- /dev/null +++ b/beacon_node/beacon_chain/src/test_utils/mod.rs @@ -0,0 +1,3 @@ +mod testing_beacon_chain_builder; + +pub use testing_beacon_chain_builder::TestingBeaconChainBuilder; diff --git a/beacon_node/beacon_chain/src/test_utils/testing_beacon_chain_builder.rs b/beacon_node/beacon_chain/src/test_utils/testing_beacon_chain_builder.rs new file mode 100644 index 000000000..5c5477e55 --- /dev/null +++ b/beacon_node/beacon_chain/src/test_utils/testing_beacon_chain_builder.rs @@ -0,0 +1,50 @@ +pub use crate::{BeaconChain, BeaconChainError, CheckPoint}; +use db::{ + stores::{BeaconBlockStore, BeaconStateStore}, + MemoryDB, +}; +use fork_choice::BitwiseLMDGhost; +use slot_clock::TestingSlotClock; +use ssz::TreeHash; +use std::sync::Arc; +use types::test_utils::TestingBeaconStateBuilder; +use types::*; + +type TestingBeaconChain = BeaconChain>; + +pub struct TestingBeaconChainBuilder { + state_builder: TestingBeaconStateBuilder, +} + +impl TestingBeaconChainBuilder { + pub fn build(self, spec: &ChainSpec) -> TestingBeaconChain { + let db = Arc::new(MemoryDB::open()); + let block_store = Arc::new(BeaconBlockStore::new(db.clone())); + let state_store = Arc::new(BeaconStateStore::new(db.clone())); + let slot_clock = TestingSlotClock::new(spec.genesis_slot.as_u64()); + let fork_choice = BitwiseLMDGhost::new(block_store.clone(), state_store.clone()); + + let (genesis_state, _keypairs) = self.state_builder.build(); + + let mut genesis_block = BeaconBlock::empty(&spec); + genesis_block.state_root = Hash256::from_slice(&genesis_state.hash_tree_root()); + + // Create the Beacon Chain + BeaconChain::from_genesis( + state_store.clone(), + block_store.clone(), + slot_clock, + genesis_state, + genesis_block, + spec.clone(), + fork_choice, + ) + .unwrap() + } +} + +impl From for TestingBeaconChainBuilder { + fn from(state_builder: TestingBeaconStateBuilder) -> TestingBeaconChainBuilder { + TestingBeaconChainBuilder { state_builder } + } +} diff --git a/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs b/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs index bc5c93b94..800dd3ce6 100644 --- a/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs +++ b/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs @@ -36,14 +36,21 @@ impl BeaconChainHarness { /// - A keypair, `BlockProducer` and `Attester` for each validator. /// - A new BeaconChain struct where the given validators are in the genesis. pub fn new(spec: ChainSpec, validator_count: usize) -> Self { + let state_builder = + TestingBeaconStateBuilder::from_default_keypairs_file_if_exists(validator_count, &spec); + Self::from_beacon_state_builder(state_builder, spec) + } + + pub fn from_beacon_state_builder( + state_builder: TestingBeaconStateBuilder, + spec: ChainSpec, + ) -> Self { let db = Arc::new(MemoryDB::open()); let block_store = Arc::new(BeaconBlockStore::new(db.clone())); let state_store = Arc::new(BeaconStateStore::new(db.clone())); let slot_clock = TestingSlotClock::new(spec.genesis_slot.as_u64()); let fork_choice = BitwiseLMDGhost::new(block_store.clone(), state_store.clone()); - let state_builder = - TestingBeaconStateBuilder::from_default_keypairs_file_if_exists(validator_count, &spec); let (genesis_state, keypairs) = state_builder.build(); let mut genesis_block = BeaconBlock::empty(&spec); diff --git a/beacon_node/libp2p/Cargo.toml b/beacon_node/libp2p/Cargo.toml index dcbc04d0b..e863c4d78 100644 --- a/beacon_node/libp2p/Cargo.toml +++ b/beacon_node/libp2p/Cargo.toml @@ -5,6 +5,7 @@ authors = ["Age Manning "] edition = "2018" [dependencies] +beacon_chain = { path = "../beacon_chain" } # SigP repository until PR is merged libp2p = { git = "https://github.com/SigP/rust-libp2p", branch = "gossipsub" } types = { path = "../../eth2/types" } diff --git a/beacon_node/libp2p/src/rpc/methods.rs b/beacon_node/libp2p/src/rpc/methods.rs index 3014afd0f..f05ade7ff 100644 --- a/beacon_node/libp2p/src/rpc/methods.rs +++ b/beacon_node/libp2p/src/rpc/methods.rs @@ -1,3 +1,4 @@ +use beacon_chain::parking_lot::RwLockReadGuard; /// Available RPC methods types and ids. use ssz_derive::{Decode, Encode}; use types::{BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; @@ -60,6 +61,20 @@ pub enum RPCRequest { BeaconChainState(BeaconChainStateRequest), } +impl RPCRequest { + pub fn method_id(&self) -> u16 { + let method = match self { + RPCRequest::Hello(_) => RPCMethod::Hello, + RPCRequest::Goodbye(_) => RPCMethod::Goodbye, + RPCRequest::BeaconBlockRoots(_) => RPCMethod::BeaconBlockRoots, + RPCRequest::BeaconBlockHeaders(_) => RPCMethod::BeaconBlockHeaders, + RPCRequest::BeaconBlockBodies(_) => RPCMethod::BeaconBlockBodies, + RPCRequest::BeaconChainState(_) => RPCMethod::BeaconChainState, + }; + method.into() + } +} + #[derive(Debug, Clone)] pub enum RPCResponse { Hello(HelloMessage), @@ -69,6 +84,19 @@ pub enum RPCResponse { BeaconChainState(BeaconChainStateResponse), } +impl RPCResponse { + pub fn method_id(&self) -> u16 { + let method = match self { + RPCResponse::Hello(_) => RPCMethod::Hello, + RPCResponse::BeaconBlockRoots(_) => RPCMethod::BeaconBlockRoots, + RPCResponse::BeaconBlockHeaders(_) => RPCMethod::BeaconBlockHeaders, + RPCResponse::BeaconBlockBodies(_) => RPCMethod::BeaconBlockBodies, + RPCResponse::BeaconChainState(_) => RPCMethod::BeaconChainState, + }; + method.into() + } +} + /* Request/Response data structures for RPC methods */ /// The HELLO request/response handshake message. diff --git a/beacon_node/network/Cargo.toml b/beacon_node/network/Cargo.toml index 8b87a9d50..260ee0896 100644 --- a/beacon_node/network/Cargo.toml +++ b/beacon_node/network/Cargo.toml @@ -4,6 +4,10 @@ version = "0.1.0" authors = ["Age Manning "] edition = "2018" +[dev-dependencies] +test_harness = { path = "../beacon_chain/test_harness" } +sloggers = "0.3.2" + [dependencies] beacon_chain = { path = "../beacon_chain" } libp2p = { path = "../libp2p" } diff --git a/beacon_node/network/src/beacon_chain.rs b/beacon_node/network/src/beacon_chain.rs index 91628cc7e..63e1eb6dd 100644 --- a/beacon_node/network/src/beacon_chain.rs +++ b/beacon_node/network/src/beacon_chain.rs @@ -7,6 +7,8 @@ use beacon_chain::{ types::{BeaconState, ChainSpec}, CheckPoint, }; +use libp2p::HelloMessage; +use types::{Epoch, Hash256, Slot}; /// The network's API to the beacon chain. pub trait BeaconChain: Send + Sync { @@ -14,9 +16,19 @@ pub trait BeaconChain: Send + Sync { fn get_state(&self) -> RwLockReadGuard; + fn slot(&self) -> Slot; + fn head(&self) -> RwLockReadGuard; + fn best_slot(&self) -> Slot; + + fn best_block_root(&self) -> Hash256; + fn finalized_head(&self) -> RwLockReadGuard; + + fn finalized_epoch(&self) -> Epoch; + + fn hello_message(&self) -> HelloMessage; } impl BeaconChain for RawBeaconChain @@ -33,11 +45,40 @@ where self.state.read() } + fn slot(&self) -> Slot { + self.get_state().slot + } + fn head(&self) -> RwLockReadGuard { self.head() } + fn finalized_epoch(&self) -> Epoch { + self.get_state().finalized_epoch + } + fn finalized_head(&self) -> RwLockReadGuard { self.finalized_head() } + + fn best_slot(&self) -> Slot { + self.head().beacon_block.slot + } + + fn best_block_root(&self) -> Hash256 { + self.head().beacon_block_root + } + + fn hello_message(&self) -> HelloMessage { + let spec = self.get_spec(); + let state = self.get_state(); + + HelloMessage { + network_id: spec.network_id, + latest_finalized_root: state.finalized_root, + latest_finalized_epoch: state.finalized_epoch, + best_root: self.best_block_root(), + best_slot: self.best_slot(), + } + } } diff --git a/beacon_node/network/src/lib.rs b/beacon_node/network/src/lib.rs index 1e47b9a73..822b05509 100644 --- a/beacon_node/network/src/lib.rs +++ b/beacon_node/network/src/lib.rs @@ -1,8 +1,8 @@ /// This crate provides the network server for Lighthouse. pub mod beacon_chain; pub mod error; -mod message_handler; -mod service; +pub mod message_handler; +pub mod service; pub mod sync; pub use libp2p::NetworkConfig; diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 4cd0ab951..8e390b4af 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -5,32 +5,28 @@ use crate::sync::SimpleSync; use crossbeam_channel::{unbounded as channel, Sender}; use futures::future; use libp2p::{ - rpc::{RPCMethod, RPCRequest, RPCResponse}, + rpc::{RPCRequest, RPCResponse}, HelloMessage, PeerId, RPCEvent, }; +use slog::debug; use slog::warn; -use slog::{debug, trace}; use std::collections::HashMap; use std::sync::Arc; -use std::time::{Duration, Instant}; +use std::time::Instant; /// Timeout for RPC requests. -const REQUEST_TIMEOUT: Duration = Duration::from_secs(30); +// const REQUEST_TIMEOUT: Duration = Duration::from_secs(30); /// Timeout before banning a peer for non-identification. -const HELLO_TIMEOUT: Duration = Duration::from_secs(30); +// const HELLO_TIMEOUT: Duration = Duration::from_secs(30); /// Handles messages received from the network and client and organises syncing. pub struct MessageHandler { /// Currently loaded and initialised beacon chain. - chain: Arc, + _chain: Arc, /// The syncing framework. sync: SimpleSync, - /// The network channel to relay messages to the Network service. - network_send: crossbeam_channel::Sender, - /// A mapping of peers and the RPC id we have sent an RPC request to. - requests: HashMap<(PeerId, u64), Instant>, - /// A counter of request id for each peer. - request_ids: HashMap, + /// The context required to send messages to, and process messages from peers. + network_context: NetworkContext, /// The `MessageHandler` logger. log: slog::Logger, } @@ -65,13 +61,9 @@ impl MessageHandler { let sync = SimpleSync::new(beacon_chain.clone(), &log); let mut handler = MessageHandler { - // TODO: The handler may not need a chain, perhaps only sync? - chain: beacon_chain.clone(), + _chain: beacon_chain.clone(), sync, - network_send, - requests: HashMap::new(), - request_ids: HashMap::new(), - + network_context: NetworkContext::new(network_send, log.clone()), log: log.clone(), }; @@ -93,8 +85,7 @@ impl MessageHandler { match message { // we have initiated a connection to a peer HandlerMessage::PeerDialed(peer_id) => { - let id = self.generate_request_id(&peer_id); - self.send_hello(peer_id, id, true); + self.sync.on_connect(&peer_id, &mut self.network_context); } // we have received an RPC message request/response HandlerMessage::RPC(peer_id, rpc_event) => { @@ -118,9 +109,11 @@ impl MessageHandler { /// A new RPC request has been received from the network. fn handle_rpc_request(&mut self, peer_id: PeerId, id: u64, request: RPCRequest) { + // TODO: ensure the id is legit match request { RPCRequest::Hello(hello_message) => { - self.handle_hello_request(peer_id, id, hello_message) + self.sync + .on_hello(&peer_id, hello_message, &mut self.network_context) } // TODO: Handle all requests _ => {} @@ -131,7 +124,12 @@ impl MessageHandler { // we match on id and ignore responses past the timeout. fn handle_rpc_response(&mut self, peer_id: PeerId, id: u64, response: RPCResponse) { // if response id is related to a request, ignore (likely RPC timeout) - if self.requests.remove(&(peer_id.clone(), id)).is_none() { + if self + .network_context + .requests + .remove(&(peer_id.clone(), id)) + .is_none() + { debug!(self.log, "Unrecognized response from peer: {:?}", peer_id); return; } @@ -145,16 +143,10 @@ impl MessageHandler { } } - /// Handle a HELLO RPC request message. - fn handle_hello_request(&mut self, peer_id: PeerId, id: u64, hello_message: HelloMessage) { - // send back a HELLO message - self.send_hello(peer_id.clone(), id, false); - // validate the peer - self.validate_hello(peer_id, hello_message); - } - /// Validate a HELLO RPC message. fn validate_hello(&mut self, peer_id: PeerId, message: HelloMessage) { + self.sync + .on_hello(&peer_id, message.clone(), &mut self.network_context); // validate the peer if !self.sync.validate_peer(peer_id.clone(), message) { debug!( @@ -164,8 +156,68 @@ impl MessageHandler { //TODO: block/ban the peer } } +} - /* General RPC helper functions */ +pub struct NetworkContext { + /// The network channel to relay messages to the Network service. + network_send: crossbeam_channel::Sender, + /// A mapping of peers and the RPC id we have sent an RPC request to. + requests: HashMap<(PeerId, u64), Instant>, + /// A counter of request id for each peer. + request_ids: HashMap, + /// The `MessageHandler` logger. + log: slog::Logger, +} + +impl NetworkContext { + pub fn new(network_send: crossbeam_channel::Sender, log: slog::Logger) -> Self { + Self { + network_send, + requests: HashMap::new(), + request_ids: HashMap::new(), + log, + } + } + + pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest) { + let id = self.generate_request_id(&peer_id); + self.send_rpc_event( + peer_id, + RPCEvent::Request { + id, + method_id: rpc_request.method_id(), + body: rpc_request, + }, + ); + } + + pub fn send_rpc_response(&mut self, peer_id: PeerId, rpc_response: RPCResponse) { + let id = self.generate_request_id(&peer_id); + self.send_rpc_event( + peer_id, + RPCEvent::Response { + id, + method_id: rpc_response.method_id(), + result: rpc_response, + }, + ); + } + + fn send_rpc_event(&self, peer_id: PeerId, rpc_event: RPCEvent) { + self.send(peer_id, OutgoingMessage::RPC(rpc_event)) + } + + fn send(&self, peer_id: PeerId, outgoing_message: OutgoingMessage) { + self.network_send + .send(NetworkMessage::Send(peer_id, outgoing_message)) + .unwrap_or_else(|_| { + warn!( + self.log, + "Could not send RPC message to the network service" + ) + }); + // + } /// Generates a new request id for a peer. fn generate_request_id(&mut self, peer_id: &PeerId) -> u64 { @@ -185,41 +237,4 @@ impl MessageHandler { ); id } - - /// Sends a HELLO RPC request or response to a newly connected peer. - //TODO: The boolean determines if sending request/respond, will be cleaner in the RPC re-write - fn send_hello(&mut self, peer_id: PeerId, id: u64, is_request: bool) { - let rpc_event = if is_request { - RPCEvent::Request { - id, - method_id: RPCMethod::Hello.into(), - body: RPCRequest::Hello(self.sync.generate_hello()), - } - } else { - RPCEvent::Response { - id, - method_id: RPCMethod::Hello.into(), - result: RPCResponse::Hello(self.sync.generate_hello()), - } - }; - - // send the hello request to the network - trace!(self.log, "Sending HELLO message to peer {:?}", peer_id); - self.send_rpc(peer_id, rpc_event); - } - - /// Sends an RPC request/response to the network server. - fn send_rpc(&self, peer_id: PeerId, rpc_event: RPCEvent) { - self.network_send - .send(NetworkMessage::Send( - peer_id, - OutgoingMessage::RPC(rpc_event), - )) - .unwrap_or_else(|_| { - warn!( - self.log, - "Could not send RPC message to the network service" - ) - }); - } } diff --git a/beacon_node/network/src/service.rs b/beacon_node/network/src/service.rs index c3045d280..d01188f5d 100644 --- a/beacon_node/network/src/service.rs +++ b/beacon_node/network/src/service.rs @@ -6,6 +6,7 @@ use crossbeam_channel::{unbounded as channel, Sender, TryRecvError}; use futures::prelude::*; use futures::sync::oneshot; use futures::Stream; +use libp2p::rpc::RPCResponse; use libp2p::RPCEvent; use libp2p::Service as LibP2PService; use libp2p::{Libp2pEvent, PeerId}; diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 95c7092c3..ae6a9e7a1 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -1,11 +1,16 @@ use crate::beacon_chain::BeaconChain; -use libp2p::rpc::HelloMessage; +use crate::message_handler::{MessageHandler, NetworkContext}; +use crate::service::NetworkMessage; +use crossbeam_channel::Sender; +use libp2p::rpc::{HelloMessage, RPCMethod, RPCRequest, RPCResponse}; use libp2p::PeerId; use slog::{debug, o}; use std::collections::HashMap; use std::sync::Arc; use types::{Epoch, Hash256, Slot}; +type NetworkSender = Sender; + /// The number of slots that we can import blocks ahead of us, before going into full Sync mode. const SLOT_IMPORT_TOLERANCE: u64 = 100; @@ -17,6 +22,32 @@ pub struct PeerSyncInfo { best_slot: Slot, } +impl PeerSyncInfo { + pub fn is_on_chain(&self, chain: &Arc) -> bool { + // TODO: make useful. + true + } + + pub fn has_higher_finalized_epoch(&self, chain: &Arc) -> bool { + self.latest_finalized_epoch > chain.get_state().finalized_epoch + } + + pub fn has_higher_best_slot(&self, chain: &Arc) -> bool { + self.latest_finalized_epoch > chain.get_state().finalized_epoch + } +} + +impl From for PeerSyncInfo { + fn from(hello: HelloMessage) -> PeerSyncInfo { + PeerSyncInfo { + latest_finalized_root: hello.latest_finalized_root, + latest_finalized_epoch: hello.latest_finalized_epoch, + best_root: hello.best_root, + best_slot: hello.best_slot, + } + } +} + /// The current syncing state. #[derive(PartialEq)] pub enum SyncState { @@ -60,17 +91,81 @@ impl SimpleSync { } } + pub fn on_connect(&self, peer_id: &PeerId, network: &mut NetworkContext) { + network.send_rpc_request( + peer_id.clone(), + RPCRequest::Hello(self.chain.hello_message()), + ); + } + + pub fn on_hello_request( + &self, + peer_id: &PeerId, + hello: HelloMessage, + network: &mut NetworkContext, + ) { + network.send_rpc_response( + peer_id.clone(), + RPCResponse::Hello(self.chain.hello_message()), + ); + self.on_hello(peer_id, hello, network); + } + + pub fn on_hello(&self, peer_id: &PeerId, hello: HelloMessage, network: &mut NetworkContext) { + // network id must match + if hello.network_id != self.network_id { + debug!(self.log, "Bad network id. Peer: {:?}", peer_id); + return; + } + + let peer = PeerSyncInfo::from(hello); + + /* + if peer.has_higher_finalized_epoch(&self.chain) { + // we need blocks + let peer_slot = peer.latest_finalized_epoch.start_slot(spec.slots_per_epoch); + let our_slot = self.chain.finalized_epoch(); + let required_slots = peer_slot - our_slot; + } else { + if !peer.is_on_chain(&self.chain) { + return (true, responses); + } + // + } + */ + + /* + // compare latest epoch and finalized root to see if they exist in our chain + if peer_info.latest_finalized_epoch <= self.latest_finalized_epoch { + // ensure their finalized root is in our chain + // TODO: Get the finalized root at hello_message.latest_epoch and ensure they match + //if (hello_message.latest_finalized_root == self.chain.get_state() { + // return false; + // } + } + + // the client is valid, add it to our list of known_peers and request sync if required + // update peer list if peer already exists + let peer_info = PeerSyncInfo::from(hello); + + debug!(self.log, "Handshake successful. Peer: {:?}", peer_id); + self.known_peers.insert(peer_id, peer_info); + + // set state to sync + if self.state == SyncState::Idle + && hello_message.best_slot > self.latest_slot + SLOT_IMPORT_TOLERANCE + { + self.state = SyncState::Downloading; + //TODO: Start requesting blocks from known peers. Ideally in batches + } + + true + */ + } + /// Generates our current state in the form of a HELLO RPC message. pub fn generate_hello(&self) -> HelloMessage { - let state = &self.chain.get_state(); - //TODO: Paul to verify the logic of these fields. - HelloMessage { - network_id: self.network_id, - latest_finalized_root: state.finalized_root, - latest_finalized_epoch: state.finalized_epoch, - best_root: state.latest_block_roots[0], //TODO: build correct value as a beacon chain function - best_slot: state.slot - 1, - } + self.chain.hello_message() } pub fn validate_peer(&mut self, peer_id: PeerId, hello_message: HelloMessage) -> bool { diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs new file mode 100644 index 000000000..dc0832fed --- /dev/null +++ b/beacon_node/network/tests/tests.rs @@ -0,0 +1,184 @@ +use beacon_chain::test_utils::TestingBeaconChainBuilder; +use crossbeam_channel::{unbounded, Receiver, Sender}; +use libp2p::rpc::{HelloMessage, RPCMethod, RPCRequest, RPCResponse}; +use libp2p::{PeerId, RPCEvent}; +use network::beacon_chain::BeaconChain as NetworkBeaconChain; +use network::message_handler::{HandlerMessage, MessageHandler}; +use network::service::{NetworkMessage, OutgoingMessage}; +use sloggers::terminal::{Destination, TerminalLoggerBuilder}; +use sloggers::types::Severity; +use sloggers::Build; +use std::sync::Arc; +use test_harness::BeaconChainHarness; +use tokio::runtime::TaskExecutor; +use types::{test_utils::TestingBeaconStateBuilder, *}; + +pub struct SyncNode { + pub id: usize, + sender: Sender, + receiver: Receiver, +} + +impl SyncNode { + pub fn new( + id: usize, + executor: &TaskExecutor, + chain: Arc, + logger: slog::Logger, + ) -> Self { + let (network_sender, network_receiver) = unbounded(); + let message_handler_sender = + MessageHandler::spawn(chain, network_sender, executor, logger).unwrap(); + + Self { + id, + sender: message_handler_sender, + receiver: network_receiver, + } + } + + fn send(&self, message: HandlerMessage) { + self.sender.send(message).unwrap(); + } + + fn recv(&self) -> NetworkMessage { + self.receiver.recv().unwrap() + } + + fn recv_rpc_response(&self) -> RPCResponse { + let network_message = self.recv(); + match network_message { + NetworkMessage::Send( + _peer_id, + OutgoingMessage::RPC(RPCEvent::Response { + id: _, + method_id: _, + result, + }), + ) => result, + _ => panic!("get_rpc_response failed! got {:?}", network_message), + } + } + + fn recv_rpc_request(&self) -> RPCRequest { + let network_message = self.recv(); + match network_message { + NetworkMessage::Send( + _peer_id, + OutgoingMessage::RPC(RPCEvent::Request { + id: _, + method_id: _, + body, + }), + ) => body, + _ => panic!("get_rpc_request failed! got {:?}", network_message), + } + } +} + +fn get_logger() -> slog::Logger { + let mut builder = TerminalLoggerBuilder::new(); + builder.level(Severity::Debug); + builder.destination(Destination::Stderr); + builder.build().unwrap() +} + +pub struct SyncMaster { + harness: BeaconChainHarness, + peer_id: PeerId, + response_ids: Vec, +} + +impl SyncMaster { + fn from_beacon_state_builder( + state_builder: TestingBeaconStateBuilder, + node_count: usize, + spec: &ChainSpec, + ) -> Self { + let harness = BeaconChainHarness::from_beacon_state_builder(state_builder, spec.clone()); + let peer_id = PeerId::random(); + let response_ids = vec![0; node_count]; + + Self { + harness, + peer_id, + response_ids, + } + } + + pub fn response_id(&mut self, node: &SyncNode) -> u64 { + let id = self.response_ids[node.id]; + self.response_ids[node.id] += 1; + id + } + + pub fn do_hello_with(&mut self, node: &SyncNode) { + let message = HandlerMessage::PeerDialed(self.peer_id.clone()); + node.send(message); + + let request = node.recv_rpc_request(); + + match request { + RPCRequest::Hello(_hello) => { + let hello = self.harness.beacon_chain.hello_message(); + let response = self.rpc_response(node, RPCResponse::Hello(hello)); + node.send(response); + } + _ => panic!("Got message other than hello from node."), + } + } + + fn rpc_response(&mut self, node: &SyncNode, rpc_response: RPCResponse) -> HandlerMessage { + HandlerMessage::RPC( + self.peer_id.clone(), + RPCEvent::Response { + id: self.response_id(node), + method_id: RPCMethod::Hello.into(), + result: rpc_response, + }, + ) + } +} + +fn test_setup( + state_builder: TestingBeaconStateBuilder, + node_count: usize, + spec: &ChainSpec, + logger: slog::Logger, +) -> (tokio::runtime::Runtime, SyncMaster, Vec) { + let runtime = tokio::runtime::Runtime::new().unwrap(); + + let mut nodes = Vec::with_capacity(node_count); + for id in 0..node_count { + let local_chain = TestingBeaconChainBuilder::from(state_builder.clone()).build(&spec); + let node = SyncNode::new( + id, + &runtime.executor(), + Arc::new(local_chain), + logger.clone(), + ); + + nodes.push(node); + } + + let master = SyncMaster::from_beacon_state_builder(state_builder, node_count, &spec); + + (runtime, master, nodes) +} + +#[test] +fn first_test() { + let logger = get_logger(); + let spec = ChainSpec::few_validators(); + let validator_count = 8; + let node_count = 1; + + let state_builder = + TestingBeaconStateBuilder::from_default_keypairs_file_if_exists(validator_count, &spec); + + let (runtime, mut master, nodes) = test_setup(state_builder, node_count, &spec, logger.clone()); + + master.do_hello_with(&nodes[0]); + + runtime.shutdown_now(); +} diff --git a/eth2/types/src/test_utils/testing_beacon_state_builder.rs b/eth2/types/src/test_utils/testing_beacon_state_builder.rs index e76a01e49..7a7a902de 100644 --- a/eth2/types/src/test_utils/testing_beacon_state_builder.rs +++ b/eth2/types/src/test_utils/testing_beacon_state_builder.rs @@ -23,6 +23,7 @@ pub fn keypairs_path() -> PathBuf { /// Builds a beacon state to be used for testing purposes. /// /// This struct should **never be used for production purposes.** +#[derive(Clone)] pub struct TestingBeaconStateBuilder { state: BeaconState, keypairs: Vec, From 430702b38dbad76a082a252772081af84842e2cb Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Thu, 21 Mar 2019 18:21:26 +1100 Subject: [PATCH 02/24] Clean println's out of state processing --- beacon_node/beacon_chain/src/beacon_chain.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index bf87adf10..01787f95b 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -596,7 +596,6 @@ where // Transition the parent state to the present slot. let mut state = parent_state; - println!("parent process state: {:?}", state.latest_block_header); let previous_block_header = parent_block.into_header(); for _ in state.slot.as_u64()..present_slot.as_u64() { if let Err(e) = per_slot_processing(&mut state, &previous_block_header, &self.spec) { @@ -614,8 +613,6 @@ where )); } - println!("process state: {:?}", state.latest_block_header); - let state_root = state.canonical_root(); if block.state_root != state_root { @@ -706,8 +703,6 @@ where per_block_processing_without_verifying_block_signature(&mut state, &block, &self.spec)?; - println!("produce state: {:?}", state.latest_block_header); - let state_root = state.canonical_root(); block.state_root = state_root; From f3c81cda99ff6e85b79780c4dc8b4d0d84796025 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Thu, 21 Mar 2019 18:21:50 +1100 Subject: [PATCH 03/24] Implement hello responses. --- beacon_node/libp2p/src/rpc/methods.rs | 47 +++--- beacon_node/libp2p/src/rpc/mod.rs | 2 +- beacon_node/network/src/message_handler.rs | 27 ++-- beacon_node/network/src/service.rs | 1 - beacon_node/network/src/sync/simple_sync.rs | 158 +++++++++----------- beacon_node/network/tests/tests.rs | 33 +++- 6 files changed, 136 insertions(+), 132 deletions(-) diff --git a/beacon_node/libp2p/src/rpc/methods.rs b/beacon_node/libp2p/src/rpc/methods.rs index f05ade7ff..381fc8b01 100644 --- a/beacon_node/libp2p/src/rpc/methods.rs +++ b/beacon_node/libp2p/src/rpc/methods.rs @@ -1,4 +1,3 @@ -use beacon_chain::parking_lot::RwLockReadGuard; /// Available RPC methods types and ids. use ssz_derive::{Decode, Encode}; use types::{BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; @@ -115,75 +114,75 @@ pub struct HelloMessage { } /// Request a number of beacon block roots from a peer. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockRootsRequest { /// The starting slot of the requested blocks. - start_slot: Slot, + pub start_slot: Slot, /// The number of blocks from the start slot. - count: u64, // this must be less than 32768. //TODO: Enforce this in the lower layers + pub count: u64, // this must be less than 32768. //TODO: Enforce this in the lower layers } /// Response containing a number of beacon block roots from a peer. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockRootsResponse { /// List of requested blocks and associated slots. - roots: Vec, + pub roots: Vec, } /// Contains a block root and associated slot. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BlockRootSlot { /// The block root. - block_root: Hash256, + pub block_root: Hash256, /// The block slot. - slot: Slot, + pub slot: Slot, } /// Request a number of beacon block headers from a peer. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockHeadersRequest { /// The starting header hash of the requested headers. - start_root: Hash256, + pub start_root: Hash256, /// The starting slot of the requested headers. - start_slot: Slot, + pub start_slot: Slot, /// The maximum number of headers than can be returned. - max_headers: u64, + pub max_headers: u64, /// The maximum number of slots to skip between blocks. - skip_slots: u64, + pub skip_slots: u64, } /// Response containing requested block headers. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockHeadersResponse { /// The list of requested beacon block headers. - headers: Vec, + pub headers: Vec, } /// Request a number of beacon block bodies from a peer. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockBodiesRequest { /// The list of beacon block bodies being requested. - block_roots: Hash256, + pub block_roots: Hash256, } /// Response containing the list of requested beacon block bodies. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockBodiesResponse { /// The list of beacon block bodies being requested. - block_bodies: Vec, + pub block_bodies: Vec, } /// Request values for tree hashes which yield a blocks `state_root`. -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconChainStateRequest { /// The tree hashes that a value is requested for. - hashes: Vec, + pub hashes: Vec, } /// Request values for tree hashes which yield a blocks `state_root`. // Note: TBD -#[derive(Encode, Decode, Clone, Debug)] +#[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconChainStateResponse { /// The values corresponding the to the requested tree hashes. - values: bool, //TBD - stubbed with encodeable bool + pub values: bool, //TBD - stubbed with encodeable bool } diff --git a/beacon_node/libp2p/src/rpc/mod.rs b/beacon_node/libp2p/src/rpc/mod.rs index a1cfadafe..a1573ec93 100644 --- a/beacon_node/libp2p/src/rpc/mod.rs +++ b/beacon_node/libp2p/src/rpc/mod.rs @@ -2,7 +2,7 @@ /// /// This is purpose built for Ethereum 2.0 serenity and the protocol listens on /// `/eth/serenity/rpc/1.0.0` -mod methods; +pub mod methods; mod protocol; use futures::prelude::*; diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 8e390b4af..afd407abe 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -6,7 +6,7 @@ use crossbeam_channel::{unbounded as channel, Sender}; use futures::future; use libp2p::{ rpc::{RPCRequest, RPCResponse}, - HelloMessage, PeerId, RPCEvent, + PeerId, RPCEvent, }; use slog::debug; use slog::warn; @@ -85,7 +85,7 @@ impl MessageHandler { match message { // we have initiated a connection to a peer HandlerMessage::PeerDialed(peer_id) => { - self.sync.on_connect(&peer_id, &mut self.network_context); + self.sync.on_connect(peer_id, &mut self.network_context); } // we have received an RPC message request/response HandlerMessage::RPC(peer_id, rpc_event) => { @@ -113,7 +113,7 @@ impl MessageHandler { match request { RPCRequest::Hello(hello_message) => { self.sync - .on_hello(&peer_id, hello_message, &mut self.network_context) + .on_hello(peer_id, hello_message, &mut self.network_context) } // TODO: Handle all requests _ => {} @@ -136,26 +136,13 @@ impl MessageHandler { match response { RPCResponse::Hello(hello_message) => { debug!(self.log, "Hello response received from peer: {:?}", peer_id); - self.validate_hello(peer_id, hello_message); + self.sync + .on_hello(peer_id, hello_message, &mut self.network_context); } // TODO: Handle all responses _ => {} } } - - /// Validate a HELLO RPC message. - fn validate_hello(&mut self, peer_id: PeerId, message: HelloMessage) { - self.sync - .on_hello(&peer_id, message.clone(), &mut self.network_context); - // validate the peer - if !self.sync.validate_peer(peer_id.clone(), message) { - debug!( - self.log, - "Peer dropped due to mismatching HELLO messages: {:?}", peer_id - ); - //TODO: block/ban the peer - } - } } pub struct NetworkContext { @@ -179,6 +166,10 @@ impl NetworkContext { } } + pub fn disconnect(&self, _peer_id: PeerId) { + // TODO: disconnect peers. + } + pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest) { let id = self.generate_request_id(&peer_id); self.send_rpc_event( diff --git a/beacon_node/network/src/service.rs b/beacon_node/network/src/service.rs index d01188f5d..c3045d280 100644 --- a/beacon_node/network/src/service.rs +++ b/beacon_node/network/src/service.rs @@ -6,7 +6,6 @@ use crossbeam_channel::{unbounded as channel, Sender, TryRecvError}; use futures::prelude::*; use futures::sync::oneshot; use futures::Stream; -use libp2p::rpc::RPCResponse; use libp2p::RPCEvent; use libp2p::Service as LibP2PService; use libp2p::{Libp2pEvent, PeerId}; diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index ae6a9e7a1..0d75f3739 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -1,8 +1,9 @@ use crate::beacon_chain::BeaconChain; -use crate::message_handler::{MessageHandler, NetworkContext}; +use crate::message_handler::NetworkContext; use crate::service::NetworkMessage; use crossbeam_channel::Sender; -use libp2p::rpc::{HelloMessage, RPCMethod, RPCRequest, RPCResponse}; +use libp2p::rpc::methods::*; +use libp2p::rpc::{RPCRequest, RPCResponse}; use libp2p::PeerId; use slog::{debug, o}; use std::collections::HashMap; @@ -15,6 +16,7 @@ type NetworkSender = Sender; const SLOT_IMPORT_TOLERANCE: u64 = 100; /// Keeps track of syncing information for known connected peers. +#[derive(Clone, Copy)] pub struct PeerSyncInfo { latest_finalized_root: Hash256, latest_finalized_epoch: Epoch, @@ -23,18 +25,37 @@ pub struct PeerSyncInfo { } impl PeerSyncInfo { - pub fn is_on_chain(&self, chain: &Arc) -> bool { + fn is_on_chain(&self, chain: &Arc) -> bool { // TODO: make useful. true } - pub fn has_higher_finalized_epoch(&self, chain: &Arc) -> bool { + fn has_higher_finalized_epoch(&self, chain: &Arc) -> bool { self.latest_finalized_epoch > chain.get_state().finalized_epoch } - pub fn has_higher_best_slot(&self, chain: &Arc) -> bool { + fn has_higher_best_slot(&self, chain: &Arc) -> bool { self.latest_finalized_epoch > chain.get_state().finalized_epoch } + + pub fn status(&self, chain: &Arc) -> PeerStatus { + if self.has_higher_finalized_epoch(chain) { + PeerStatus::HigherFinalizedEpoch + } else if !self.is_on_chain(chain) { + PeerStatus::HigherFinalizedEpoch + } else if self.has_higher_best_slot(chain) { + PeerStatus::HigherBestSlot + } else { + PeerStatus::NotInteresting + } + } +} + +pub enum PeerStatus { + OnDifferentChain, + HigherFinalizedEpoch, + HigherBestSlot, + NotInteresting, } impl From for PeerSyncInfo { @@ -91,16 +112,13 @@ impl SimpleSync { } } - pub fn on_connect(&self, peer_id: &PeerId, network: &mut NetworkContext) { - network.send_rpc_request( - peer_id.clone(), - RPCRequest::Hello(self.chain.hello_message()), - ); + pub fn on_connect(&self, peer_id: PeerId, network: &mut NetworkContext) { + network.send_rpc_request(peer_id, RPCRequest::Hello(self.chain.hello_message())); } pub fn on_hello_request( - &self, - peer_id: &PeerId, + &mut self, + peer_id: PeerId, hello: HelloMessage, network: &mut NetworkContext, ) { @@ -111,97 +129,63 @@ impl SimpleSync { self.on_hello(peer_id, hello, network); } - pub fn on_hello(&self, peer_id: &PeerId, hello: HelloMessage, network: &mut NetworkContext) { + pub fn on_hello(&mut self, peer_id: PeerId, hello: HelloMessage, network: &mut NetworkContext) { + let spec = self.chain.get_spec(); + // network id must match if hello.network_id != self.network_id { debug!(self.log, "Bad network id. Peer: {:?}", peer_id); + network.disconnect(peer_id); return; } let peer = PeerSyncInfo::from(hello); - - /* - if peer.has_higher_finalized_epoch(&self.chain) { - // we need blocks - let peer_slot = peer.latest_finalized_epoch.start_slot(spec.slots_per_epoch); - let our_slot = self.chain.finalized_epoch(); - let required_slots = peer_slot - our_slot; - } else { - if !peer.is_on_chain(&self.chain) { - return (true, responses); - } - // - } - */ - - /* - // compare latest epoch and finalized root to see if they exist in our chain - if peer_info.latest_finalized_epoch <= self.latest_finalized_epoch { - // ensure their finalized root is in our chain - // TODO: Get the finalized root at hello_message.latest_epoch and ensure they match - //if (hello_message.latest_finalized_root == self.chain.get_state() { - // return false; - // } - } - - // the client is valid, add it to our list of known_peers and request sync if required - // update peer list if peer already exists - let peer_info = PeerSyncInfo::from(hello); - debug!(self.log, "Handshake successful. Peer: {:?}", peer_id); - self.known_peers.insert(peer_id, peer_info); + self.known_peers.insert(peer_id.clone(), peer); - // set state to sync - if self.state == SyncState::Idle - && hello_message.best_slot > self.latest_slot + SLOT_IMPORT_TOLERANCE - { + match peer.status(&self.chain) { + PeerStatus::OnDifferentChain => { + debug!(self.log, "Peer is on different chain. Peer: {:?}", peer_id); + + network.disconnect(peer_id); + } + PeerStatus::HigherFinalizedEpoch => { + let start_slot = peer.latest_finalized_epoch.start_slot(spec.slots_per_epoch); + let required_slots = start_slot - self.chain.slot(); + + self.request_block_roots(peer_id, start_slot, required_slots.as_u64(), network); + } + PeerStatus::HigherBestSlot => { + let start_slot = peer.best_slot; + let required_slots = start_slot - self.chain.slot(); + + self.request_block_roots(peer_id, start_slot, required_slots.as_u64(), network); + } + PeerStatus::NotInteresting => {} + } + } + + fn request_block_roots( + &mut self, + peer_id: PeerId, + start_slot: Slot, + count: u64, + network: &mut NetworkContext, + ) { + // Potentially set state to sync. + if self.state == SyncState::Idle && count > SLOT_IMPORT_TOLERANCE { self.state = SyncState::Downloading; - //TODO: Start requesting blocks from known peers. Ideally in batches } - true - */ + // TODO: handle count > max count. + network.send_rpc_request( + peer_id.clone(), + RPCRequest::BeaconBlockRoots(BeaconBlockRootsRequest { start_slot, count }), + ); } /// Generates our current state in the form of a HELLO RPC message. pub fn generate_hello(&self) -> HelloMessage { self.chain.hello_message() } - - pub fn validate_peer(&mut self, peer_id: PeerId, hello_message: HelloMessage) -> bool { - // network id must match - if hello_message.network_id != self.network_id { - return false; - } - // compare latest epoch and finalized root to see if they exist in our chain - if hello_message.latest_finalized_epoch <= self.latest_finalized_epoch { - // ensure their finalized root is in our chain - // TODO: Get the finalized root at hello_message.latest_epoch and ensure they match - //if (hello_message.latest_finalized_root == self.chain.get_state() { - // return false; - // } - } - - // the client is valid, add it to our list of known_peers and request sync if required - // update peer list if peer already exists - let peer_info = PeerSyncInfo { - latest_finalized_root: hello_message.latest_finalized_root, - latest_finalized_epoch: hello_message.latest_finalized_epoch, - best_root: hello_message.best_root, - best_slot: hello_message.best_slot, - }; - - debug!(self.log, "Handshake successful. Peer: {:?}", peer_id); - self.known_peers.insert(peer_id, peer_info); - - // set state to sync - if self.state == SyncState::Idle - && hello_message.best_slot > self.latest_slot + SLOT_IMPORT_TOLERANCE - { - self.state = SyncState::Downloading; - //TODO: Start requesting blocks from known peers. Ideally in batches - } - - true - } } diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index dc0832fed..b5635cf8c 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -1,5 +1,6 @@ use beacon_chain::test_utils::TestingBeaconChainBuilder; use crossbeam_channel::{unbounded, Receiver, Sender}; +use libp2p::rpc::methods::*; use libp2p::rpc::{HelloMessage, RPCMethod, RPCRequest, RPCResponse}; use libp2p::{PeerId, RPCEvent}; use network::beacon_chain::BeaconChain as NetworkBeaconChain; @@ -9,6 +10,7 @@ use sloggers::terminal::{Destination, TerminalLoggerBuilder}; use sloggers::types::Severity; use sloggers::Build; use std::sync::Arc; +use std::time::Duration; use test_harness::BeaconChainHarness; use tokio::runtime::TaskExecutor; use types::{test_utils::TestingBeaconStateBuilder, *}; @@ -42,7 +44,9 @@ impl SyncNode { } fn recv(&self) -> NetworkMessage { - self.receiver.recv().unwrap() + self.receiver + .recv_timeout(Duration::from_millis(500)) + .unwrap() } fn recv_rpc_response(&self) -> RPCResponse { @@ -106,6 +110,12 @@ impl SyncMaster { } } + pub fn build_blocks(&mut self, blocks: usize) { + for _ in 0..blocks { + self.harness.advance_chain_with_block(); + } + } + pub fn response_id(&mut self, node: &SyncNode) -> u64 { let id = self.response_ids[node.id]; self.response_ids[node.id] += 1; @@ -140,6 +150,17 @@ impl SyncMaster { } } +fn assert_sent_block_root_request(node: &SyncNode, expected: BeaconBlockRootsRequest) { + let request = node.recv_rpc_request(); + + match request { + RPCRequest::BeaconBlockRoots(response) => { + assert_eq!(expected, response, "Bad block roots response"); + } + _ => assert!(false, "Did not get block root request"), + } +} + fn test_setup( state_builder: TestingBeaconStateBuilder, node_count: usize, @@ -178,7 +199,17 @@ fn first_test() { let (runtime, mut master, nodes) = test_setup(state_builder, node_count, &spec, logger.clone()); + master.build_blocks(10); + master.do_hello_with(&nodes[0]); + assert_sent_block_root_request( + &nodes[0], + BeaconBlockRootsRequest { + start_slot: Slot::new(1), + count: 10, + }, + ); + runtime.shutdown_now(); } From 4fc6e435d2a0c79c93fd1dc474404da6a3a67ce3 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Fri, 22 Mar 2019 11:38:57 +1100 Subject: [PATCH 04/24] Add type alias to test_harness --- .../beacon_chain/test_harness/src/beacon_chain_harness.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs b/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs index 4b5d5558f..1207fcf28 100644 --- a/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs +++ b/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs @@ -15,6 +15,8 @@ use std::iter::FromIterator; use std::sync::Arc; use types::{test_utils::TestingBeaconStateBuilder, *}; +type TestingBeaconChain = BeaconChain>; + /// The beacon chain harness simulates a single beacon node with `validator_count` validators connected /// to it. Each validator is provided a borrow to the beacon chain, where it may read /// information and submit blocks/attestations for processing. @@ -23,7 +25,7 @@ use types::{test_utils::TestingBeaconStateBuilder, *}; /// is not useful for testing that multiple beacon nodes can reach consensus. pub struct BeaconChainHarness { pub db: Arc, - pub beacon_chain: Arc>>, + pub beacon_chain: Arc, pub block_store: Arc>, pub state_store: Arc>, pub validators: Vec, From f96a3282b5072b6f30e16b9e94cccbfe64d7a3a7 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Fri, 22 Mar 2019 11:39:16 +1100 Subject: [PATCH 05/24] Pass first sync test --- beacon_node/network/src/sync/simple_sync.rs | 85 ++++++++++----------- beacon_node/network/tests/tests.rs | 62 ++++++++++----- 2 files changed, 81 insertions(+), 66 deletions(-) diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 6a40ecf60..0b2f736c1 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -1,7 +1,5 @@ use crate::beacon_chain::BeaconChain; use crate::message_handler::NetworkContext; -use crate::service::NetworkMessage; -use crossbeam_channel::Sender; use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{RPCRequest, RPCResponse}; use eth2_libp2p::PeerId; @@ -10,14 +8,13 @@ use std::collections::HashMap; use std::sync::Arc; use types::{Epoch, Hash256, Slot}; -type NetworkSender = Sender; - /// The number of slots that we can import blocks ahead of us, before going into full Sync mode. const SLOT_IMPORT_TOLERANCE: u64 = 100; /// Keeps track of syncing information for known connected peers. -#[derive(Clone, Copy)] +#[derive(Clone, Copy, Debug)] pub struct PeerSyncInfo { + network_id: u8, latest_finalized_root: Hash256, latest_finalized_epoch: Epoch, best_root: Hash256, @@ -25,25 +22,24 @@ pub struct PeerSyncInfo { } impl PeerSyncInfo { - fn is_on_chain(&self, chain: &Arc) -> bool { - // TODO: make useful. - true + fn is_on_same_chain(&self, other: Self) -> bool { + self.network_id == other.network_id } - fn has_higher_finalized_epoch(&self, chain: &Arc) -> bool { - self.latest_finalized_epoch > chain.get_state().finalized_epoch + fn has_higher_finalized_epoch_than(&self, other: Self) -> bool { + self.latest_finalized_epoch > other.latest_finalized_epoch } - fn has_higher_best_slot(&self, chain: &Arc) -> bool { - self.latest_finalized_epoch > chain.get_state().finalized_epoch + fn has_higher_best_slot_than(&self, other: Self) -> bool { + self.best_slot > other.best_slot } - pub fn status(&self, chain: &Arc) -> PeerStatus { - if self.has_higher_finalized_epoch(chain) { + pub fn status_compared_to(&self, other: Self) -> PeerStatus { + if self.has_higher_finalized_epoch_than(other) { PeerStatus::HigherFinalizedEpoch - } else if !self.is_on_chain(chain) { - PeerStatus::HigherFinalizedEpoch - } else if self.has_higher_best_slot(chain) { + } else if !self.is_on_same_chain(other) { + PeerStatus::OnDifferentChain + } else if self.has_higher_best_slot_than(other) { PeerStatus::HigherBestSlot } else { PeerStatus::NotInteresting @@ -62,6 +58,7 @@ pub enum PeerStatus { impl From for PeerSyncInfo { fn from(hello: HelloMessage) -> PeerSyncInfo { PeerSyncInfo { + network_id: hello.network_id, latest_finalized_root: hello.latest_finalized_root, latest_finalized_epoch: hello.latest_finalized_epoch, best_root: hello.best_root, @@ -70,6 +67,12 @@ impl From for PeerSyncInfo { } } +impl From<&Arc> for PeerSyncInfo { + fn from(chain: &Arc) -> PeerSyncInfo { + Self::from(chain.hello_message()) + } +} + /// The current syncing state. #[derive(PartialEq)] pub enum SyncState { @@ -88,12 +91,6 @@ pub struct SimpleSync { known_peers: HashMap, /// The current state of the syncing protocol. state: SyncState, - /// The network id, for quick HELLO RPC message lookup. - network_id: u8, - /// The latest epoch of the syncing chain. - latest_finalized_epoch: Epoch, - /// The latest block of the syncing chain. - latest_slot: Slot, /// Sync logger. log: slog::Logger, } @@ -106,9 +103,6 @@ impl SimpleSync { chain: beacon_chain.clone(), known_peers: HashMap::new(), state: SyncState::Idle, - network_id: beacon_chain.get_spec().network_id, - latest_finalized_epoch: state.finalized_epoch, - latest_slot: state.slot - 1, //TODO: Build latest block function into Beacon chain and correct this log: sync_logger, } } @@ -133,40 +127,39 @@ impl SimpleSync { pub fn on_hello(&mut self, peer_id: PeerId, hello: HelloMessage, network: &mut NetworkContext) { let spec = self.chain.get_spec(); + let remote = PeerSyncInfo::from(hello); + let local = PeerSyncInfo::from(&self.chain); + let remote_status = remote.status_compared_to(local); + // network id must match - if hello.network_id != self.network_id { - debug!(self.log, "Bad network id. Peer: {:?}", peer_id); - network.disconnect(peer_id); - return; + if remote_status != PeerStatus::OnDifferentChain { + debug!(self.log, "Handshake successful. Peer: {:?}", peer_id); + self.known_peers.insert(peer_id.clone(), remote); } - let peer = PeerSyncInfo::from(hello); - debug!(self.log, "Handshake successful. Peer: {:?}", peer_id); - self.known_peers.insert(peer_id.clone(), peer); - - debug!( - self.log, - "Peer hello. Status: {:?}", - peer.status(&self.chain) - ); - - match peer.status(&self.chain) { + match remote_status { PeerStatus::OnDifferentChain => { debug!(self.log, "Peer is on different chain. Peer: {:?}", peer_id); network.disconnect(peer_id); } PeerStatus::HigherFinalizedEpoch => { - let start_slot = peer.latest_finalized_epoch.start_slot(spec.slots_per_epoch); - let required_slots = start_slot - self.chain.slot(); + let start_slot = remote + .latest_finalized_epoch + .start_slot(spec.slots_per_epoch); + let required_slots = start_slot - local.best_slot; self.request_block_roots(peer_id, start_slot, required_slots.as_u64(), network); } PeerStatus::HigherBestSlot => { - let start_slot = peer.best_slot; - let required_slots = start_slot - self.chain.slot(); + let required_slots = remote.best_slot - local.best_slot; - self.request_block_roots(peer_id, start_slot, required_slots.as_u64(), network); + self.request_block_roots( + peer_id, + local.best_slot, + required_slots.as_u64(), + network, + ); } PeerStatus::NotInteresting => {} } diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index 7941ffb99..dea57982e 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -1,4 +1,3 @@ -use beacon_chain::test_utils::TestingBeaconChainBuilder; use crossbeam_channel::{unbounded, Receiver, RecvTimeoutError, Sender}; use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{RPCMethod, RPCRequest, RPCResponse}; @@ -9,7 +8,6 @@ use network::service::{NetworkMessage, OutgoingMessage}; use sloggers::terminal::{Destination, TerminalLoggerBuilder}; use sloggers::types::Severity; use sloggers::Build; -use std::sync::Arc; use std::time::Duration; use test_harness::BeaconChainHarness; use tokio::runtime::TaskExecutor; @@ -19,26 +17,40 @@ pub struct SyncNode { pub id: usize, sender: Sender, receiver: Receiver, + harness: BeaconChainHarness, } impl SyncNode { - pub fn new( + fn from_beacon_state_builder( id: usize, executor: &TaskExecutor, - chain: Arc, + state_builder: TestingBeaconStateBuilder, + spec: &ChainSpec, logger: slog::Logger, ) -> Self { + let harness = BeaconChainHarness::from_beacon_state_builder(state_builder, spec.clone()); + let (network_sender, network_receiver) = unbounded(); - let message_handler_sender = - MessageHandler::spawn(chain, network_sender, executor, logger).unwrap(); + let message_handler_sender = MessageHandler::spawn( + harness.beacon_chain.clone(), + network_sender, + executor, + logger, + ) + .unwrap(); Self { id, sender: message_handler_sender, receiver: network_receiver, + harness, } } + fn increment_beacon_chain_slot(&mut self) { + self.harness.increment_beacon_chain_slot(); + } + fn send(&self, message: HandlerMessage) { self.sender.send(message).unwrap(); } @@ -47,7 +59,11 @@ impl SyncNode { self.receiver.recv_timeout(Duration::from_millis(500)) } - fn recv_rpc_response(&self) -> Result { + fn hello_message(&self) -> HelloMessage { + self.harness.beacon_chain.hello_message() + } + + fn _recv_rpc_response(&self) -> Result { let network_message = self.recv()?; Ok(match network_message { NetworkMessage::Send( @@ -108,12 +124,6 @@ impl SyncMaster { } } - pub fn build_blocks(&mut self, blocks: usize) { - for _ in 0..blocks { - self.harness.advance_chain_with_block(); - } - } - pub fn response_id(&mut self, node: &SyncNode) -> u64 { let id = self.response_ids[node.id]; self.response_ids[node.id] += 1; @@ -169,11 +179,11 @@ fn test_setup( let mut nodes = Vec::with_capacity(node_count); for id in 0..node_count { - let local_chain = TestingBeaconChainBuilder::from(state_builder.clone()).build(&spec); - let node = SyncNode::new( + let node = SyncNode::from_beacon_state_builder( id, &runtime.executor(), - Arc::new(local_chain), + state_builder.clone(), + &spec, logger.clone(), ); @@ -185,6 +195,15 @@ fn test_setup( (runtime, master, nodes) } +pub fn build_blocks(blocks: usize, master: &mut SyncMaster, nodes: &mut Vec) { + for _ in 0..blocks { + master.harness.advance_chain_with_block(); + for i in 0..nodes.len() { + nodes[i].increment_beacon_chain_slot(); + } + } +} + #[test] fn first_test() { let logger = get_logger(); @@ -195,17 +214,20 @@ fn first_test() { let state_builder = TestingBeaconStateBuilder::from_default_keypairs_file_if_exists(validator_count, &spec); - let (runtime, mut master, nodes) = test_setup(state_builder, node_count, &spec, logger.clone()); + let (runtime, mut master, mut nodes) = + test_setup(state_builder, node_count, &spec, logger.clone()); - master.build_blocks(10); + let original_node_slot = nodes[0].hello_message().best_slot; + + build_blocks(2, &mut master, &mut nodes); master.do_hello_with(&nodes[0]); assert_sent_block_root_request( &nodes[0], BeaconBlockRootsRequest { - start_slot: Slot::new(1), - count: 10, + start_slot: original_node_slot, + count: 2, }, ); From 96ba1c8f77064a24c3a8ab1b03954d80d333c21b Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Fri, 22 Mar 2019 14:20:49 +1100 Subject: [PATCH 06/24] Implement `get_block_roots` for syncing --- beacon_node/beacon_chain/src/beacon_chain.rs | 64 ++++++++++++++++++++ beacon_node/network/src/beacon_chain.rs | 16 ++++- beacon_node/network/src/message_handler.rs | 13 +++- beacon_node/network/src/sync/simple_sync.rs | 12 ++++ beacon_node/network/tests/tests.rs | 61 +++++++++++++------ 5 files changed, 146 insertions(+), 20 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 4b151d70b..dccd9842e 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -122,6 +122,70 @@ where }) } + /// Returns beacon block roots for `count` slots, starting from `start_slot`. + /// + /// ## Errors: + /// + /// - `SlotOutOfBounds`: Unable to return the full specified range. + /// - `SlotOutOfBounds`: Unable to load a state from the DB. + /// - `SlotOutOfBounds`: Start slot is higher than the first slot. + /// - Other: BeaconState` is inconsistent. + pub fn get_block_roots( + &self, + start_slot: Slot, + count: Slot, + ) -> Result, BeaconStateError> { + let spec = &self.spec; + + let mut roots: Vec = vec![]; + let mut state = self.state.read().clone(); + let mut slot = start_slot + count - 1; + + loop { + // Return if the slot required is greater than the current state. + if slot >= state.slot { + return Err(BeaconStateError::SlotOutOfBounds); + } + + // If the slot is within the range of the current state's block roots, append the root + // to the output vec. + // + // If we get `SlotOutOfBounds` error, load the oldest known state to the present state + // from the DB. + match state.get_block_root(slot, spec) { + Ok(root) => { + roots.push(*root); + + if slot == start_slot { + break; + } else { + slot -= 1; + } + } + Err(BeaconStateError::SlotOutOfBounds) => { + // Read the earliest historic state in the current slot. + let earliest_historic_slot = + state.slot - Slot::from(spec.slots_per_historical_root); + // Load the earlier state from disk. + let new_state_root = state.get_state_root(earliest_historic_slot, spec)?; + + // Break if the DB is unable to load the state. + state = match self.state_store.get_deserialized(&new_state_root) { + Ok(Some(state)) => state, + _ => break, + } + } + Err(e) => return Err(e), + }; + } + + if (slot == start_slot) && (roots.len() == count.as_usize()) { + Ok(roots) + } else { + Err(BeaconStateError::SlotOutOfBounds) + } + } + /// Update the canonical head to some new values. pub fn update_canonical_head( &self, diff --git a/beacon_node/network/src/beacon_chain.rs b/beacon_node/network/src/beacon_chain.rs index 5246c87c2..ba429e688 100644 --- a/beacon_node/network/src/beacon_chain.rs +++ b/beacon_node/network/src/beacon_chain.rs @@ -8,7 +8,7 @@ use beacon_chain::{ CheckPoint, }; use eth2_libp2p::HelloMessage; -use types::{Epoch, Hash256, Slot}; +use types::{BeaconStateError, Epoch, Hash256, Slot}; /// The network's API to the beacon chain. pub trait BeaconChain: Send + Sync { @@ -29,6 +29,12 @@ pub trait BeaconChain: Send + Sync { fn finalized_epoch(&self) -> Epoch; fn hello_message(&self) -> HelloMessage; + + fn get_block_roots( + &self, + start_slot: Slot, + count: Slot, + ) -> Result, BeaconStateError>; } impl BeaconChain for RawBeaconChain @@ -81,4 +87,12 @@ where best_slot: self.best_slot(), } } + + fn get_block_roots( + &self, + start_slot: Slot, + count: Slot, + ) -> Result, BeaconStateError> { + self.get_block_roots(start_slot, count) + } } diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 5b39de997..99a263ed8 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -139,8 +139,19 @@ impl MessageHandler { self.sync .on_hello(peer_id, hello_message, &mut self.network_context); } + RPCResponse::BeaconBlockRoots(response) => { + debug!( + self.log, + "BeaconBlockRoots response received from peer: {:?}", peer_id + ); + self.sync.on_beacon_block_roots_response( + peer_id, + response, + &mut self.network_context, + ) + } // TODO: Handle all responses - _ => {} + _ => panic!("Unknown response: {:?}", response), } } } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 0b2f736c1..ab29d0db5 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -165,6 +165,15 @@ impl SimpleSync { } } + pub fn on_beacon_block_roots_response( + &mut self, + peer_id: PeerId, + reponse: BeaconBlockRootsResponse, + network: &mut NetworkContext, + ) { + // + } + fn request_block_roots( &mut self, peer_id: PeerId, @@ -174,9 +183,12 @@ impl SimpleSync { ) { // Potentially set state to sync. if self.state == SyncState::Idle && count > SLOT_IMPORT_TOLERANCE { + debug!(self.log, "Entering downloading sync state."); self.state = SyncState::Downloading; } + debug!(self.log, "Requesting {} blocks from {:?}.", count, &peer_id); + // TODO: handle count > max count. network.send_rpc_request( peer_id.clone(), diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index dea57982e..fbfb827f2 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -63,6 +63,15 @@ impl SyncNode { self.harness.beacon_chain.hello_message() } + pub fn get_block_root_request(&self) -> BeaconBlockRootsRequest { + let request = self.recv_rpc_request().expect("No block root request"); + + match request { + RPCRequest::BeaconBlockRoots(response) => response, + _ => panic!("Did not get block root request"), + } + } + fn _recv_rpc_response(&self) -> Result { let network_message = self.recv()?; Ok(match network_message { @@ -146,6 +155,34 @@ impl SyncMaster { } } + pub fn respond_to_block_roots_request( + &mut self, + node: &SyncNode, + request: BeaconBlockRootsRequest, + ) { + let roots = self + .harness + .beacon_chain + .get_block_roots(request.start_slot, Slot::from(request.count)) + .expect("Beacon chain did not give blocks"); + + let roots = roots + .iter() + .enumerate() + .map(|(i, root)| BlockRootSlot { + block_root: *root, + slot: Slot::from(i) + request.start_slot, + }) + .collect(); + + let response = RPCResponse::BeaconBlockRoots(BeaconBlockRootsResponse { roots }); + self.send_rpc_response(node, response) + } + + fn send_rpc_response(&mut self, node: &SyncNode, rpc_response: RPCResponse) { + node.send(self.rpc_response(node, rpc_response)); + } + fn rpc_response(&mut self, node: &SyncNode, rpc_response: RPCResponse) -> HandlerMessage { HandlerMessage::RPC( self.peer_id.clone(), @@ -158,17 +195,6 @@ impl SyncMaster { } } -fn assert_sent_block_root_request(node: &SyncNode, expected: BeaconBlockRootsRequest) { - let request = node.recv_rpc_request().expect("No block root request"); - - match request { - RPCRequest::BeaconBlockRoots(response) => { - assert_eq!(expected, response, "Bad block roots response"); - } - _ => assert!(false, "Did not get block root request"), - } -} - fn test_setup( state_builder: TestingBeaconStateBuilder, node_count: usize, @@ -223,13 +249,12 @@ fn first_test() { master.do_hello_with(&nodes[0]); - assert_sent_block_root_request( - &nodes[0], - BeaconBlockRootsRequest { - start_slot: original_node_slot, - count: 2, - }, - ); + let request = nodes[0].get_block_root_request(); + assert_eq!(request.start_slot, original_node_slot); + assert_eq!(request.count, 2); + master.respond_to_block_roots_request(&nodes[0], request); + + std::thread::sleep(Duration::from_millis(500)); runtime.shutdown_now(); } From 4b5b5851a64a4cd2920bb7f4ba0a10b6a5d1ac0b Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sat, 23 Mar 2019 13:23:44 +1100 Subject: [PATCH 07/24] Allow sync to to request block bodies. --- beacon_node/beacon_chain/src/beacon_chain.rs | 30 ++- beacon_node/eth2-libp2p/src/rpc/methods.rs | 2 +- beacon_node/network/Cargo.toml | 1 + beacon_node/network/src/beacon_chain.rs | 16 +- beacon_node/network/src/message_handler.rs | 17 +- beacon_node/network/src/sync/simple_sync.rs | 200 +++++++++++++++++-- beacon_node/network/tests/tests.rs | 72 ++++++- 7 files changed, 305 insertions(+), 33 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index dccd9842e..33198f0a3 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -142,8 +142,14 @@ where let mut slot = start_slot + count - 1; loop { - // Return if the slot required is greater than the current state. - if slot >= state.slot { + // If the highest slot requested is that of the current state insert the root of the + // head block, unless the head block's slot is not matching. + if slot == state.slot && self.head().beacon_block.slot == slot { + roots.push(self.head().beacon_block_root); + + slot -= 1; + continue; + } else if slot >= state.slot { return Err(BeaconStateError::SlotOutOfBounds); } @@ -180,12 +186,25 @@ where } if (slot == start_slot) && (roots.len() == count.as_usize()) { - Ok(roots) + // Reverse the ordering of the roots. We extracted them in reverse order to make it + // simpler to lookup historic states. + // + // This is a potential optimisation target. + Ok(roots.iter().rev().cloned().collect()) } else { Err(BeaconStateError::SlotOutOfBounds) } } + /// Returns the block at the given root, if any. + /// + /// ## Errors + /// + /// May return a database error. + pub fn get_block(&self, block_root: &Hash256) -> Result, Error> { + Ok(self.block_store.get_deserialized(block_root)?) + } + /// Update the canonical head to some new values. pub fn update_canonical_head( &self, @@ -622,6 +641,11 @@ where } } + /// Returns `true` if the given block root has not been processed. + pub fn is_new_block_root(&self, beacon_block_root: &Hash256) -> Result { + Ok(!self.block_store.exists(beacon_block_root)?) + } + /// Accept some block and attempt to add it to block DAG. /// /// Will accept blocks from prior slots, however it will reject any block from a future slot. diff --git a/beacon_node/eth2-libp2p/src/rpc/methods.rs b/beacon_node/eth2-libp2p/src/rpc/methods.rs index 381fc8b01..f6a5f2829 100644 --- a/beacon_node/eth2-libp2p/src/rpc/methods.rs +++ b/beacon_node/eth2-libp2p/src/rpc/methods.rs @@ -162,7 +162,7 @@ pub struct BeaconBlockHeadersResponse { #[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockBodiesRequest { /// The list of beacon block bodies being requested. - pub block_roots: Hash256, + pub block_roots: Vec, } /// Response containing the list of requested beacon block bodies. diff --git a/beacon_node/network/Cargo.toml b/beacon_node/network/Cargo.toml index a53097159..c6411a020 100644 --- a/beacon_node/network/Cargo.toml +++ b/beacon_node/network/Cargo.toml @@ -14,6 +14,7 @@ eth2-libp2p = { path = "../eth2-libp2p" } version = { path = "../version" } types = { path = "../../eth2/types" } slog = "2.4.1" +ssz = { path = "../../eth2/utils/ssz" } futures = "0.1.25" error-chain = "0.12.0" crossbeam-channel = "0.3.8" diff --git a/beacon_node/network/src/beacon_chain.rs b/beacon_node/network/src/beacon_chain.rs index ba429e688..e2829cfa6 100644 --- a/beacon_node/network/src/beacon_chain.rs +++ b/beacon_node/network/src/beacon_chain.rs @@ -8,7 +8,9 @@ use beacon_chain::{ CheckPoint, }; use eth2_libp2p::HelloMessage; -use types::{BeaconStateError, Epoch, Hash256, Slot}; +use types::{BeaconBlock, BeaconStateError, Epoch, Hash256, Slot}; + +pub use beacon_chain::BeaconChainError; /// The network's API to the beacon chain. pub trait BeaconChain: Send + Sync { @@ -20,6 +22,8 @@ pub trait BeaconChain: Send + Sync { fn head(&self) -> RwLockReadGuard; + fn get_block(&self, block_root: &Hash256) -> Result, BeaconChainError>; + fn best_slot(&self) -> Slot; fn best_block_root(&self) -> Hash256; @@ -35,6 +39,8 @@ pub trait BeaconChain: Send + Sync { start_slot: Slot, count: Slot, ) -> Result, BeaconStateError>; + + fn is_new_block_root(&self, beacon_block_root: &Hash256) -> Result; } impl BeaconChain for RawBeaconChain @@ -59,6 +65,10 @@ where self.head() } + fn get_block(&self, block_root: &Hash256) -> Result, BeaconChainError> { + self.get_block(block_root) + } + fn finalized_epoch(&self) -> Epoch { self.get_state().finalized_epoch } @@ -95,4 +105,8 @@ where ) -> Result, BeaconStateError> { self.get_block_roots(start_slot, count) } + + fn is_new_block_root(&self, beacon_block_root: &Hash256) -> Result { + self.is_new_block_root(beacon_block_root) + } } diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 99a263ed8..1a790eee1 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -142,7 +142,7 @@ impl MessageHandler { RPCResponse::BeaconBlockRoots(response) => { debug!( self.log, - "BeaconBlockRoots response received from peer: {:?}", peer_id + "BeaconBlockRoots response received"; "peer" => format!("{:?}", peer_id) ); self.sync.on_beacon_block_roots_response( peer_id, @@ -150,6 +150,17 @@ impl MessageHandler { &mut self.network_context, ) } + RPCResponse::BeaconBlockHeaders(response) => { + debug!( + self.log, + "BeaconBlockHeaders response received"; "peer" => format!("{:?}", peer_id) + ); + self.sync.on_beacon_block_headers_response( + peer_id, + response, + &mut self.network_context, + ) + } // TODO: Handle all responses _ => panic!("Unknown response: {:?}", response), } @@ -233,10 +244,6 @@ impl NetworkContext { }; // register RPC request self.requests.insert((peer_id.clone(), id), Instant::now()); - debug!( - self.log, - "Hello request registered with peer: {:?}", peer_id - ); id } } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index ab29d0db5..ee0646dbb 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -3,10 +3,12 @@ use crate::message_handler::NetworkContext; use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{RPCRequest, RPCResponse}; use eth2_libp2p::PeerId; -use slog::{debug, o}; +use slog::{debug, error, o, warn}; +use ssz::TreeHash; use std::collections::HashMap; use std::sync::Arc; -use types::{Epoch, Hash256, Slot}; +use std::time::Instant; +use types::{BeaconBlockHeader, Epoch, Hash256, Slot}; /// The number of slots that we can import blocks ahead of us, before going into full Sync mode. const SLOT_IMPORT_TOLERANCE: u64 = 100; @@ -78,7 +80,7 @@ impl From<&Arc> for PeerSyncInfo { pub enum SyncState { Idle, Downloading, - Stopped, + _Stopped, } /// Simple Syncing protocol. @@ -89,6 +91,8 @@ pub struct SimpleSync { chain: Arc, /// A mapping of Peers to their respective PeerSyncInfo. known_peers: HashMap, + /// A queue to allow importing of blocks + import_queue: ImportQueue, /// The current state of the syncing protocol. state: SyncState, /// Sync logger. @@ -97,11 +101,12 @@ pub struct SimpleSync { impl SimpleSync { pub fn new(beacon_chain: Arc, log: &slog::Logger) -> Self { - let state = beacon_chain.get_state(); let sync_logger = log.new(o!("Service"=> "Sync")); + let import_queue = ImportQueue::new(beacon_chain.clone(), log.clone()); SimpleSync { chain: beacon_chain.clone(), known_peers: HashMap::new(), + import_queue, state: SyncState::Idle, log: sync_logger, } @@ -149,15 +154,24 @@ impl SimpleSync { .start_slot(spec.slots_per_epoch); let required_slots = start_slot - local.best_slot; - self.request_block_roots(peer_id, start_slot, required_slots.as_u64(), network); + self.request_block_roots( + peer_id, + BeaconBlockRootsRequest { + start_slot, + count: required_slots.into(), + }, + network, + ); } PeerStatus::HigherBestSlot => { let required_slots = remote.best_slot - local.best_slot; self.request_block_roots( peer_id, - local.best_slot, - required_slots.as_u64(), + BeaconBlockRootsRequest { + start_slot: local.best_slot + 1, + count: required_slots.into(), + }, network, ); } @@ -168,32 +182,109 @@ impl SimpleSync { pub fn on_beacon_block_roots_response( &mut self, peer_id: PeerId, - reponse: BeaconBlockRootsResponse, + response: BeaconBlockRootsResponse, network: &mut NetworkContext, ) { + if response.roots.is_empty() { + warn!( + self.log, + "Peer returned empty block roots response. PeerId: {:?}", peer_id + ); + return; + } + + let new_root_index = self.import_queue.first_new_root(&response.roots); + + // If a new block root is found, request it and all the headers following it. // + // We make an assumption here that if we don't know a block then we don't know of all + // it's parents. This might not be the case if syncing becomes more sophisticated. + if let Some(i) = new_root_index { + let new = &response.roots[i]; + + self.request_block_headers( + peer_id, + BeaconBlockHeadersRequest { + start_root: new.block_root, + start_slot: new.slot, + max_headers: (response.roots.len() - i) as u64, + skip_slots: 0, + }, + network, + ) + } + } + + pub fn on_beacon_block_headers_response( + &mut self, + peer_id: PeerId, + response: BeaconBlockHeadersResponse, + network: &mut NetworkContext, + ) { + if response.headers.is_empty() { + warn!( + self.log, + "Peer returned empty block headers response. PeerId: {:?}", peer_id + ); + return; + } + + let block_roots = self.import_queue.enqueue_headers(response.headers); + + if !block_roots.is_empty() { + self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network); + } } fn request_block_roots( &mut self, peer_id: PeerId, - start_slot: Slot, - count: u64, + request: BeaconBlockRootsRequest, network: &mut NetworkContext, ) { // Potentially set state to sync. - if self.state == SyncState::Idle && count > SLOT_IMPORT_TOLERANCE { + if self.state == SyncState::Idle && request.count > SLOT_IMPORT_TOLERANCE { debug!(self.log, "Entering downloading sync state."); self.state = SyncState::Downloading; } - debug!(self.log, "Requesting {} blocks from {:?}.", count, &peer_id); + debug!( + self.log, + "Requesting {} block roots from {:?}.", request.count, &peer_id + ); // TODO: handle count > max count. - network.send_rpc_request( - peer_id.clone(), - RPCRequest::BeaconBlockRoots(BeaconBlockRootsRequest { start_slot, count }), + network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockRoots(request)); + } + + fn request_block_headers( + &mut self, + peer_id: PeerId, + request: BeaconBlockHeadersRequest, + network: &mut NetworkContext, + ) { + debug!( + self.log, + "Requesting {} headers from {:?}.", request.max_headers, &peer_id ); + + network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockHeaders(request)); + } + + fn request_block_bodies( + &mut self, + peer_id: PeerId, + request: BeaconBlockBodiesRequest, + network: &mut NetworkContext, + ) { + debug!( + self.log, + "Requesting {} bodies from {:?}.", + request.block_roots.len(), + &peer_id + ); + + network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockBodies(request)); } /// Generates our current state in the form of a HELLO RPC message. @@ -201,3 +292,82 @@ impl SimpleSync { self.chain.hello_message() } } + +pub struct ImportQueue { + /// BeaconChain + pub chain: Arc, + /// Partially imported blocks, keyed by the root of `BeaconBlockBody`. + pub partials: HashMap, + /// Logging + log: slog::Logger, +} + +impl ImportQueue { + pub fn new(chain: Arc, log: slog::Logger) -> Self { + Self { + chain, + partials: HashMap::new(), + log, + } + } + + fn is_new_block(&self, block_root: &Hash256) -> bool { + self.chain + .is_new_block_root(&block_root) + .unwrap_or_else(|_| { + error!(self.log, "Unable to determine if block is new."); + true + }) + } + + /// Returns the index of the first new root in the list of block roots. + pub fn first_new_root(&mut self, roots: &[BlockRootSlot]) -> Option { + for root in roots { + println!("root {}", root.block_root); + } + roots + .iter() + .position(|brs| self.is_new_block(&brs.block_root)) + } + + /// Adds the `headers` to the `partials` queue. Returns a list of `Hash256` block roots for + /// which we should use to request `BeaconBlockBodies`. + /// + /// If a `header` is not in the queue and has not been processed by the chain it is added to + /// the queue and it's block root is included in the output. + /// + /// If a `header` is already in the queue, but not yet processed by the chain the block root is + /// included in the output and the `inserted` time for the partial record is set to + /// `Instant::now()`. Updating the `inserted` time stops the partial from becoming stale. + pub fn enqueue_headers(&mut self, headers: Vec) -> Vec { + let mut required_bodies: Vec = vec![]; + + for header in headers { + let block_root = Hash256::from_slice(&header.hash_tree_root()[..]); + + if self.is_new_block(&block_root) { + self.insert_partial(block_root, header); + required_bodies.push(block_root) + } + } + + required_bodies + } + + fn insert_partial(&mut self, block_root: Hash256, header: BeaconBlockHeader) { + self.partials.insert( + header.block_body_root, + PartialBeaconBlock { + block_root, + header, + inserted: Instant::now(), + }, + ); + } +} + +pub struct PartialBeaconBlock { + pub block_root: Hash256, + pub header: BeaconBlockHeader, + pub inserted: Instant, +} diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index fbfb827f2..076a3f529 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -67,7 +67,16 @@ impl SyncNode { let request = self.recv_rpc_request().expect("No block root request"); match request { - RPCRequest::BeaconBlockRoots(response) => response, + RPCRequest::BeaconBlockRoots(request) => request, + _ => panic!("Did not get block root request"), + } + } + + pub fn get_block_headers_request(&self) -> BeaconBlockHeadersRequest { + let request = self.recv_rpc_request().expect("No block headers request"); + + match request { + RPCRequest::BeaconBlockHeaders(request) => request, _ => panic!("Did not get block root request"), } } @@ -164,9 +173,7 @@ impl SyncMaster { .harness .beacon_chain .get_block_roots(request.start_slot, Slot::from(request.count)) - .expect("Beacon chain did not give blocks"); - - let roots = roots + .expect("Beacon chain did not give block roots") .iter() .enumerate() .map(|(i, root)| BlockRootSlot { @@ -179,6 +186,43 @@ impl SyncMaster { self.send_rpc_response(node, response) } + pub fn respond_to_block_headers_request( + &mut self, + node: &SyncNode, + request: BeaconBlockHeadersRequest, + ) { + let roots = self + .harness + .beacon_chain + .get_block_roots(request.start_slot, Slot::from(request.max_headers)) + .expect("Beacon chain did not give blocks"); + + if roots.is_empty() { + panic!("Roots was empty when trying to get headers.") + } + + assert_eq!( + roots[0], request.start_root, + "Got the wrong start root when getting headers" + ); + + let headers: Vec = roots + .iter() + .map(|root| { + let block = self + .harness + .beacon_chain + .get_block(root) + .expect("Failed to load block") + .expect("Block did not exist"); + block.block_header() + }) + .collect(); + + let response = RPCResponse::BeaconBlockHeaders(BeaconBlockHeadersResponse { headers }); + self.send_rpc_response(node, response) + } + fn send_rpc_response(&mut self, node: &SyncNode, rpc_response: RPCResponse) { node.send(self.rpc_response(node, rpc_response)); } @@ -228,6 +272,11 @@ pub fn build_blocks(blocks: usize, master: &mut SyncMaster, nodes: &mut Vec Date: Sat, 23 Mar 2019 18:48:09 +1100 Subject: [PATCH 08/24] Implement block imports for sync --- beacon_node/beacon_chain/src/beacon_chain.rs | 24 ++- beacon_node/network/src/beacon_chain.rs | 12 +- beacon_node/network/src/message_handler.rs | 11 ++ beacon_node/network/src/sync/simple_sync.rs | 160 +++++++++++++++++-- beacon_node/network/tests/tests.rs | 41 ++++- eth2/types/src/beacon_block_header.rs | 13 ++ 6 files changed, 239 insertions(+), 22 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 33198f0a3..eb8df6f2a 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -46,6 +46,26 @@ pub enum BlockProcessingOutcome { InvalidBlock(InvalidBlock), } +impl BlockProcessingOutcome { + /// Returns `true` if the block was objectively invalid and we should disregard the peer who + /// sent it. + pub fn is_invalid(&self) -> bool { + match self { + BlockProcessingOutcome::ValidBlock(_) => false, + BlockProcessingOutcome::InvalidBlock(r) => match r { + InvalidBlock::FutureSlot => true, + InvalidBlock::StateRootMismatch => true, + InvalidBlock::ParentUnknown => false, + InvalidBlock::SlotProcessingError(_) => false, + InvalidBlock::PerBlockProcessingError(e) => match e { + BlockProcessingError::Invalid(_) => true, + BlockProcessingError::BeaconStateError(_) => false, + }, + }, + } + } +} + pub struct BeaconChain { pub block_store: Arc>, pub state_store: Arc>, @@ -685,10 +705,10 @@ where // TODO: check the block proposer signature BEFORE doing a state transition. This will // significantly lower exposure surface to DoS attacks. - // Transition the parent state to the present slot. + // Transition the parent state to the block slot. let mut state = parent_state; let previous_block_header = parent_block.block_header(); - for _ in state.slot.as_u64()..present_slot.as_u64() { + for _ in state.slot.as_u64()..block.slot.as_u64() { if let Err(e) = per_slot_processing(&mut state, &previous_block_header, &self.spec) { return Ok(BlockProcessingOutcome::InvalidBlock( InvalidBlock::SlotProcessingError(e), diff --git a/beacon_node/network/src/beacon_chain.rs b/beacon_node/network/src/beacon_chain.rs index e2829cfa6..bb4e8e71e 100644 --- a/beacon_node/network/src/beacon_chain.rs +++ b/beacon_node/network/src/beacon_chain.rs @@ -10,7 +10,7 @@ use beacon_chain::{ use eth2_libp2p::HelloMessage; use types::{BeaconBlock, BeaconStateError, Epoch, Hash256, Slot}; -pub use beacon_chain::BeaconChainError; +pub use beacon_chain::{BeaconChainError, BlockProcessingOutcome}; /// The network's API to the beacon chain. pub trait BeaconChain: Send + Sync { @@ -34,6 +34,9 @@ pub trait BeaconChain: Send + Sync { fn hello_message(&self) -> HelloMessage; + fn process_block(&self, block: BeaconBlock) + -> Result; + fn get_block_roots( &self, start_slot: Slot, @@ -98,6 +101,13 @@ where } } + fn process_block( + &self, + block: BeaconBlock, + ) -> Result { + self.process_block(block) + } + fn get_block_roots( &self, start_slot: Slot, diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 1a790eee1..2a84616e5 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -161,6 +161,17 @@ impl MessageHandler { &mut self.network_context, ) } + RPCResponse::BeaconBlockBodies(response) => { + debug!( + self.log, + "BeaconBlockBodies response received"; "peer" => format!("{:?}", peer_id) + ); + self.sync.on_beacon_block_bodies_response( + peer_id, + response, + &mut self.network_context, + ) + } // TODO: Handle all responses _ => panic!("Unknown response: {:?}", response), } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index ee0646dbb..b190f787f 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -3,12 +3,12 @@ use crate::message_handler::NetworkContext; use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{RPCRequest, RPCResponse}; use eth2_libp2p::PeerId; -use slog::{debug, error, o, warn}; +use slog::{debug, error, info, o, warn}; use ssz::TreeHash; use std::collections::HashMap; use std::sync::Arc; -use std::time::Instant; -use types::{BeaconBlockHeader, Epoch, Hash256, Slot}; +use std::time::{Duration, Instant}; +use types::{BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; /// The number of slots that we can import blocks ahead of us, before going into full Sync mode. const SLOT_IMPORT_TOLERANCE: u64 = 100; @@ -102,7 +102,11 @@ pub struct SimpleSync { impl SimpleSync { pub fn new(beacon_chain: Arc, log: &slog::Logger) -> Self { let sync_logger = log.new(o!("Service"=> "Sync")); - let import_queue = ImportQueue::new(beacon_chain.clone(), log.clone()); + + let queue_item_stale_time = Duration::from_secs(600); + + let import_queue = + ImportQueue::new(beacon_chain.clone(), queue_item_stale_time, log.clone()); SimpleSync { chain: beacon_chain.clone(), known_peers: HashMap::new(), @@ -229,13 +233,72 @@ impl SimpleSync { return; } - let block_roots = self.import_queue.enqueue_headers(response.headers); + let block_roots = self + .import_queue + .enqueue_headers(response.headers, peer_id.clone()); if !block_roots.is_empty() { self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network); } } + pub fn on_beacon_block_bodies_response( + &mut self, + peer_id: PeerId, + response: BeaconBlockBodiesResponse, + network: &mut NetworkContext, + ) { + self.import_queue + .enqueue_bodies(response.block_bodies, peer_id.clone()); + self.process_import_queue(network); + } + + pub fn process_import_queue(&mut self, network: &mut NetworkContext) { + let mut blocks: Vec<(Hash256, BeaconBlock, PeerId)> = self + .import_queue + .partials + .iter() + .filter_map(|(key, partial)| { + if let Some(_) = partial.body { + let (block, _root) = partial.clone().complete().expect("Body must be Some"); + Some((*key, block, partial.sender.clone())) + } else { + None + } + }) + .collect(); + + // Sort the blocks to be in ascending slot order. + blocks.sort_unstable_by(|a, b| a.1.slot.partial_cmp(&b.1.slot).unwrap()); + + let mut imported_keys = vec![]; + + for (key, block, sender) in blocks { + match self.chain.process_block(block) { + Ok(outcome) => { + if outcome.is_invalid() { + warn!(self.log, "Invalid block: {:?}", outcome); + network.disconnect(sender); + } else { + imported_keys.push(key) + } + } + Err(e) => { + error!(self.log, "Error during block processing"; "error" => format!("{:?}", e)) + } + } + } + + println!("imported_keys.len: {:?}", imported_keys.len()); + + if !imported_keys.is_empty() { + info!(self.log, "Imported {} blocks", imported_keys.len()); + for key in imported_keys { + self.import_queue.partials.remove(&key); + } + } + } + fn request_block_roots( &mut self, peer_id: PeerId, @@ -298,19 +361,41 @@ pub struct ImportQueue { pub chain: Arc, /// Partially imported blocks, keyed by the root of `BeaconBlockBody`. pub partials: HashMap, + /// Time before a queue entry is consider state. + pub stale_time: Duration, /// Logging log: slog::Logger, } impl ImportQueue { - pub fn new(chain: Arc, log: slog::Logger) -> Self { + pub fn new(chain: Arc, stale_time: Duration, log: slog::Logger) -> Self { Self { chain, partials: HashMap::new(), + stale_time, log, } } + pub fn remove_stale(&mut self) { + let keys: Vec = self + .partials + .iter() + .filter_map(|(key, partial)| { + if partial.inserted + self.stale_time >= Instant::now() { + Some(*key) + } else { + None + } + }) + .collect(); + + keys.iter().for_each(|key| { + self.partials.remove(&key); + }); + } + + /// Returns `true` if `self.chain` has not yet processed this block. fn is_new_block(&self, block_root: &Hash256) -> bool { self.chain .is_new_block_root(&block_root) @@ -322,9 +407,6 @@ impl ImportQueue { /// Returns the index of the first new root in the list of block roots. pub fn first_new_root(&mut self, roots: &[BlockRootSlot]) -> Option { - for root in roots { - println!("root {}", root.block_root); - } roots .iter() .position(|brs| self.is_new_block(&brs.block_root)) @@ -339,14 +421,18 @@ impl ImportQueue { /// If a `header` is already in the queue, but not yet processed by the chain the block root is /// included in the output and the `inserted` time for the partial record is set to /// `Instant::now()`. Updating the `inserted` time stops the partial from becoming stale. - pub fn enqueue_headers(&mut self, headers: Vec) -> Vec { + pub fn enqueue_headers( + &mut self, + headers: Vec, + sender: PeerId, + ) -> Vec { let mut required_bodies: Vec = vec![]; for header in headers { let block_root = Hash256::from_slice(&header.hash_tree_root()[..]); if self.is_new_block(&block_root) { - self.insert_partial(block_root, header); + self.insert_header(block_root, header, sender.clone()); required_bodies.push(block_root) } } @@ -354,20 +440,60 @@ impl ImportQueue { required_bodies } - fn insert_partial(&mut self, block_root: Hash256, header: BeaconBlockHeader) { - self.partials.insert( - header.block_body_root, - PartialBeaconBlock { + /// If there is a matching `header` for this `body`, adds it to the queue. + /// + /// If there is no `header` for the `body`, the body is simply discarded. + pub fn enqueue_bodies(&mut self, bodies: Vec, sender: PeerId) { + for body in bodies { + self.insert_body(body, sender.clone()); + } + } + + /// Inserts a header to the queue. + /// + /// If the header already exists, the `inserted` time is set to `now` and not other + /// modifications are made. + fn insert_header(&mut self, block_root: Hash256, header: BeaconBlockHeader, sender: PeerId) { + self.partials + .entry(header.block_body_root) + .and_modify(|p| p.inserted = Instant::now()) + .or_insert(PartialBeaconBlock { block_root, header, + body: None, inserted: Instant::now(), - }, - ); + sender, + }); + } + + /// Updates an existing partial with the `body`. + /// + /// If there is no header for the `body`, the body is simply discarded. + fn insert_body(&mut self, body: BeaconBlockBody, sender: PeerId) { + let body_root = Hash256::from_slice(&body.hash_tree_root()[..]); + + self.partials.entry(body_root).and_modify(|p| { + if body_root == p.header.block_body_root { + p.body = Some(body); + p.inserted = Instant::now(); + p.sender = sender; + } + }); } } +#[derive(Clone, Debug)] pub struct PartialBeaconBlock { pub block_root: Hash256, pub header: BeaconBlockHeader, + pub body: Option, pub inserted: Instant, + pub sender: PeerId, +} + +impl PartialBeaconBlock { + /// Given a `body`, consumes `self` and returns a complete `BeaconBlock` along with its root. + pub fn complete(self) -> Option<(BeaconBlock, Hash256)> { + Some((self.header.into_block(self.body?), self.block_root)) + } } diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index 076a3f529..57587717b 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -77,7 +77,16 @@ impl SyncNode { match request { RPCRequest::BeaconBlockHeaders(request) => request, - _ => panic!("Did not get block root request"), + _ => panic!("Did not get block headers request"), + } + } + + pub fn get_block_bodies_request(&self) -> BeaconBlockBodiesRequest { + let request = self.recv_rpc_request().expect("No block bodies request"); + + match request { + RPCRequest::BeaconBlockBodies(request) => request, + _ => panic!("Did not get block bodies request"), } } @@ -223,6 +232,29 @@ impl SyncMaster { self.send_rpc_response(node, response) } + pub fn respond_to_block_bodies_request( + &mut self, + node: &SyncNode, + request: BeaconBlockBodiesRequest, + ) { + let block_bodies: Vec = request + .block_roots + .iter() + .map(|root| { + let block = self + .harness + .beacon_chain + .get_block(root) + .expect("Failed to load block") + .expect("Block did not exist"); + block.body + }) + .collect(); + + let response = RPCResponse::BeaconBlockBodies(BeaconBlockBodiesResponse { block_bodies }); + self.send_rpc_response(node, response) + } + fn send_rpc_response(&mut self, node: &SyncNode, rpc_response: RPCResponse) { node.send(self.rpc_response(node, rpc_response)); } @@ -311,6 +343,11 @@ fn first_test() { master.respond_to_block_headers_request(&nodes[0], headers_request); - std::thread::sleep(Duration::from_millis(500)); + let bodies_request = nodes[0].get_block_bodies_request(); + assert_eq!(bodies_request.block_roots.len(), 2); + + master.respond_to_block_bodies_request(&nodes[0], bodies_request); + + std::thread::sleep(Duration::from_millis(10000)); runtime.shutdown_now(); } diff --git a/eth2/types/src/beacon_block_header.rs b/eth2/types/src/beacon_block_header.rs index 3d8b08cc8..f4bee27e1 100644 --- a/eth2/types/src/beacon_block_header.rs +++ b/eth2/types/src/beacon_block_header.rs @@ -37,6 +37,19 @@ impl BeaconBlockHeader { pub fn canonical_root(&self) -> Hash256 { Hash256::from_slice(&self.hash_tree_root()[..]) } + + /// Given a `body`, consumes `self` and returns a complete `BeaconBlock`. + /// + /// Spec v0.5.0 + pub fn into_block(self, body: BeaconBlockBody) -> BeaconBlock { + BeaconBlock { + slot: self.slot, + previous_block_root: self.previous_block_root, + state_root: self.state_root, + body, + signature: self.signature, + } + } } #[cfg(test)] From 3eae7b3fc5258fa611ecab23aa3dc8d72f4e8fb9 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 12:49:59 +1100 Subject: [PATCH 09/24] Add skip slot support to BeaconChain fns --- beacon_node/beacon_chain/src/beacon_chain.rs | 87 ++++++++++++++------ beacon_node/network/src/beacon_chain.rs | 41 +++++++-- 2 files changed, 97 insertions(+), 31 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index eb8df6f2a..d5fd113a8 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -142,7 +142,38 @@ where }) } - /// Returns beacon block roots for `count` slots, starting from `start_slot`. + /// Returns the beacon block body for each beacon block root in `roots`. + /// + /// Fails if any root in `roots` does not have a corresponding block. + pub fn get_block_bodies(&self, roots: &[Hash256]) -> Result, Error> { + let bodies: Result, _> = roots + .iter() + .map(|root| match self.get_block(root)? { + Some(block) => Ok(block.body), + None => Err(Error::DBInconsistent("Missing block".into())), + }) + .collect(); + + Ok(bodies?) + } + + /// Returns the beacon block header for each beacon block root in `roots`. + /// + /// Fails if any root in `roots` does not have a corresponding block. + pub fn get_block_headers(&self, roots: &[Hash256]) -> Result, Error> { + let headers: Result, _> = roots + .iter() + .map(|root| match self.get_block(root)? { + Some(block) => Ok(block.block_header()), + None => Err(Error::DBInconsistent("Missing block".into())), + }) + .collect(); + + Ok(headers?) + } + + /// Returns `count `beacon block roots, starting from `start_slot` with an + /// interval of `skip` slots between each root. /// /// ## Errors: /// @@ -152,40 +183,45 @@ where /// - Other: BeaconState` is inconsistent. pub fn get_block_roots( &self, - start_slot: Slot, - count: Slot, - ) -> Result, BeaconStateError> { + earliest_slot: Slot, + count: usize, + skip: usize, + ) -> Result, Error> { let spec = &self.spec; + let step_by = Slot::from(skip + 1); let mut roots: Vec = vec![]; + + // The state for reading block roots. Will be updated with an older state if slots go too + // far back in history. let mut state = self.state.read().clone(); - let mut slot = start_slot + count - 1; + + // The final slot in this series, will be reduced by `skip` each loop iteration. + let mut slot = earliest_slot + Slot::from(count * (skip + 1)) - 1; + + // If the highest slot requested is that of the current state insert the root of the + // head block, unless the head block's slot is not matching. + if slot == state.slot && self.head().beacon_block.slot == slot { + roots.push(self.head().beacon_block_root); + + slot -= step_by; + } else if slot >= state.slot { + return Err(BeaconStateError::SlotOutOfBounds.into()); + } loop { - // If the highest slot requested is that of the current state insert the root of the - // head block, unless the head block's slot is not matching. - if slot == state.slot && self.head().beacon_block.slot == slot { - roots.push(self.head().beacon_block_root); - - slot -= 1; - continue; - } else if slot >= state.slot { - return Err(BeaconStateError::SlotOutOfBounds); - } - // If the slot is within the range of the current state's block roots, append the root // to the output vec. // - // If we get `SlotOutOfBounds` error, load the oldest known state to the present state - // from the DB. + // If we get `SlotOutOfBounds` error, load the oldest available historic + // state from the DB. match state.get_block_root(slot, spec) { Ok(root) => { - roots.push(*root); - - if slot == start_slot { + if slot < earliest_slot { break; } else { - slot -= 1; + roots.push(*root); + slot -= step_by; } } Err(BeaconStateError::SlotOutOfBounds) => { @@ -201,18 +237,19 @@ where _ => break, } } - Err(e) => return Err(e), + Err(e) => return Err(e.into()), }; } - if (slot == start_slot) && (roots.len() == count.as_usize()) { + // Return the results if they pass a sanity check. + if (slot <= earliest_slot) && (roots.len() == count) { // Reverse the ordering of the roots. We extracted them in reverse order to make it // simpler to lookup historic states. // // This is a potential optimisation target. Ok(roots.iter().rev().cloned().collect()) } else { - Err(BeaconStateError::SlotOutOfBounds) + Err(BeaconStateError::SlotOutOfBounds.into()) } } diff --git a/beacon_node/network/src/beacon_chain.rs b/beacon_node/network/src/beacon_chain.rs index bb4e8e71e..cc54e8ae0 100644 --- a/beacon_node/network/src/beacon_chain.rs +++ b/beacon_node/network/src/beacon_chain.rs @@ -8,7 +8,7 @@ use beacon_chain::{ CheckPoint, }; use eth2_libp2p::HelloMessage; -use types::{BeaconBlock, BeaconStateError, Epoch, Hash256, Slot}; +use types::{BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; pub use beacon_chain::{BeaconChainError, BlockProcessingOutcome}; @@ -40,8 +40,19 @@ pub trait BeaconChain: Send + Sync { fn get_block_roots( &self, start_slot: Slot, - count: Slot, - ) -> Result, BeaconStateError>; + count: usize, + skip: usize, + ) -> Result, BeaconChainError>; + + fn get_block_headers( + &self, + start_slot: Slot, + count: usize, + skip: usize, + ) -> Result, BeaconChainError>; + + fn get_block_bodies(&self, roots: &[Hash256]) + -> Result, BeaconChainError>; fn is_new_block_root(&self, beacon_block_root: &Hash256) -> Result; } @@ -111,9 +122,27 @@ where fn get_block_roots( &self, start_slot: Slot, - count: Slot, - ) -> Result, BeaconStateError> { - self.get_block_roots(start_slot, count) + count: usize, + skip: usize, + ) -> Result, BeaconChainError> { + self.get_block_roots(start_slot, count, skip) + } + + fn get_block_headers( + &self, + start_slot: Slot, + count: usize, + skip: usize, + ) -> Result, BeaconChainError> { + let roots = self.get_block_roots(start_slot, count, skip)?; + self.get_block_headers(&roots) + } + + fn get_block_bodies( + &self, + roots: &[Hash256], + ) -> Result, BeaconChainError> { + self.get_block_bodies(roots) } fn is_new_block_root(&self, beacon_block_root: &Hash256) -> Result { From 6b39c693af28675c488202ad8def2980fb3c772d Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 12:50:23 +1100 Subject: [PATCH 10/24] Extend syncing --- beacon_node/network/src/message_handler.rs | 49 ++--- beacon_node/network/src/sync/simple_sync.rs | 158 ++++++++++++++-- beacon_node/network/tests/tests.rs | 196 +++++++++++++++++++- 3 files changed, 364 insertions(+), 39 deletions(-) diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 2a84616e5..58ba0171d 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -113,10 +113,24 @@ impl MessageHandler { match request { RPCRequest::Hello(hello_message) => { self.sync - .on_hello(peer_id, hello_message, &mut self.network_context) + .on_hello_request(peer_id, hello_message, &mut self.network_context) } + RPCRequest::BeaconBlockRoots(request) => { + self.sync + .on_beacon_block_roots_request(peer_id, request, &mut self.network_context) + } + RPCRequest::BeaconBlockHeaders(request) => self.sync.on_beacon_block_headers_request( + peer_id, + request, + &mut self.network_context, + ), + RPCRequest::BeaconBlockBodies(request) => self.sync.on_beacon_block_bodies_request( + peer_id, + request, + &mut self.network_context, + ), // TODO: Handle all requests - _ => {} + _ => panic!("Unknown request: {:?}", request), } } @@ -133,48 +147,41 @@ impl MessageHandler { debug!(self.log, "Unrecognized response from peer: {:?}", peer_id); return; } - match response { + let response_str = match response { RPCResponse::Hello(hello_message) => { - debug!(self.log, "Hello response received from peer: {:?}", peer_id); self.sync - .on_hello(peer_id, hello_message, &mut self.network_context); + .on_hello_response(peer_id, hello_message, &mut self.network_context); + "Hello" } RPCResponse::BeaconBlockRoots(response) => { - debug!( - self.log, - "BeaconBlockRoots response received"; "peer" => format!("{:?}", peer_id) - ); self.sync.on_beacon_block_roots_response( peer_id, response, &mut self.network_context, - ) + ); + "BeaconBlockRoots" } RPCResponse::BeaconBlockHeaders(response) => { - debug!( - self.log, - "BeaconBlockHeaders response received"; "peer" => format!("{:?}", peer_id) - ); self.sync.on_beacon_block_headers_response( peer_id, response, &mut self.network_context, - ) + ); + "BeaconBlockHeaders" } RPCResponse::BeaconBlockBodies(response) => { - debug!( - self.log, - "BeaconBlockBodies response received"; "peer" => format!("{:?}", peer_id) - ); self.sync.on_beacon_block_bodies_response( peer_id, response, &mut self.network_context, - ) + ); + "BeaconBlockBodies" } // TODO: Handle all responses _ => panic!("Unknown response: {:?}", response), - } + }; + + debug!(self.log, "RPCResponse"; "type" => response_str); } } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index b190f787f..4726419d5 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -126,14 +126,31 @@ impl SimpleSync { hello: HelloMessage, network: &mut NetworkContext, ) { + // Say hello back. network.send_rpc_response( peer_id.clone(), RPCResponse::Hello(self.chain.hello_message()), ); - self.on_hello(peer_id, hello, network); + + self.process_hello(peer_id, hello, network); } - pub fn on_hello(&mut self, peer_id: PeerId, hello: HelloMessage, network: &mut NetworkContext) { + pub fn on_hello_response( + &mut self, + peer_id: PeerId, + hello: HelloMessage, + network: &mut NetworkContext, + ) { + // Process the hello message, without sending back another hello. + self.process_hello(peer_id, hello, network); + } + + fn process_hello( + &mut self, + peer_id: PeerId, + hello: HelloMessage, + network: &mut NetworkContext, + ) { let spec = self.chain.get_spec(); let remote = PeerSyncInfo::from(hello); @@ -142,7 +159,7 @@ impl SimpleSync { // network id must match if remote_status != PeerStatus::OnDifferentChain { - debug!(self.log, "Handshake successful. Peer: {:?}", peer_id); + info!(self.log, "HandshakeSuccess"; "peer" => format!("{:?}", peer_id)); self.known_peers.insert(peer_id.clone(), remote); } @@ -183,6 +200,44 @@ impl SimpleSync { } } + pub fn on_beacon_block_roots_request( + &mut self, + peer_id: PeerId, + request: BeaconBlockRootsRequest, + network: &mut NetworkContext, + ) { + let roots = match self + .chain + .get_block_roots(request.start_slot, request.count as usize, 0) + { + Ok(roots) => roots, + Err(e) => { + // TODO: return RPC error. + warn!( + self.log, + "RPCRequest"; "peer" => format!("{:?}", peer_id), + "request" => "BeaconBlockRoots", + "error" => format!("{:?}", e) + ); + return; + } + }; + + let roots = roots + .iter() + .enumerate() + .map(|(i, &block_root)| BlockRootSlot { + slot: request.start_slot + Slot::from(i), + block_root, + }) + .collect(); + + network.send_rpc_response( + peer_id, + RPCResponse::BeaconBlockRoots(BeaconBlockRootsResponse { roots }), + ) + } + pub fn on_beacon_block_roots_response( &mut self, peer_id: PeerId, @@ -219,6 +274,36 @@ impl SimpleSync { } } + pub fn on_beacon_block_headers_request( + &mut self, + peer_id: PeerId, + request: BeaconBlockHeadersRequest, + network: &mut NetworkContext, + ) { + let headers = match self.chain.get_block_headers( + request.start_slot, + request.max_headers as usize, + request.skip_slots as usize, + ) { + Ok(headers) => headers, + Err(e) => { + // TODO: return RPC error. + warn!( + self.log, + "RPCRequest"; "peer" => format!("{:?}", peer_id), + "request" => "BeaconBlockHeaders", + "error" => format!("{:?}", e) + ); + return; + } + }; + + network.send_rpc_response( + peer_id, + RPCResponse::BeaconBlockHeaders(BeaconBlockHeadersResponse { headers }), + ) + } + pub fn on_beacon_block_headers_response( &mut self, peer_id: PeerId, @@ -237,9 +322,33 @@ impl SimpleSync { .import_queue .enqueue_headers(response.headers, peer_id.clone()); - if !block_roots.is_empty() { - self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network); - } + self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network); + } + + pub fn on_beacon_block_bodies_request( + &mut self, + peer_id: PeerId, + request: BeaconBlockBodiesRequest, + network: &mut NetworkContext, + ) { + let block_bodies = match self.chain.get_block_bodies(&request.block_roots) { + Ok(bodies) => bodies, + Err(e) => { + // TODO: return RPC error. + warn!( + self.log, + "RPCRequest"; "peer" => format!("{:?}", peer_id), + "request" => "BeaconBlockBodies", + "error" => format!("{:?}", e) + ); + return; + } + }; + + network.send_rpc_response( + peer_id, + RPCResponse::BeaconBlockBodies(BeaconBlockBodiesResponse { block_bodies }), + ) } pub fn on_beacon_block_bodies_response( @@ -250,6 +359,11 @@ impl SimpleSync { ) { self.import_queue .enqueue_bodies(response.block_bodies, peer_id.clone()); + + // Clear out old entries + self.import_queue.remove_stale(); + + // Import blocks, if possible. self.process_import_queue(network); } @@ -268,10 +382,14 @@ impl SimpleSync { }) .collect(); + if !blocks.is_empty() { + info!(self.log, "Processing blocks"; "count" => blocks.len()); + } + // Sort the blocks to be in ascending slot order. blocks.sort_unstable_by(|a, b| a.1.slot.partial_cmp(&b.1.slot).unwrap()); - let mut imported_keys = vec![]; + let mut keys_to_delete = vec![]; for (key, block, sender) in blocks { match self.chain.process_block(block) { @@ -279,8 +397,10 @@ impl SimpleSync { if outcome.is_invalid() { warn!(self.log, "Invalid block: {:?}", outcome); network.disconnect(sender); + keys_to_delete.push(key) } else { - imported_keys.push(key) + // TODO: don't delete if was not invalid but not successfully processed. + keys_to_delete.push(key) } } Err(e) => { @@ -289,11 +409,9 @@ impl SimpleSync { } } - println!("imported_keys.len: {:?}", imported_keys.len()); - - if !imported_keys.is_empty() { - info!(self.log, "Imported {} blocks", imported_keys.len()); - for key in imported_keys { + if !keys_to_delete.is_empty() { + info!(self.log, "Processed {} blocks", keys_to_delete.len()); + for key in keys_to_delete { self.import_queue.partials.remove(&key); } } @@ -313,7 +431,10 @@ impl SimpleSync { debug!( self.log, - "Requesting {} block roots from {:?}.", request.count, &peer_id + "RPCRequest"; + "type" => "BeaconBlockRoots", + "count" => request.count, + "peer" => format!("{:?}", peer_id) ); // TODO: handle count > max count. @@ -328,7 +449,10 @@ impl SimpleSync { ) { debug!( self.log, - "Requesting {} headers from {:?}.", request.max_headers, &peer_id + "RPCRequest"; + "type" => "BeaconBlockHeaders", + "max_headers" => request.max_headers, + "peer" => format!("{:?}", peer_id) ); network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockHeaders(request)); @@ -377,6 +501,10 @@ impl ImportQueue { } } + /// Flushes all stale entries from the queue. + /// + /// An entry is stale if it has as a `inserted` time that is more than `self.stale_time` in the + /// past. pub fn remove_stale(&mut self) { let keys: Vec = self .partials diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index 57587717b..b951d7d2a 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -17,6 +17,7 @@ pub struct SyncNode { pub id: usize, sender: Sender, receiver: Receiver, + peer_id: PeerId, harness: BeaconChainHarness, } @@ -43,6 +44,7 @@ impl SyncNode { id, sender: message_handler_sender, receiver: network_receiver, + peer_id: PeerId::random(), harness, } } @@ -63,6 +65,138 @@ impl SyncNode { self.harness.beacon_chain.hello_message() } + pub fn connect_to(&mut self, node: &SyncNode) { + let message = HandlerMessage::PeerDialed(self.peer_id.clone()); + node.send(message); + } + + /// Reads the receive queue from one node and passes the message to the other. Also returns a + /// copy of the message. + /// + /// self -----> node + /// | + /// us + /// + /// Named after the unix `tee` command. + fn tee(&mut self, node: &SyncNode) -> NetworkMessage { + let network_message = self.recv().expect("Timeout on tee"); + + let handler_message = match network_message.clone() { + NetworkMessage::Send(peer_id, OutgoingMessage::RPC(event)) => { + HandlerMessage::RPC(peer_id, event) + } + _ => panic!("tee cannot parse {:?}", network_message), + }; + + node.send(handler_message); + + network_message + } + + fn tee_hello_request(&mut self, node: &SyncNode) -> HelloMessage { + let request = self.tee_rpc_request(node); + + match request { + RPCRequest::Hello(message) => message, + _ => panic!("tee_hello_request got: {:?}", request), + } + } + + fn tee_hello_response(&mut self, node: &SyncNode) -> HelloMessage { + let response = self.tee_rpc_response(node); + + match response { + RPCResponse::Hello(message) => message, + _ => panic!("tee_hello_response got: {:?}", response), + } + } + + fn tee_block_root_request(&mut self, node: &SyncNode) -> BeaconBlockRootsRequest { + let msg = self.tee_rpc_request(node); + + match msg { + RPCRequest::BeaconBlockRoots(data) => data, + _ => panic!("tee_block_root_request got: {:?}", msg), + } + } + + fn tee_block_root_response(&mut self, node: &SyncNode) -> BeaconBlockRootsResponse { + let msg = self.tee_rpc_response(node); + + match msg { + RPCResponse::BeaconBlockRoots(data) => data, + _ => panic!("tee_block_root_response got: {:?}", msg), + } + } + + fn tee_block_header_request(&mut self, node: &SyncNode) -> BeaconBlockHeadersRequest { + let msg = self.tee_rpc_request(node); + + match msg { + RPCRequest::BeaconBlockHeaders(data) => data, + _ => panic!("tee_block_header_request got: {:?}", msg), + } + } + + fn tee_block_header_response(&mut self, node: &SyncNode) -> BeaconBlockHeadersResponse { + let msg = self.tee_rpc_response(node); + + match msg { + RPCResponse::BeaconBlockHeaders(data) => data, + _ => panic!("tee_block_header_response got: {:?}", msg), + } + } + + fn tee_block_body_request(&mut self, node: &SyncNode) -> BeaconBlockBodiesRequest { + let msg = self.tee_rpc_request(node); + + match msg { + RPCRequest::BeaconBlockBodies(data) => data, + _ => panic!("tee_block_body_request got: {:?}", msg), + } + } + + fn tee_block_body_response(&mut self, node: &SyncNode) -> BeaconBlockBodiesResponse { + let msg = self.tee_rpc_response(node); + + match msg { + RPCResponse::BeaconBlockBodies(data) => data, + _ => panic!("tee_block_body_response got: {:?}", msg), + } + } + + fn tee_rpc_request(&mut self, node: &SyncNode) -> RPCRequest { + let network_message = self.tee(node); + + match network_message { + NetworkMessage::Send( + _peer_id, + OutgoingMessage::RPC(RPCEvent::Request { + id: _, + method_id: _, + body, + }), + ) => body, + _ => panic!("tee_rpc_request failed! got {:?}", network_message), + } + } + + fn tee_rpc_response(&mut self, node: &SyncNode) -> RPCResponse { + let network_message = self.tee(node); + + match network_message { + NetworkMessage::Send( + _peer_id, + OutgoingMessage::RPC(RPCEvent::Response { + id: _, + method_id: _, + result, + }), + ) => result, + _ => panic!("tee_rpc_response failed! got {:?}", network_message), + } + } + pub fn get_block_root_request(&self) -> BeaconBlockRootsRequest { let request = self.recv_rpc_request().expect("No block root request"); @@ -181,7 +315,7 @@ impl SyncMaster { let roots = self .harness .beacon_chain - .get_block_roots(request.start_slot, Slot::from(request.count)) + .get_block_roots(request.start_slot, request.count as usize, 0) .expect("Beacon chain did not give block roots") .iter() .enumerate() @@ -203,7 +337,11 @@ impl SyncMaster { let roots = self .harness .beacon_chain - .get_block_roots(request.start_slot, Slot::from(request.max_headers)) + .get_block_roots( + request.start_slot, + request.max_headers as usize, + request.skip_slots as usize, + ) .expect("Beacon chain did not give blocks"); if roots.is_empty() { @@ -312,7 +450,7 @@ pub fn build_blocks(blocks: usize, master: &mut SyncMaster, nodes: &mut Vec Date: Sun, 24 Mar 2019 13:06:17 +1100 Subject: [PATCH 11/24] Tidy logging, fix bug with stale time in queue --- beacon_node/network/src/message_handler.rs | 2 +- beacon_node/network/src/sync/simple_sync.rs | 34 +++++++++++++++------ 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 58ba0171d..24fe04950 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -181,7 +181,7 @@ impl MessageHandler { _ => panic!("Unknown response: {:?}", response), }; - debug!(self.log, "RPCResponse"; "type" => response_str); + debug!(self.log, "RPCResponse({})", response_str); } } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 4726419d5..ff6092be1 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -13,6 +13,9 @@ use types::{BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slo /// The number of slots that we can import blocks ahead of us, before going into full Sync mode. const SLOT_IMPORT_TOLERANCE: u64 = 100; +/// The amount of seconds a block (or partial block) may exist in the import queue. +const QUEUE_STALE_SECS: u64 = 60; + /// Keeps track of syncing information for known connected peers. #[derive(Clone, Copy, Debug)] pub struct PeerSyncInfo { @@ -103,7 +106,7 @@ impl SimpleSync { pub fn new(beacon_chain: Arc, log: &slog::Logger) -> Self { let sync_logger = log.new(o!("Service"=> "Sync")); - let queue_item_stale_time = Duration::from_secs(600); + let queue_item_stale_time = Duration::from_secs(QUEUE_STALE_SECS); let import_queue = ImportQueue::new(beacon_chain.clone(), queue_item_stale_time, log.clone()); @@ -165,7 +168,11 @@ impl SimpleSync { match remote_status { PeerStatus::OnDifferentChain => { - debug!(self.log, "Peer is on different chain. Peer: {:?}", peer_id); + info!( + self.log, "Failure"; + "peer" => format!("{:?}", peer_id), + "reason" => "network_id" + ); network.disconnect(peer_id); } @@ -431,8 +438,7 @@ impl SimpleSync { debug!( self.log, - "RPCRequest"; - "type" => "BeaconBlockRoots", + "RPCRequest(BeaconBlockRoots)"; "count" => request.count, "peer" => format!("{:?}", peer_id) ); @@ -449,8 +455,7 @@ impl SimpleSync { ) { debug!( self.log, - "RPCRequest"; - "type" => "BeaconBlockHeaders", + "RPCRequest(BeaconBlockHeaders)"; "max_headers" => request.max_headers, "peer" => format!("{:?}", peer_id) ); @@ -466,9 +471,9 @@ impl SimpleSync { ) { debug!( self.log, - "Requesting {} bodies from {:?}.", - request.block_roots.len(), - &peer_id + "RPCRequest(BeaconBlockBodies)"; + "count" => request.block_roots.len(), + "peer" => format!("{:?}", peer_id) ); network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockBodies(request)); @@ -510,7 +515,7 @@ impl ImportQueue { .partials .iter() .filter_map(|(key, partial)| { - if partial.inserted + self.stale_time >= Instant::now() { + if partial.inserted + self.stale_time <= Instant::now() { Some(*key) } else { None @@ -518,6 +523,15 @@ impl ImportQueue { }) .collect(); + if !keys.is_empty() { + debug!( + self.log, + "ImportQueue removing stale entries"; + "stale_count" => keys.len(), + "stale_time_seconds" => self.stale_time.as_secs() + ); + } + keys.iter().for_each(|key| { self.partials.remove(&key); }); From 15f853416bab576ef9c21e2813e52b8b6c7031e9 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 13:59:27 +1100 Subject: [PATCH 12/24] Add more detail to sync logs --- beacon_node/network/src/message_handler.rs | 13 +-- beacon_node/network/src/sync/simple_sync.rs | 114 ++++++++++++++------ 2 files changed, 85 insertions(+), 42 deletions(-) diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 24fe04950..77a2ab8db 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -8,8 +8,7 @@ use eth2_libp2p::{ PeerId, RPCEvent, }; use futures::future; -use slog::debug; -use slog::warn; +use slog::{debug, warn}; use std::collections::HashMap; use std::sync::Arc; use std::time::Instant; @@ -144,14 +143,13 @@ impl MessageHandler { .remove(&(peer_id.clone(), id)) .is_none() { - debug!(self.log, "Unrecognized response from peer: {:?}", peer_id); + debug!(self.log, "Unrecognised response from peer: {:?}", peer_id); return; } - let response_str = match response { + match response { RPCResponse::Hello(hello_message) => { self.sync .on_hello_response(peer_id, hello_message, &mut self.network_context); - "Hello" } RPCResponse::BeaconBlockRoots(response) => { self.sync.on_beacon_block_roots_response( @@ -159,7 +157,6 @@ impl MessageHandler { response, &mut self.network_context, ); - "BeaconBlockRoots" } RPCResponse::BeaconBlockHeaders(response) => { self.sync.on_beacon_block_headers_response( @@ -167,7 +164,6 @@ impl MessageHandler { response, &mut self.network_context, ); - "BeaconBlockHeaders" } RPCResponse::BeaconBlockBodies(response) => { self.sync.on_beacon_block_bodies_response( @@ -175,13 +171,10 @@ impl MessageHandler { response, &mut self.network_context, ); - "BeaconBlockBodies" } // TODO: Handle all responses _ => panic!("Unknown response: {:?}", response), }; - - debug!(self.log, "RPCResponse({})", response_str); } } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index ff6092be1..7609f5750 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -120,6 +120,8 @@ impl SimpleSync { } pub fn on_connect(&self, peer_id: PeerId, network: &mut NetworkContext) { + info!(self.log, "PeerConnect"; "peer" => format!("{:?}", peer_id)); + network.send_rpc_request(peer_id, RPCRequest::Hello(self.chain.hello_message())); } @@ -129,6 +131,8 @@ impl SimpleSync { hello: HelloMessage, network: &mut NetworkContext, ) { + debug!(self.log, "HelloRequest"; "peer" => format!("{:?}", peer_id)); + // Say hello back. network.send_rpc_response( peer_id.clone(), @@ -144,6 +148,8 @@ impl SimpleSync { hello: HelloMessage, network: &mut NetworkContext, ) { + debug!(self.log, "HelloResponse"; "peer" => format!("{:?}", peer_id)); + // Process the hello message, without sending back another hello. self.process_hello(peer_id, hello, network); } @@ -210,12 +216,19 @@ impl SimpleSync { pub fn on_beacon_block_roots_request( &mut self, peer_id: PeerId, - request: BeaconBlockRootsRequest, + req: BeaconBlockRootsRequest, network: &mut NetworkContext, ) { + debug!( + self.log, + "BlockRootsRequest"; + "peer" => format!("{:?}", peer_id), + "count" => req.count, + ); + let roots = match self .chain - .get_block_roots(request.start_slot, request.count as usize, 0) + .get_block_roots(req.start_slot, req.count as usize, 0) { Ok(roots) => roots, Err(e) => { @@ -223,7 +236,7 @@ impl SimpleSync { warn!( self.log, "RPCRequest"; "peer" => format!("{:?}", peer_id), - "request" => "BeaconBlockRoots", + "req" => "BeaconBlockRoots", "error" => format!("{:?}", e) ); return; @@ -234,7 +247,7 @@ impl SimpleSync { .iter() .enumerate() .map(|(i, &block_root)| BlockRootSlot { - slot: request.start_slot + Slot::from(i), + slot: req.start_slot + Slot::from(i), block_root, }) .collect(); @@ -248,10 +261,17 @@ impl SimpleSync { pub fn on_beacon_block_roots_response( &mut self, peer_id: PeerId, - response: BeaconBlockRootsResponse, + res: BeaconBlockRootsResponse, network: &mut NetworkContext, ) { - if response.roots.is_empty() { + debug!( + self.log, + "BlockRootsResponse"; + "peer" => format!("{:?}", peer_id), + "count" => res.roots.len(), + ); + + if res.roots.is_empty() { warn!( self.log, "Peer returned empty block roots response. PeerId: {:?}", peer_id @@ -259,21 +279,21 @@ impl SimpleSync { return; } - let new_root_index = self.import_queue.first_new_root(&response.roots); + let new_root_index = self.import_queue.first_new_root(&res.roots); // If a new block root is found, request it and all the headers following it. // // We make an assumption here that if we don't know a block then we don't know of all // it's parents. This might not be the case if syncing becomes more sophisticated. if let Some(i) = new_root_index { - let new = &response.roots[i]; + let new = &res.roots[i]; self.request_block_headers( peer_id, BeaconBlockHeadersRequest { start_root: new.block_root, start_slot: new.slot, - max_headers: (response.roots.len() - i) as u64, + max_headers: (res.roots.len() - i) as u64, skip_slots: 0, }, network, @@ -284,13 +304,20 @@ impl SimpleSync { pub fn on_beacon_block_headers_request( &mut self, peer_id: PeerId, - request: BeaconBlockHeadersRequest, + req: BeaconBlockHeadersRequest, network: &mut NetworkContext, ) { + debug!( + self.log, + "BlockHeadersRequest"; + "peer" => format!("{:?}", peer_id), + "count" => req.max_headers, + ); + let headers = match self.chain.get_block_headers( - request.start_slot, - request.max_headers as usize, - request.skip_slots as usize, + req.start_slot, + req.max_headers as usize, + req.skip_slots as usize, ) { Ok(headers) => headers, Err(e) => { @@ -298,7 +325,7 @@ impl SimpleSync { warn!( self.log, "RPCRequest"; "peer" => format!("{:?}", peer_id), - "request" => "BeaconBlockHeaders", + "req" => "BeaconBlockHeaders", "error" => format!("{:?}", e) ); return; @@ -314,10 +341,17 @@ impl SimpleSync { pub fn on_beacon_block_headers_response( &mut self, peer_id: PeerId, - response: BeaconBlockHeadersResponse, + res: BeaconBlockHeadersResponse, network: &mut NetworkContext, ) { - if response.headers.is_empty() { + debug!( + self.log, + "BlockHeadersResponse"; + "peer" => format!("{:?}", peer_id), + "count" => res.headers.len(), + ); + + if res.headers.is_empty() { warn!( self.log, "Peer returned empty block headers response. PeerId: {:?}", peer_id @@ -325,9 +359,11 @@ impl SimpleSync { return; } + // Enqueue the headers, obtaining a list of the roots of the headers which were newly added + // to the queue. let block_roots = self .import_queue - .enqueue_headers(response.headers, peer_id.clone()); + .enqueue_headers(res.headers, peer_id.clone()); self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network); } @@ -335,17 +371,24 @@ impl SimpleSync { pub fn on_beacon_block_bodies_request( &mut self, peer_id: PeerId, - request: BeaconBlockBodiesRequest, + req: BeaconBlockBodiesRequest, network: &mut NetworkContext, ) { - let block_bodies = match self.chain.get_block_bodies(&request.block_roots) { + debug!( + self.log, + "BlockBodiesRequest"; + "peer" => format!("{:?}", peer_id), + "count" => req.block_roots.len(), + ); + + let block_bodies = match self.chain.get_block_bodies(&req.block_roots) { Ok(bodies) => bodies, Err(e) => { // TODO: return RPC error. warn!( self.log, "RPCRequest"; "peer" => format!("{:?}", peer_id), - "request" => "BeaconBlockBodies", + "req" => "BeaconBlockBodies", "error" => format!("{:?}", e) ); return; @@ -361,11 +404,18 @@ impl SimpleSync { pub fn on_beacon_block_bodies_response( &mut self, peer_id: PeerId, - response: BeaconBlockBodiesResponse, + res: BeaconBlockBodiesResponse, network: &mut NetworkContext, ) { + debug!( + self.log, + "BlockBodiesResponse"; + "peer" => format!("{:?}", peer_id), + "count" => res.block_bodies.len(), + ); + self.import_queue - .enqueue_bodies(response.block_bodies, peer_id.clone()); + .enqueue_bodies(res.block_bodies, peer_id.clone()); // Clear out old entries self.import_queue.remove_stale(); @@ -427,11 +477,11 @@ impl SimpleSync { fn request_block_roots( &mut self, peer_id: PeerId, - request: BeaconBlockRootsRequest, + req: BeaconBlockRootsRequest, network: &mut NetworkContext, ) { // Potentially set state to sync. - if self.state == SyncState::Idle && request.count > SLOT_IMPORT_TOLERANCE { + if self.state == SyncState::Idle && req.count > SLOT_IMPORT_TOLERANCE { debug!(self.log, "Entering downloading sync state."); self.state = SyncState::Downloading; } @@ -439,44 +489,44 @@ impl SimpleSync { debug!( self.log, "RPCRequest(BeaconBlockRoots)"; - "count" => request.count, + "count" => req.count, "peer" => format!("{:?}", peer_id) ); // TODO: handle count > max count. - network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockRoots(request)); + network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockRoots(req)); } fn request_block_headers( &mut self, peer_id: PeerId, - request: BeaconBlockHeadersRequest, + req: BeaconBlockHeadersRequest, network: &mut NetworkContext, ) { debug!( self.log, "RPCRequest(BeaconBlockHeaders)"; - "max_headers" => request.max_headers, + "max_headers" => req.max_headers, "peer" => format!("{:?}", peer_id) ); - network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockHeaders(request)); + network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockHeaders(req)); } fn request_block_bodies( &mut self, peer_id: PeerId, - request: BeaconBlockBodiesRequest, + req: BeaconBlockBodiesRequest, network: &mut NetworkContext, ) { debug!( self.log, "RPCRequest(BeaconBlockBodies)"; - "count" => request.block_roots.len(), + "count" => req.block_roots.len(), "peer" => format!("{:?}", peer_id) ); - network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockBodies(request)); + network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockBodies(req)); } /// Generates our current state in the form of a HELLO RPC message. From 5f4f67f46f1f089b598f960e9e2b868e4e6c1a6d Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 14:22:12 +1100 Subject: [PATCH 13/24] Swap ImportQueue from a Map to a Vec There's an edge case where different blocks can have the same block body. --- beacon_node/beacon_chain/src/beacon_chain.rs | 9 ++++ beacon_node/network/src/sync/simple_sync.rs | 54 ++++++++++++-------- beacon_node/network/tests/tests.rs | 3 ++ 3 files changed, 45 insertions(+), 21 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index d5fd113a8..966e73210 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -64,6 +64,15 @@ impl BlockProcessingOutcome { }, } } + + /// Returns `true` if the block was successfully processed and can be removed from any import + /// queues or temporary storage. + pub fn sucessfully_processed(&self) -> bool { + match self { + BlockProcessingOutcome::ValidBlock(_) => true, + _ => false, + } + } } pub struct BeaconChain { diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 7609f5750..b77a976b1 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -425,14 +425,15 @@ impl SimpleSync { } pub fn process_import_queue(&mut self, network: &mut NetworkContext) { - let mut blocks: Vec<(Hash256, BeaconBlock, PeerId)> = self + let mut blocks: Vec<(usize, BeaconBlock, PeerId)> = self .import_queue .partials .iter() - .filter_map(|(key, partial)| { + .enumerate() + .filter_map(|(i, partial)| { if let Some(_) = partial.body { let (block, _root) = partial.clone().complete().expect("Body must be Some"); - Some((*key, block, partial.sender.clone())) + Some((i, block, partial.sender.clone())) } else { None } @@ -469,7 +470,7 @@ impl SimpleSync { if !keys_to_delete.is_empty() { info!(self.log, "Processed {} blocks", keys_to_delete.len()); for key in keys_to_delete { - self.import_queue.partials.remove(&key); + self.import_queue.partials.remove(key); } } } @@ -539,7 +540,7 @@ pub struct ImportQueue { /// BeaconChain pub chain: Arc, /// Partially imported blocks, keyed by the root of `BeaconBlockBody`. - pub partials: HashMap, + pub partials: Vec, /// Time before a queue entry is consider state. pub stale_time: Duration, /// Logging @@ -550,7 +551,7 @@ impl ImportQueue { pub fn new(chain: Arc, stale_time: Duration, log: slog::Logger) -> Self { Self { chain, - partials: HashMap::new(), + partials: vec![], stale_time, log, } @@ -561,29 +562,30 @@ impl ImportQueue { /// An entry is stale if it has as a `inserted` time that is more than `self.stale_time` in the /// past. pub fn remove_stale(&mut self) { - let keys: Vec = self + let stale_indices: Vec = self .partials .iter() - .filter_map(|(key, partial)| { + .enumerate() + .filter_map(|(i, partial)| { if partial.inserted + self.stale_time <= Instant::now() { - Some(*key) + Some(i) } else { None } }) .collect(); - if !keys.is_empty() { + if !stale_indices.is_empty() { debug!( self.log, "ImportQueue removing stale entries"; - "stale_count" => keys.len(), + "stale_items" => stale_indices.len(), "stale_time_seconds" => self.stale_time.as_secs() ); } - keys.iter().for_each(|key| { - self.partials.remove(&key); + stale_indices.iter().for_each(|&i| { + self.partials.remove(i); }); } @@ -646,29 +648,39 @@ impl ImportQueue { /// If the header already exists, the `inserted` time is set to `now` and not other /// modifications are made. fn insert_header(&mut self, block_root: Hash256, header: BeaconBlockHeader, sender: PeerId) { - self.partials - .entry(header.block_body_root) - .and_modify(|p| p.inserted = Instant::now()) - .or_insert(PartialBeaconBlock { + if let Some(i) = self + .partials + .iter() + .position(|p| p.block_root == block_root) + { + self.partials[i].inserted = Instant::now(); + } else { + self.partials.push(PartialBeaconBlock { block_root, header, body: None, inserted: Instant::now(), sender, - }); + }) + } } /// Updates an existing partial with the `body`. /// /// If there is no header for the `body`, the body is simply discarded. + /// + /// If the body already existed, the `inserted` time is set to `now`. fn insert_body(&mut self, body: BeaconBlockBody, sender: PeerId) { let body_root = Hash256::from_slice(&body.hash_tree_root()[..]); - self.partials.entry(body_root).and_modify(|p| { + self.partials.iter_mut().for_each(|mut p| { if body_root == p.header.block_body_root { - p.body = Some(body); p.inserted = Instant::now(); - p.sender = sender; + + if p.body.is_none() { + p.body = Some(body.clone()); + p.sender = sender.clone(); + } } }); } diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index b951d7d2a..2952e5105 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -511,7 +511,10 @@ fn sync_two_nodes() { // Node A builds out a longer, better chain. for _ in 0..blocks { + // Node A should build a block. node_a.harness.advance_chain_with_block(); + // Node B should just increment it's slot without a block. + node_b.harness.increment_beacon_chain_slot(); } node_a.harness.run_fork_choice(); From 1ea995963254ea4e4e8cd7f1a52d27d4a1b2e1ba Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 15:18:21 +1100 Subject: [PATCH 14/24] Fix bug with block processing in sync --- beacon_node/network/src/sync/simple_sync.rs | 104 +++++++++++++------- 1 file changed, 66 insertions(+), 38 deletions(-) diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index b77a976b1..369564a5e 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -425,54 +425,45 @@ impl SimpleSync { } pub fn process_import_queue(&mut self, network: &mut NetworkContext) { - let mut blocks: Vec<(usize, BeaconBlock, PeerId)> = self - .import_queue - .partials - .iter() - .enumerate() - .filter_map(|(i, partial)| { - if let Some(_) = partial.body { - let (block, _root) = partial.clone().complete().expect("Body must be Some"); - Some((i, block, partial.sender.clone())) - } else { - None - } - }) - .collect(); + let mut successful = 0; + let mut invalid = 0; + let mut errored = 0; - if !blocks.is_empty() { - info!(self.log, "Processing blocks"; "count" => blocks.len()); - } - - // Sort the blocks to be in ascending slot order. - blocks.sort_unstable_by(|a, b| a.1.slot.partial_cmp(&b.1.slot).unwrap()); - - let mut keys_to_delete = vec![]; - - for (key, block, sender) in blocks { + // Loop through all of the complete blocks in the queue. + for (queue_index, block, sender) in self.import_queue.complete_blocks() { match self.chain.process_block(block) { Ok(outcome) => { if outcome.is_invalid() { - warn!(self.log, "Invalid block: {:?}", outcome); + invalid += 1; + warn!( + self.log, + "InvalidBlock"; + "sender_peer_id" => format!("{:?}", sender), + "reason" => format!("{:?}", outcome), + ); network.disconnect(sender); - keys_to_delete.push(key) - } else { - // TODO: don't delete if was not invalid but not successfully processed. - keys_to_delete.push(key) + } + + // If this results to true, the item will be removed from the queue. + if outcome.sucessfully_processed() { + successful += 1; + self.import_queue.partials.remove(queue_index); } } Err(e) => { - error!(self.log, "Error during block processing"; "error" => format!("{:?}", e)) + errored += 1; + error!(self.log, "BlockProcessingError"; "error" => format!("{:?}", e)); } } } - if !keys_to_delete.is_empty() { - info!(self.log, "Processed {} blocks", keys_to_delete.len()); - for key in keys_to_delete { - self.import_queue.partials.remove(key); - } - } + info!( + self.log, + "ProcessBlocks"; + "invalid" => invalid, + "successful" => successful, + "errored" => errored, + ) } fn request_block_roots( @@ -557,6 +548,35 @@ impl ImportQueue { } } + /// Completes all possible partials into `BeaconBlock` and returns them, sorted by slot number. + /// Does not delete the partials from the queue, this must be done manually. + /// + /// Returns `(queue_index, block, sender)`: + /// + /// - `queue_index`: used to remove the entry if it is successfully processed. + /// - `block`: the completed block. + /// - `sender`: the `PeerId` the provided the `BeaconBlockBody` which completed the partial. + pub fn complete_blocks(&self) -> Vec<(usize, BeaconBlock, PeerId)> { + let mut completable: Vec<(usize, &PartialBeaconBlock)> = self + .partials + .iter() + .enumerate() + .filter(|(_i, partial)| partial.completable()) + .collect(); + + // Sort the completable partials to be in ascending slot order. + completable.sort_unstable_by(|a, b| a.1.header.slot.partial_cmp(&b.1.header.slot).unwrap()); + + completable + .iter() + .map(|(i, partial)| { + let (block, _root, sender) = + (*partial).clone().complete().expect("Body must be Some"); + (*i, block, sender) + }) + .collect() + } + /// Flushes all stale entries from the queue. /// /// An entry is stale if it has as a `inserted` time that is more than `self.stale_time` in the @@ -696,8 +716,16 @@ pub struct PartialBeaconBlock { } impl PartialBeaconBlock { + pub fn completable(&self) -> bool { + self.body.is_some() + } + /// Given a `body`, consumes `self` and returns a complete `BeaconBlock` along with its root. - pub fn complete(self) -> Option<(BeaconBlock, Hash256)> { - Some((self.header.into_block(self.body?), self.block_root)) + pub fn complete(self) -> Option<(BeaconBlock, Hash256, PeerId)> { + Some(( + self.header.into_block(self.body?), + self.block_root, + self.sender, + )) } } From 3dc5595a6fbbc3fd22e09fbcd36ca978d4b4186b Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 16:35:07 +1100 Subject: [PATCH 15/24] Fix last errors stopping full chain sync --- beacon_node/beacon_chain/src/beacon_chain.rs | 87 +++++++++++++------ beacon_node/beacon_chain/src/checkpoint.rs | 2 +- beacon_node/beacon_chain/src/errors.rs | 5 ++ .../test_harness/src/beacon_chain_harness.rs | 4 +- beacon_node/network/src/sync/simple_sync.rs | 60 ++++++------- beacon_node/network/tests/tests.rs | 24 ++++- 6 files changed, 121 insertions(+), 61 deletions(-) diff --git a/beacon_node/beacon_chain/src/beacon_chain.rs b/beacon_node/beacon_chain/src/beacon_chain.rs index 966e73210..745ba5155 100644 --- a/beacon_node/beacon_chain/src/beacon_chain.rs +++ b/beacon_node/beacon_chain/src/beacon_chain.rs @@ -26,7 +26,10 @@ pub enum ValidBlock { #[derive(Debug, PartialEq)] pub enum InvalidBlock { /// The block slot is greater than the present slot. - FutureSlot, + FutureSlot { + present_slot: Slot, + block_slot: Slot, + }, /// The block state_root does not match the generated state. StateRootMismatch, /// The blocks parent_root is unknown. @@ -53,7 +56,7 @@ impl BlockProcessingOutcome { match self { BlockProcessingOutcome::ValidBlock(_) => false, BlockProcessingOutcome::InvalidBlock(r) => match r { - InvalidBlock::FutureSlot => true, + InvalidBlock::FutureSlot { .. } => true, InvalidBlock::StateRootMismatch => true, InvalidBlock::ParentUnknown => false, InvalidBlock::SlotProcessingError(_) => false, @@ -302,6 +305,49 @@ where self.canonical_head.read() } + /// Updates the canonical `BeaconState` with the supplied state. + /// + /// Advances the chain forward to the present slot. This method is better than just setting + /// state and calling `catchup_state` as it will not result in an old state being installed and + /// then having it iteratively updated -- in such a case it's possible for another thread to + /// find the state at an old slot. + pub fn update_state(&self, mut state: BeaconState) -> Result<(), Error> { + let latest_block_header = self.head().beacon_block.block_header(); + + let present_slot = match self.slot_clock.present_slot() { + Ok(Some(slot)) => slot, + _ => return Err(Error::UnableToReadSlot), + }; + + // If required, transition the new state to the present slot. + for _ in state.slot.as_u64()..present_slot.as_u64() { + per_slot_processing(&mut state, &latest_block_header, &self.spec)?; + } + + *self.state.write() = state; + + Ok(()) + } + + /// Ensures the current canonical `BeaconState` has been transitioned to match the `slot_clock`. + pub fn catchup_state(&self) -> Result<(), Error> { + let latest_block_header = self.head().beacon_block.block_header(); + + let present_slot = match self.slot_clock.present_slot() { + Ok(Some(slot)) => slot, + _ => return Err(Error::UnableToReadSlot), + }; + + let mut state = self.state.write(); + + // If required, transition the new state to the present slot. + for _ in state.slot.as_u64()..present_slot.as_u64() { + per_slot_processing(&mut *state, &latest_block_header, &self.spec)?; + } + + Ok(()) + } + /// Update the justified head to some new values. pub fn update_finalized_head( &self, @@ -325,28 +371,6 @@ where self.finalized_head.read() } - /// Advance the `self.state` `BeaconState` to the supplied slot. - /// - /// This will perform per_slot and per_epoch processing as required. - /// - /// The `previous_block_root` will be set to the root of the current head block (as determined - /// by the fork-choice rule). - /// - /// It is important to note that this is _not_ the state corresponding to the canonical head - /// block, instead it is that state which may or may not have had additional per slot/epoch - /// processing applied to it. - pub fn advance_state(&self, slot: Slot) -> Result<(), SlotProcessingError> { - let state_slot = self.state.read().slot; - - let latest_block_header = self.head().beacon_block.block_header(); - - for _ in state_slot.as_u64()..slot.as_u64() { - per_slot_processing(&mut *self.state.write(), &latest_block_header, &self.spec)?; - } - - Ok(()) - } - /// Returns the validator index (if any) for the given public key. /// /// Information is retrieved from the present `beacon_state.validator_registry`. @@ -724,7 +748,10 @@ where if block.slot > present_slot { return Ok(BlockProcessingOutcome::InvalidBlock( - InvalidBlock::FutureSlot, + InvalidBlock::FutureSlot { + present_slot, + block_slot: block.slot, + }, )); } @@ -800,8 +827,9 @@ where // run instead. if self.head().beacon_block_root == parent_block_root { self.update_canonical_head(block.clone(), block_root, state.clone(), state_root); - // Update the local state variable. - *self.state.write() = state; + + // Update the canonical `BeaconState`. + self.update_state(state)?; } Ok(BlockProcessingOutcome::ValidBlock(ValidBlock::Processed)) @@ -891,7 +919,10 @@ where .ok_or_else(|| Error::MissingBeaconState(block.state_root))?; let state_root = state.canonical_root(); - self.update_canonical_head(block, block_root, state, state_root); + self.update_canonical_head(block, block_root, state.clone(), state_root); + + // Update the canonical `BeaconState`. + self.update_state(state)?; } Ok(()) diff --git a/beacon_node/beacon_chain/src/checkpoint.rs b/beacon_node/beacon_chain/src/checkpoint.rs index 828e462de..78227e5c8 100644 --- a/beacon_node/beacon_chain/src/checkpoint.rs +++ b/beacon_node/beacon_chain/src/checkpoint.rs @@ -3,7 +3,7 @@ use types::{BeaconBlock, BeaconState, Hash256}; /// Represents some block and it's associated state. Generally, this will be used for tracking the /// head, justified head and finalized head. -#[derive(Clone, Serialize)] +#[derive(Clone, Serialize, PartialEq, Debug)] pub struct CheckPoint { pub beacon_block: BeaconBlock, pub beacon_block_root: Hash256, diff --git a/beacon_node/beacon_chain/src/errors.rs b/beacon_node/beacon_chain/src/errors.rs index b5f17efd2..a84e4b10e 100644 --- a/beacon_node/beacon_chain/src/errors.rs +++ b/beacon_node/beacon_chain/src/errors.rs @@ -1,5 +1,6 @@ use fork_choice::ForkChoiceError; use state_processing::BlockProcessingError; +use state_processing::SlotProcessingError; use types::*; macro_rules! easy_from_to { @@ -16,14 +17,18 @@ macro_rules! easy_from_to { pub enum BeaconChainError { InsufficientValidators, BadRecentBlockRoots, + UnableToReadSlot, BeaconStateError(BeaconStateError), DBInconsistent(String), DBError(String), ForkChoiceError(ForkChoiceError), MissingBeaconBlock(Hash256), MissingBeaconState(Hash256), + SlotProcessingError(SlotProcessingError), } +easy_from_to!(SlotProcessingError, BeaconChainError); + #[derive(Debug, PartialEq)] pub enum BlockProductionError { UnableToGetBlockRootFromState, diff --git a/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs b/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs index 1207fcf28..1498796b1 100644 --- a/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs +++ b/beacon_node/beacon_chain/test_harness/src/beacon_chain_harness.rs @@ -131,7 +131,9 @@ impl BeaconChainHarness { ); self.beacon_chain.slot_clock.set_slot(slot.as_u64()); - self.beacon_chain.advance_state(slot).unwrap(); + self.beacon_chain + .catchup_state() + .expect("Failed to catch state"); slot } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 369564a5e..76d630b9a 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -430,7 +430,7 @@ impl SimpleSync { let mut errored = 0; // Loop through all of the complete blocks in the queue. - for (queue_index, block, sender) in self.import_queue.complete_blocks() { + for (block_root, block, sender) in self.import_queue.complete_blocks() { match self.chain.process_block(block) { Ok(outcome) => { if outcome.is_invalid() { @@ -447,7 +447,7 @@ impl SimpleSync { // If this results to true, the item will be removed from the queue. if outcome.sucessfully_processed() { successful += 1; - self.import_queue.partials.remove(queue_index); + self.import_queue.remove(block_root); } } Err(e) => { @@ -457,13 +457,15 @@ impl SimpleSync { } } - info!( - self.log, - "ProcessBlocks"; - "invalid" => invalid, - "successful" => successful, - "errored" => errored, - ) + if successful > 0 { + info!(self.log, "Imported {} blocks", successful) + } + if invalid > 0 { + warn!(self.log, "Rejected {} invalid blocks", invalid) + } + if errored > 0 { + warn!(self.log, "Failed to process {} blocks", errored) + } } fn request_block_roots( @@ -548,33 +550,35 @@ impl ImportQueue { } } - /// Completes all possible partials into `BeaconBlock` and returns them, sorted by slot number. - /// Does not delete the partials from the queue, this must be done manually. + /// Completes all possible partials into `BeaconBlock` and returns them, sorted by increasing + /// slot number. Does not delete the partials from the queue, this must be done manually. /// /// Returns `(queue_index, block, sender)`: /// - /// - `queue_index`: used to remove the entry if it is successfully processed. + /// - `block_root`: may be used to remove the entry if it is successfully processed. /// - `block`: the completed block. /// - `sender`: the `PeerId` the provided the `BeaconBlockBody` which completed the partial. - pub fn complete_blocks(&self) -> Vec<(usize, BeaconBlock, PeerId)> { - let mut completable: Vec<(usize, &PartialBeaconBlock)> = self + pub fn complete_blocks(&self) -> Vec<(Hash256, BeaconBlock, PeerId)> { + let mut complete: Vec<(Hash256, BeaconBlock, PeerId)> = self .partials .iter() - .enumerate() - .filter(|(_i, partial)| partial.completable()) + .filter_map(|partial| partial.clone().complete()) .collect(); // Sort the completable partials to be in ascending slot order. - completable.sort_unstable_by(|a, b| a.1.header.slot.partial_cmp(&b.1.header.slot).unwrap()); + complete.sort_unstable_by(|a, b| a.1.slot.partial_cmp(&b.1.slot).unwrap()); - completable + complete + } + + /// Removes the first `PartialBeaconBlock` with a matching `block_root`, returning the partial + /// if it exists. + pub fn remove(&mut self, block_root: Hash256) -> Option { + let position = self + .partials .iter() - .map(|(i, partial)| { - let (block, _root, sender) = - (*partial).clone().complete().expect("Body must be Some"); - (*i, block, sender) - }) - .collect() + .position(|p| p.block_root == block_root)?; + Some(self.partials.remove(position)) } /// Flushes all stale entries from the queue. @@ -716,15 +720,11 @@ pub struct PartialBeaconBlock { } impl PartialBeaconBlock { - pub fn completable(&self) -> bool { - self.body.is_some() - } - /// Given a `body`, consumes `self` and returns a complete `BeaconBlock` along with its root. - pub fn complete(self) -> Option<(BeaconBlock, Hash256, PeerId)> { + pub fn complete(self) -> Option<(Hash256, BeaconBlock, PeerId)> { Some(( - self.header.into_block(self.body?), self.block_root, + self.header.into_block(self.body?), self.sender, )) } diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index 2952e5105..e12bf2628 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -541,6 +541,28 @@ fn sync_two_nodes() { // A provides block bodies to B. node_a.tee_block_body_response(&node_b); - std::thread::sleep(Duration::from_secs(60)); + std::thread::sleep(Duration::from_secs(10)); + + node_b.harness.run_fork_choice(); + + let node_a_chain = node_a + .harness + .beacon_chain + .chain_dump() + .expect("Can't dump node a chain"); + + let node_b_chain = node_b + .harness + .beacon_chain + .chain_dump() + .expect("Can't dump node b chain"); + + assert_eq!( + node_a_chain.len(), + node_b_chain.len(), + "Chains should be equal length" + ); + assert_eq!(node_a_chain, node_b_chain, "Chains should be identical"); + runtime.shutdown_now(); } From 796b68dc042920a49336aa646cbb04ed16da1add Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 17:51:32 +1100 Subject: [PATCH 16/24] Implement Goodbye and BeaconState msg handlers --- beacon_node/eth2-libp2p/src/rpc/methods.rs | 52 ++++++++++++++++++++- beacon_node/eth2-libp2p/src/rpc/protocol.rs | 4 +- beacon_node/network/src/message_handler.rs | 22 ++++++--- beacon_node/network/src/sync/simple_sync.rs | 10 ++++ 4 files changed, 79 insertions(+), 9 deletions(-) diff --git a/beacon_node/eth2-libp2p/src/rpc/methods.rs b/beacon_node/eth2-libp2p/src/rpc/methods.rs index f6a5f2829..85ef7e06f 100644 --- a/beacon_node/eth2-libp2p/src/rpc/methods.rs +++ b/beacon_node/eth2-libp2p/src/rpc/methods.rs @@ -1,3 +1,4 @@ +use ssz::{Decodable, DecodeError, Encodable, SszStream}; /// Available RPC methods types and ids. use ssz_derive::{Decode, Encode}; use types::{BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; @@ -53,7 +54,7 @@ impl Into for RPCMethod { #[derive(Debug, Clone)] pub enum RPCRequest { Hello(HelloMessage), - Goodbye(u64), + Goodbye(GoodbyeReason), BeaconBlockRoots(BeaconBlockRootsRequest), BeaconBlockHeaders(BeaconBlockHeadersRequest), BeaconBlockBodies(BeaconBlockBodiesRequest), @@ -113,6 +114,55 @@ pub struct HelloMessage { pub best_slot: Slot, } +/// The reason given for a `Goodbye` message. +/// +/// Note: any unknown `u64::into(n)` will resolve to `GoodbyeReason::Unknown` for any unknown `n`, +/// however `GoodbyeReason::Unknown.into()` will go into `0_u64`. Therefore de-serializing then +/// re-serializing may not return the same bytes. +#[derive(Debug, Clone)] +pub enum GoodbyeReason { + ClientShutdown, + IrreleventNetwork, + Fault, + Unknown, +} + +impl From for GoodbyeReason { + fn from(id: u64) -> GoodbyeReason { + match id { + 1 => GoodbyeReason::ClientShutdown, + 2 => GoodbyeReason::IrreleventNetwork, + 3 => GoodbyeReason::Fault, + _ => GoodbyeReason::Unknown, + } + } +} + +impl Into for GoodbyeReason { + fn into(self) -> u64 { + match self { + GoodbyeReason::Unknown => 0, + GoodbyeReason::ClientShutdown => 1, + GoodbyeReason::IrreleventNetwork => 2, + GoodbyeReason::Fault => 3, + } + } +} + +impl Encodable for GoodbyeReason { + fn ssz_append(&self, s: &mut SszStream) { + let id: u64 = (*self).clone().into(); + id.ssz_append(s); + } +} + +impl Decodable for GoodbyeReason { + fn ssz_decode(bytes: &[u8], index: usize) -> Result<(Self, usize), DecodeError> { + let (id, index) = u64::ssz_decode(bytes, index)?; + Ok((Self::from(id), index)) + } +} + /// Request a number of beacon block roots from a peer. #[derive(Encode, Decode, Clone, Debug, PartialEq)] pub struct BeaconBlockRootsRequest { diff --git a/beacon_node/eth2-libp2p/src/rpc/protocol.rs b/beacon_node/eth2-libp2p/src/rpc/protocol.rs index f4fe26fac..b328dd0dd 100644 --- a/beacon_node/eth2-libp2p/src/rpc/protocol.rs +++ b/beacon_node/eth2-libp2p/src/rpc/protocol.rs @@ -82,8 +82,8 @@ fn decode(packet: Vec) -> Result { RPCRequest::Hello(hello_body) } RPCMethod::Goodbye => { - let (goodbye_code, _index) = u64::ssz_decode(&packet, index)?; - RPCRequest::Goodbye(goodbye_code) + let (goodbye_reason, _index) = GoodbyeReason::ssz_decode(&packet, index)?; + RPCRequest::Goodbye(goodbye_reason) } RPCMethod::BeaconBlockRoots => { let (block_roots_request, _index) = diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 77a2ab8db..57923b2c3 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -107,13 +107,14 @@ impl MessageHandler { } /// A new RPC request has been received from the network. - fn handle_rpc_request(&mut self, peer_id: PeerId, id: u64, request: RPCRequest) { - // TODO: ensure the id is legit + fn handle_rpc_request(&mut self, peer_id: PeerId, _id: u64, request: RPCRequest) { + // TODO: process the `id`. match request { RPCRequest::Hello(hello_message) => { self.sync .on_hello_request(peer_id, hello_message, &mut self.network_context) } + RPCRequest::Goodbye(goodbye_reason) => self.sync.on_goodbye(peer_id, goodbye_reason), RPCRequest::BeaconBlockRoots(request) => { self.sync .on_beacon_block_roots_request(peer_id, request, &mut self.network_context) @@ -128,8 +129,11 @@ impl MessageHandler { request, &mut self.network_context, ), - // TODO: Handle all requests - _ => panic!("Unknown request: {:?}", request), + RPCRequest::BeaconChainState(_) => { + // We do not implement this endpoint, it is not required and will only likely be + // useful for light-client support in later phases. + warn!(self.log, "BeaconChainState RPC call is not supported."); + } } } @@ -172,8 +176,14 @@ impl MessageHandler { &mut self.network_context, ); } - // TODO: Handle all responses - _ => panic!("Unknown response: {:?}", response), + RPCResponse::BeaconChainState(_) => { + // We do not implement this endpoint, it is not required and will only likely be + // useful for light-client support in later phases. + // + // Theoretically, we shouldn't reach this code because we should never send a + // beacon state RPC request. + warn!(self.log, "BeaconChainState RPC call is not supported."); + } }; } } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 76d630b9a..14564aa37 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -119,6 +119,16 @@ impl SimpleSync { } } + pub fn on_goodbye(&mut self, peer_id: PeerId, reason: GoodbyeReason) { + info!( + self.log, "PeerGoodbye"; + "peer" => format!("{:?}", peer_id), + "reason" => format!("{:?}", reason), + ); + + self.known_peers.remove(&peer_id); + } + pub fn on_connect(&self, peer_id: PeerId, network: &mut NetworkContext) { info!(self.log, "PeerConnect"; "peer" => format!("{:?}", peer_id)); From 07b6f7084292c864b3d0f969c36abd6c6dde4ecd Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 17:58:30 +1100 Subject: [PATCH 17/24] Ignore syncing tests (they run too long) --- beacon_node/network/tests/tests.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index e12bf2628..110450dc9 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -450,6 +450,7 @@ pub fn build_blocks(blocks: usize, master: &mut SyncMaster, nodes: &mut Vec Date: Sun, 24 Mar 2019 18:31:03 +1100 Subject: [PATCH 18/24] Add comments to SimpleSync --- beacon_node/network/src/sync/simple_sync.rs | 75 +++++++++++++++++++-- 1 file changed, 69 insertions(+), 6 deletions(-) diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 14564aa37..d6b9e63ae 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -27,18 +27,22 @@ pub struct PeerSyncInfo { } impl PeerSyncInfo { + /// Returns `true` if the peer is on the same chain as `other`. fn is_on_same_chain(&self, other: Self) -> bool { self.network_id == other.network_id } + /// Returns `true` if the peer has a higher finalized epoch than `other`. fn has_higher_finalized_epoch_than(&self, other: Self) -> bool { self.latest_finalized_epoch > other.latest_finalized_epoch } + /// Returns `true` if the peer has a higher best slot than `other`. fn has_higher_best_slot_than(&self, other: Self) -> bool { self.best_slot > other.best_slot } + /// Returns the `PeerStatus` of `self` in relation to `other`. pub fn status_compared_to(&self, other: Self) -> PeerStatus { if self.has_higher_finalized_epoch_than(other) { PeerStatus::HigherFinalizedEpoch @@ -52,11 +56,17 @@ impl PeerSyncInfo { } } +/// The status of a peers view on the chain, relative to some other view of the chain (presumably +/// our view). #[derive(PartialEq, Clone, Copy, Debug)] pub enum PeerStatus { + /// The peer is on a completely different chain. OnDifferentChain, + /// The peer has a higher finalized epoch. HigherFinalizedEpoch, + /// The peer has a higher best slot. HigherBestSlot, + /// The peer has the same or lesser view of the chain. We have nothing to request of them. NotInteresting, } @@ -87,8 +97,6 @@ pub enum SyncState { } /// Simple Syncing protocol. -//TODO: Decide for HELLO messages whether its better to keep current in RAM or build on the fly -//when asked. pub struct SimpleSync { /// A reference to the underlying beacon chain. chain: Arc, @@ -103,6 +111,7 @@ pub struct SimpleSync { } impl SimpleSync { + /// Instantiate a `SimpleSync` instance, with no peers and an empty queue. pub fn new(beacon_chain: Arc, log: &slog::Logger) -> Self { let sync_logger = log.new(o!("Service"=> "Sync")); @@ -119,7 +128,15 @@ impl SimpleSync { } } - pub fn on_goodbye(&mut self, peer_id: PeerId, reason: GoodbyeReason) { + /// Handle a `Goodbye` message from a peer. + /// + /// Removes the peer from `known_peers`. + pub fn on_goodbye( + &mut self, + peer_id: PeerId, + reason: GoodbyeReason, + ddnetwork: &mut NetworkContext, + ) { info!( self.log, "PeerGoodbye"; "peer" => format!("{:?}", peer_id), @@ -129,12 +146,18 @@ impl SimpleSync { self.known_peers.remove(&peer_id); } + /// Handle the connection of a new peer. + /// + /// Sends a `Hello` message to the peer. pub fn on_connect(&self, peer_id: PeerId, network: &mut NetworkContext) { info!(self.log, "PeerConnect"; "peer" => format!("{:?}", peer_id)); network.send_rpc_request(peer_id, RPCRequest::Hello(self.chain.hello_message())); } + /// Handle a `Hello` request. + /// + /// Processes the `HelloMessage` from the remote peer and sends back our `Hello`. pub fn on_hello_request( &mut self, peer_id: PeerId, @@ -152,6 +175,7 @@ impl SimpleSync { self.process_hello(peer_id, hello, network); } + /// Process a `Hello` response from a peer. pub fn on_hello_response( &mut self, peer_id: PeerId, @@ -164,6 +188,9 @@ impl SimpleSync { self.process_hello(peer_id, hello, network); } + /// Process a `Hello` message, requesting new blocks if appropriate. + /// + /// Disconnects the peer if required. fn process_hello( &mut self, peer_id: PeerId, @@ -223,6 +250,7 @@ impl SimpleSync { } } + /// Handle a `BeaconBlockRoots` request from the peer. pub fn on_beacon_block_roots_request( &mut self, peer_id: PeerId, @@ -268,6 +296,7 @@ impl SimpleSync { ) } + /// Handle a `BeaconBlockRoots` response from the peer. pub fn on_beacon_block_roots_response( &mut self, peer_id: PeerId, @@ -311,6 +340,7 @@ impl SimpleSync { } } + /// Handle a `BeaconBlockHeaders` request from the peer. pub fn on_beacon_block_headers_request( &mut self, peer_id: PeerId, @@ -348,6 +378,7 @@ impl SimpleSync { ) } + /// Handle a `BeaconBlockHeaders` response from the peer. pub fn on_beacon_block_headers_response( &mut self, peer_id: PeerId, @@ -378,6 +409,7 @@ impl SimpleSync { self.request_block_bodies(peer_id, BeaconBlockBodiesRequest { block_roots }, network); } + /// Handle a `BeaconBlockBodies` request from the peer. pub fn on_beacon_block_bodies_request( &mut self, peer_id: PeerId, @@ -411,6 +443,7 @@ impl SimpleSync { ) } + /// Handle a `BeaconBlockBodies` response from the peer. pub fn on_beacon_block_bodies_response( &mut self, peer_id: PeerId, @@ -434,6 +467,10 @@ impl SimpleSync { self.process_import_queue(network); } + /// Iterate through the `import_queue` and process any complete blocks. + /// + /// If a block is successfully processed it is removed from the queue, otherwise it remains in + /// the queue. pub fn process_import_queue(&mut self, network: &mut NetworkContext) { let mut successful = 0; let mut invalid = 0; @@ -478,6 +515,7 @@ impl SimpleSync { } } + /// Request some `BeaconBlockRoots` from the remote peer. fn request_block_roots( &mut self, peer_id: PeerId, @@ -501,6 +539,7 @@ impl SimpleSync { network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockRoots(req)); } + /// Request some `BeaconBlockHeaders` from the remote peer. fn request_block_headers( &mut self, peer_id: PeerId, @@ -517,6 +556,7 @@ impl SimpleSync { network.send_rpc_request(peer_id.clone(), RPCRequest::BeaconBlockHeaders(req)); } + /// Request some `BeaconBlockBodies` from the remote peer. fn request_block_bodies( &mut self, peer_id: PeerId, @@ -539,18 +579,30 @@ impl SimpleSync { } } +/// Provides a queue for fully and partially built `BeaconBlock`s. +/// +/// The queue is fundamentally a `Vec` where no two items have the same +/// `item.block_root`. This struct it backed by a `Vec` not a `HashMap` for the following two +/// reasons: +/// +/// - When we receive a `BeaconBlockBody`, the only way we can find it's matching +/// `BeaconBlockHeader` is to find a header such that `header.beacon_block_body == +/// hash_tree_root(body)`. Therefore, if we used a `HashMap` we would need to use the root of +/// `BeaconBlockBody` as the key. +/// - It is possible for multiple distinct blocks to have identical `BeaconBlockBodies`. Therefore +/// we cannot use a `HashMap` keyed by the root of `BeaconBlockBody`. pub struct ImportQueue { - /// BeaconChain pub chain: Arc, /// Partially imported blocks, keyed by the root of `BeaconBlockBody`. pub partials: Vec, - /// Time before a queue entry is consider state. + /// Time before a queue entry is considered state. pub stale_time: Duration, /// Logging log: slog::Logger, } impl ImportQueue { + /// Return a new, empty queue. pub fn new(chain: Arc, stale_time: Duration, log: slog::Logger) -> Self { Self { chain, @@ -649,6 +701,10 @@ impl ImportQueue { /// If a `header` is already in the queue, but not yet processed by the chain the block root is /// included in the output and the `inserted` time for the partial record is set to /// `Instant::now()`. Updating the `inserted` time stops the partial from becoming stale. + /// + /// Presently the queue enforces that a `BeaconBlockHeader` _must_ be received before its + /// `BeaconBlockBody`. This is not a natural requirement and we could enhance the queue to lift + /// this restraint. pub fn enqueue_headers( &mut self, headers: Vec, @@ -720,17 +776,24 @@ impl ImportQueue { } } +/// Individual components of a `BeaconBlock`, potentially all that are required to form a full +/// `BeaconBlock`. #[derive(Clone, Debug)] pub struct PartialBeaconBlock { + /// `BeaconBlock` root. pub block_root: Hash256, pub header: BeaconBlockHeader, pub body: Option, + /// The instant at which this record was created or last meaningfully modified. Used to + /// determine if an entry is stale and should be removed. pub inserted: Instant, + /// The `PeerId` that last meaningfully contributed to this item. pub sender: PeerId, } impl PartialBeaconBlock { - /// Given a `body`, consumes `self` and returns a complete `BeaconBlock` along with its root. + /// Consumes `self` and returns a full built `BeaconBlock`, it's root and the `sender` + /// `PeerId`, if enough information exists to complete the block. Otherwise, returns `None`. pub fn complete(self) -> Option<(Hash256, BeaconBlock, PeerId)> { Some(( self.block_root, From 755a09d164f7790cd74fda8efd3ae8d033579a88 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 18:34:44 +1100 Subject: [PATCH 19/24] Move ImportQueue into own file --- beacon_node/network/src/sync/import_queue.rs | 232 +++++++++++++++++++ beacon_node/network/src/sync/mod.rs | 1 + beacon_node/network/src/sync/simple_sync.rs | 230 +----------------- 3 files changed, 236 insertions(+), 227 deletions(-) create mode 100644 beacon_node/network/src/sync/import_queue.rs diff --git a/beacon_node/network/src/sync/import_queue.rs b/beacon_node/network/src/sync/import_queue.rs new file mode 100644 index 000000000..6508af89e --- /dev/null +++ b/beacon_node/network/src/sync/import_queue.rs @@ -0,0 +1,232 @@ +use crate::beacon_chain::BeaconChain; +use eth2_libp2p::rpc::methods::*; +use eth2_libp2p::PeerId; +use slog::{debug, error}; +use ssz::TreeHash; +use std::sync::Arc; +use std::time::{Duration, Instant}; +use types::{BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Hash256}; + +/// Provides a queue for fully and partially built `BeaconBlock`s. +/// +/// The queue is fundamentally a `Vec` where no two items have the same +/// `item.block_root`. This struct it backed by a `Vec` not a `HashMap` for the following two +/// reasons: +/// +/// - When we receive a `BeaconBlockBody`, the only way we can find it's matching +/// `BeaconBlockHeader` is to find a header such that `header.beacon_block_body == +/// hash_tree_root(body)`. Therefore, if we used a `HashMap` we would need to use the root of +/// `BeaconBlockBody` as the key. +/// - It is possible for multiple distinct blocks to have identical `BeaconBlockBodies`. Therefore +/// we cannot use a `HashMap` keyed by the root of `BeaconBlockBody`. +pub struct ImportQueue { + pub chain: Arc, + /// Partially imported blocks, keyed by the root of `BeaconBlockBody`. + pub partials: Vec, + /// Time before a queue entry is considered state. + pub stale_time: Duration, + /// Logging + log: slog::Logger, +} + +impl ImportQueue { + /// Return a new, empty queue. + pub fn new(chain: Arc, stale_time: Duration, log: slog::Logger) -> Self { + Self { + chain, + partials: vec![], + stale_time, + log, + } + } + + /// Completes all possible partials into `BeaconBlock` and returns them, sorted by increasing + /// slot number. Does not delete the partials from the queue, this must be done manually. + /// + /// Returns `(queue_index, block, sender)`: + /// + /// - `block_root`: may be used to remove the entry if it is successfully processed. + /// - `block`: the completed block. + /// - `sender`: the `PeerId` the provided the `BeaconBlockBody` which completed the partial. + pub fn complete_blocks(&self) -> Vec<(Hash256, BeaconBlock, PeerId)> { + let mut complete: Vec<(Hash256, BeaconBlock, PeerId)> = self + .partials + .iter() + .filter_map(|partial| partial.clone().complete()) + .collect(); + + // Sort the completable partials to be in ascending slot order. + complete.sort_unstable_by(|a, b| a.1.slot.partial_cmp(&b.1.slot).unwrap()); + + complete + } + + /// Removes the first `PartialBeaconBlock` with a matching `block_root`, returning the partial + /// if it exists. + pub fn remove(&mut self, block_root: Hash256) -> Option { + let position = self + .partials + .iter() + .position(|p| p.block_root == block_root)?; + Some(self.partials.remove(position)) + } + + /// Flushes all stale entries from the queue. + /// + /// An entry is stale if it has as a `inserted` time that is more than `self.stale_time` in the + /// past. + pub fn remove_stale(&mut self) { + let stale_indices: Vec = self + .partials + .iter() + .enumerate() + .filter_map(|(i, partial)| { + if partial.inserted + self.stale_time <= Instant::now() { + Some(i) + } else { + None + } + }) + .collect(); + + if !stale_indices.is_empty() { + debug!( + self.log, + "ImportQueue removing stale entries"; + "stale_items" => stale_indices.len(), + "stale_time_seconds" => self.stale_time.as_secs() + ); + } + + stale_indices.iter().for_each(|&i| { + self.partials.remove(i); + }); + } + + /// Returns `true` if `self.chain` has not yet processed this block. + fn is_new_block(&self, block_root: &Hash256) -> bool { + self.chain + .is_new_block_root(&block_root) + .unwrap_or_else(|_| { + error!(self.log, "Unable to determine if block is new."); + true + }) + } + + /// Returns the index of the first new root in the list of block roots. + pub fn first_new_root(&mut self, roots: &[BlockRootSlot]) -> Option { + roots + .iter() + .position(|brs| self.is_new_block(&brs.block_root)) + } + + /// Adds the `headers` to the `partials` queue. Returns a list of `Hash256` block roots for + /// which we should use to request `BeaconBlockBodies`. + /// + /// If a `header` is not in the queue and has not been processed by the chain it is added to + /// the queue and it's block root is included in the output. + /// + /// If a `header` is already in the queue, but not yet processed by the chain the block root is + /// included in the output and the `inserted` time for the partial record is set to + /// `Instant::now()`. Updating the `inserted` time stops the partial from becoming stale. + /// + /// Presently the queue enforces that a `BeaconBlockHeader` _must_ be received before its + /// `BeaconBlockBody`. This is not a natural requirement and we could enhance the queue to lift + /// this restraint. + pub fn enqueue_headers( + &mut self, + headers: Vec, + sender: PeerId, + ) -> Vec { + let mut required_bodies: Vec = vec![]; + + for header in headers { + let block_root = Hash256::from_slice(&header.hash_tree_root()[..]); + + if self.is_new_block(&block_root) { + self.insert_header(block_root, header, sender.clone()); + required_bodies.push(block_root) + } + } + + required_bodies + } + + /// If there is a matching `header` for this `body`, adds it to the queue. + /// + /// If there is no `header` for the `body`, the body is simply discarded. + pub fn enqueue_bodies(&mut self, bodies: Vec, sender: PeerId) { + for body in bodies { + self.insert_body(body, sender.clone()); + } + } + + /// Inserts a header to the queue. + /// + /// If the header already exists, the `inserted` time is set to `now` and not other + /// modifications are made. + fn insert_header(&mut self, block_root: Hash256, header: BeaconBlockHeader, sender: PeerId) { + if let Some(i) = self + .partials + .iter() + .position(|p| p.block_root == block_root) + { + self.partials[i].inserted = Instant::now(); + } else { + self.partials.push(PartialBeaconBlock { + block_root, + header, + body: None, + inserted: Instant::now(), + sender, + }) + } + } + + /// Updates an existing partial with the `body`. + /// + /// If there is no header for the `body`, the body is simply discarded. + /// + /// If the body already existed, the `inserted` time is set to `now`. + fn insert_body(&mut self, body: BeaconBlockBody, sender: PeerId) { + let body_root = Hash256::from_slice(&body.hash_tree_root()[..]); + + self.partials.iter_mut().for_each(|mut p| { + if body_root == p.header.block_body_root { + p.inserted = Instant::now(); + + if p.body.is_none() { + p.body = Some(body.clone()); + p.sender = sender.clone(); + } + } + }); + } +} + +/// Individual components of a `BeaconBlock`, potentially all that are required to form a full +/// `BeaconBlock`. +#[derive(Clone, Debug)] +pub struct PartialBeaconBlock { + /// `BeaconBlock` root. + pub block_root: Hash256, + pub header: BeaconBlockHeader, + pub body: Option, + /// The instant at which this record was created or last meaningfully modified. Used to + /// determine if an entry is stale and should be removed. + pub inserted: Instant, + /// The `PeerId` that last meaningfully contributed to this item. + pub sender: PeerId, +} + +impl PartialBeaconBlock { + /// Consumes `self` and returns a full built `BeaconBlock`, it's root and the `sender` + /// `PeerId`, if enough information exists to complete the block. Otherwise, returns `None`. + pub fn complete(self) -> Option<(Hash256, BeaconBlock, PeerId)> { + Some(( + self.block_root, + self.header.into_block(self.body?), + self.sender, + )) + } +} diff --git a/beacon_node/network/src/sync/mod.rs b/beacon_node/network/src/sync/mod.rs index 8f5216b85..fac1b46eb 100644 --- a/beacon_node/network/src/sync/mod.rs +++ b/beacon_node/network/src/sync/mod.rs @@ -1,3 +1,4 @@ +mod import_queue; /// Syncing for lighthouse. /// /// Stores the various syncing methods for the beacon chain. diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index d6b9e63ae..c23a6ec56 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -1,14 +1,14 @@ +use super::import_queue::ImportQueue; use crate::beacon_chain::BeaconChain; use crate::message_handler::NetworkContext; use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{RPCRequest, RPCResponse}; use eth2_libp2p::PeerId; use slog::{debug, error, info, o, warn}; -use ssz::TreeHash; use std::collections::HashMap; use std::sync::Arc; -use std::time::{Duration, Instant}; -use types::{BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; +use std::time::Duration; +use types::{Epoch, Hash256, Slot}; /// The number of slots that we can import blocks ahead of us, before going into full Sync mode. const SLOT_IMPORT_TOLERANCE: u64 = 100; @@ -578,227 +578,3 @@ impl SimpleSync { self.chain.hello_message() } } - -/// Provides a queue for fully and partially built `BeaconBlock`s. -/// -/// The queue is fundamentally a `Vec` where no two items have the same -/// `item.block_root`. This struct it backed by a `Vec` not a `HashMap` for the following two -/// reasons: -/// -/// - When we receive a `BeaconBlockBody`, the only way we can find it's matching -/// `BeaconBlockHeader` is to find a header such that `header.beacon_block_body == -/// hash_tree_root(body)`. Therefore, if we used a `HashMap` we would need to use the root of -/// `BeaconBlockBody` as the key. -/// - It is possible for multiple distinct blocks to have identical `BeaconBlockBodies`. Therefore -/// we cannot use a `HashMap` keyed by the root of `BeaconBlockBody`. -pub struct ImportQueue { - pub chain: Arc, - /// Partially imported blocks, keyed by the root of `BeaconBlockBody`. - pub partials: Vec, - /// Time before a queue entry is considered state. - pub stale_time: Duration, - /// Logging - log: slog::Logger, -} - -impl ImportQueue { - /// Return a new, empty queue. - pub fn new(chain: Arc, stale_time: Duration, log: slog::Logger) -> Self { - Self { - chain, - partials: vec![], - stale_time, - log, - } - } - - /// Completes all possible partials into `BeaconBlock` and returns them, sorted by increasing - /// slot number. Does not delete the partials from the queue, this must be done manually. - /// - /// Returns `(queue_index, block, sender)`: - /// - /// - `block_root`: may be used to remove the entry if it is successfully processed. - /// - `block`: the completed block. - /// - `sender`: the `PeerId` the provided the `BeaconBlockBody` which completed the partial. - pub fn complete_blocks(&self) -> Vec<(Hash256, BeaconBlock, PeerId)> { - let mut complete: Vec<(Hash256, BeaconBlock, PeerId)> = self - .partials - .iter() - .filter_map(|partial| partial.clone().complete()) - .collect(); - - // Sort the completable partials to be in ascending slot order. - complete.sort_unstable_by(|a, b| a.1.slot.partial_cmp(&b.1.slot).unwrap()); - - complete - } - - /// Removes the first `PartialBeaconBlock` with a matching `block_root`, returning the partial - /// if it exists. - pub fn remove(&mut self, block_root: Hash256) -> Option { - let position = self - .partials - .iter() - .position(|p| p.block_root == block_root)?; - Some(self.partials.remove(position)) - } - - /// Flushes all stale entries from the queue. - /// - /// An entry is stale if it has as a `inserted` time that is more than `self.stale_time` in the - /// past. - pub fn remove_stale(&mut self) { - let stale_indices: Vec = self - .partials - .iter() - .enumerate() - .filter_map(|(i, partial)| { - if partial.inserted + self.stale_time <= Instant::now() { - Some(i) - } else { - None - } - }) - .collect(); - - if !stale_indices.is_empty() { - debug!( - self.log, - "ImportQueue removing stale entries"; - "stale_items" => stale_indices.len(), - "stale_time_seconds" => self.stale_time.as_secs() - ); - } - - stale_indices.iter().for_each(|&i| { - self.partials.remove(i); - }); - } - - /// Returns `true` if `self.chain` has not yet processed this block. - fn is_new_block(&self, block_root: &Hash256) -> bool { - self.chain - .is_new_block_root(&block_root) - .unwrap_or_else(|_| { - error!(self.log, "Unable to determine if block is new."); - true - }) - } - - /// Returns the index of the first new root in the list of block roots. - pub fn first_new_root(&mut self, roots: &[BlockRootSlot]) -> Option { - roots - .iter() - .position(|brs| self.is_new_block(&brs.block_root)) - } - - /// Adds the `headers` to the `partials` queue. Returns a list of `Hash256` block roots for - /// which we should use to request `BeaconBlockBodies`. - /// - /// If a `header` is not in the queue and has not been processed by the chain it is added to - /// the queue and it's block root is included in the output. - /// - /// If a `header` is already in the queue, but not yet processed by the chain the block root is - /// included in the output and the `inserted` time for the partial record is set to - /// `Instant::now()`. Updating the `inserted` time stops the partial from becoming stale. - /// - /// Presently the queue enforces that a `BeaconBlockHeader` _must_ be received before its - /// `BeaconBlockBody`. This is not a natural requirement and we could enhance the queue to lift - /// this restraint. - pub fn enqueue_headers( - &mut self, - headers: Vec, - sender: PeerId, - ) -> Vec { - let mut required_bodies: Vec = vec![]; - - for header in headers { - let block_root = Hash256::from_slice(&header.hash_tree_root()[..]); - - if self.is_new_block(&block_root) { - self.insert_header(block_root, header, sender.clone()); - required_bodies.push(block_root) - } - } - - required_bodies - } - - /// If there is a matching `header` for this `body`, adds it to the queue. - /// - /// If there is no `header` for the `body`, the body is simply discarded. - pub fn enqueue_bodies(&mut self, bodies: Vec, sender: PeerId) { - for body in bodies { - self.insert_body(body, sender.clone()); - } - } - - /// Inserts a header to the queue. - /// - /// If the header already exists, the `inserted` time is set to `now` and not other - /// modifications are made. - fn insert_header(&mut self, block_root: Hash256, header: BeaconBlockHeader, sender: PeerId) { - if let Some(i) = self - .partials - .iter() - .position(|p| p.block_root == block_root) - { - self.partials[i].inserted = Instant::now(); - } else { - self.partials.push(PartialBeaconBlock { - block_root, - header, - body: None, - inserted: Instant::now(), - sender, - }) - } - } - - /// Updates an existing partial with the `body`. - /// - /// If there is no header for the `body`, the body is simply discarded. - /// - /// If the body already existed, the `inserted` time is set to `now`. - fn insert_body(&mut self, body: BeaconBlockBody, sender: PeerId) { - let body_root = Hash256::from_slice(&body.hash_tree_root()[..]); - - self.partials.iter_mut().for_each(|mut p| { - if body_root == p.header.block_body_root { - p.inserted = Instant::now(); - - if p.body.is_none() { - p.body = Some(body.clone()); - p.sender = sender.clone(); - } - } - }); - } -} - -/// Individual components of a `BeaconBlock`, potentially all that are required to form a full -/// `BeaconBlock`. -#[derive(Clone, Debug)] -pub struct PartialBeaconBlock { - /// `BeaconBlock` root. - pub block_root: Hash256, - pub header: BeaconBlockHeader, - pub body: Option, - /// The instant at which this record was created or last meaningfully modified. Used to - /// determine if an entry is stale and should be removed. - pub inserted: Instant, - /// The `PeerId` that last meaningfully contributed to this item. - pub sender: PeerId, -} - -impl PartialBeaconBlock { - /// Consumes `self` and returns a full built `BeaconBlock`, it's root and the `sender` - /// `PeerId`, if enough information exists to complete the block. Otherwise, returns `None`. - pub fn complete(self) -> Option<(Hash256, BeaconBlock, PeerId)> { - Some(( - self.block_root, - self.header.into_block(self.body?), - self.sender, - )) - } -} From 267477ffc248e35487358a7f8d57b2a0de23e3a2 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 24 Mar 2019 18:37:23 +1100 Subject: [PATCH 20/24] Fix typo from previous commit --- beacon_node/network/src/sync/simple_sync.rs | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index c23a6ec56..4ee349043 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -131,12 +131,7 @@ impl SimpleSync { /// Handle a `Goodbye` message from a peer. /// /// Removes the peer from `known_peers`. - pub fn on_goodbye( - &mut self, - peer_id: PeerId, - reason: GoodbyeReason, - ddnetwork: &mut NetworkContext, - ) { + pub fn on_goodbye(&mut self, peer_id: PeerId, reason: GoodbyeReason) { info!( self.log, "PeerGoodbye"; "peer" => format!("{:?}", peer_id), From 708d9b5674aeb1921778a489a4129ba4194e6713 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 25 Mar 2019 14:27:20 +1100 Subject: [PATCH 21/24] Add basic Gossip sync handlers --- beacon_node/beacon_chain/src/lib.rs | 1 + beacon_node/eth2-libp2p/src/rpc/methods.rs | 20 ++++++- beacon_node/eth2-libp2p/src/rpc/mod.rs | 2 +- beacon_node/network/src/beacon_chain.rs | 21 ++++++- beacon_node/network/src/message_handler.rs | 23 +++++++- beacon_node/network/src/sync/import_queue.rs | 2 +- beacon_node/network/src/sync/simple_sync.rs | 58 ++++++++++++++++++++ 7 files changed, 119 insertions(+), 8 deletions(-) diff --git a/beacon_node/beacon_chain/src/lib.rs b/beacon_node/beacon_chain/src/lib.rs index 5fa7e7a77..48a42b941 100644 --- a/beacon_node/beacon_chain/src/lib.rs +++ b/beacon_node/beacon_chain/src/lib.rs @@ -8,6 +8,7 @@ pub mod test_utils; pub use self::beacon_chain::{BeaconChain, BlockProcessingOutcome, InvalidBlock, ValidBlock}; pub use self::checkpoint::CheckPoint; pub use self::errors::BeaconChainError; +pub use attestation_aggregator::Outcome as AggregationOutcome; pub use db; pub use fork_choice; pub use parking_lot; diff --git a/beacon_node/eth2-libp2p/src/rpc/methods.rs b/beacon_node/eth2-libp2p/src/rpc/methods.rs index 85ef7e06f..47e47f3eb 100644 --- a/beacon_node/eth2-libp2p/src/rpc/methods.rs +++ b/beacon_node/eth2-libp2p/src/rpc/methods.rs @@ -1,7 +1,7 @@ use ssz::{Decodable, DecodeError, Encodable, SszStream}; /// Available RPC methods types and ids. use ssz_derive::{Decode, Encode}; -use types::{BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; +use types::{Attestation, BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; #[derive(Debug)] /// Available Serenity Libp2p RPC methods @@ -97,6 +97,12 @@ impl RPCResponse { } } +#[derive(Debug, Clone)] +pub enum IncomingGossip { + Block(BlockGossip), + Attestation(AttestationGossip), +} + /* Request/Response data structures for RPC methods */ /// The HELLO request/response handshake message. @@ -236,3 +242,15 @@ pub struct BeaconChainStateResponse { /// The values corresponding the to the requested tree hashes. pub values: bool, //TBD - stubbed with encodeable bool } + +/// Gossipsub message providing notification of a new block. +#[derive(Encode, Decode, Clone, Debug, PartialEq)] +pub struct BlockGossip { + pub root: BlockRootSlot, +} + +/// Gossipsub message providing notification of a new attestation. +#[derive(Encode, Decode, Clone, Debug, PartialEq)] +pub struct AttestationGossip { + pub attestation: Attestation, +} diff --git a/beacon_node/eth2-libp2p/src/rpc/mod.rs b/beacon_node/eth2-libp2p/src/rpc/mod.rs index a1573ec93..925c36616 100644 --- a/beacon_node/eth2-libp2p/src/rpc/mod.rs +++ b/beacon_node/eth2-libp2p/src/rpc/mod.rs @@ -11,7 +11,7 @@ use libp2p::core::swarm::{ ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters, }; use libp2p::{Multiaddr, PeerId}; -pub use methods::{HelloMessage, RPCMethod, RPCRequest, RPCResponse}; +pub use methods::{HelloMessage, IncomingGossip, RPCMethod, RPCRequest, RPCResponse}; pub use protocol::{RPCEvent, RPCProtocol}; use slog::o; use std::marker::PhantomData; diff --git a/beacon_node/network/src/beacon_chain.rs b/beacon_node/network/src/beacon_chain.rs index cc54e8ae0..26cea0065 100644 --- a/beacon_node/network/src/beacon_chain.rs +++ b/beacon_node/network/src/beacon_chain.rs @@ -5,10 +5,10 @@ use beacon_chain::{ parking_lot::RwLockReadGuard, slot_clock::SlotClock, types::{BeaconState, ChainSpec}, - CheckPoint, + AggregationOutcome, CheckPoint, }; use eth2_libp2p::HelloMessage; -use types::{BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; +use types::{Attestation, BeaconBlock, BeaconBlockBody, BeaconBlockHeader, Epoch, Hash256, Slot}; pub use beacon_chain::{BeaconChainError, BlockProcessingOutcome}; @@ -37,6 +37,11 @@ pub trait BeaconChain: Send + Sync { fn process_block(&self, block: BeaconBlock) -> Result; + fn process_attestation( + &self, + attestation: Attestation, + ) -> Result; + fn get_block_roots( &self, start_slot: Slot, @@ -119,6 +124,18 @@ where self.process_block(block) } + fn process_attestation( + &self, + _attestation: Attestation, + ) -> Result { + // Awaiting a proper operations pool before we can import attestations. + // + // Returning a useless error for now. + // + // https://github.com/sigp/lighthouse/issues/281 + return Err(BeaconChainError::DBInconsistent("CANNOT PROCESS".into())); + } + fn get_block_roots( &self, start_slot: Slot, diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 57923b2c3..a788e83c9 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -4,7 +4,7 @@ use crate::service::{NetworkMessage, OutgoingMessage}; use crate::sync::SimpleSync; use crossbeam_channel::{unbounded as channel, Sender}; use eth2_libp2p::{ - rpc::{RPCRequest, RPCResponse}, + rpc::{IncomingGossip, RPCRequest, RPCResponse}, PeerId, RPCEvent, }; use futures::future; @@ -39,8 +39,8 @@ pub enum HandlerMessage { PeerDisconnected(PeerId), /// An RPC response/request has been received. RPC(PeerId, RPCEvent), - /// A block has been imported. - BlockImported(), //TODO: This comes from pub-sub - decide its contents + /// A gossip message has been received. + IncomingGossip(PeerId, IncomingGossip), } impl MessageHandler { @@ -90,6 +90,10 @@ impl MessageHandler { HandlerMessage::RPC(peer_id, rpc_event) => { self.handle_rpc_message(peer_id, rpc_event); } + // we have received an RPC message request/response + HandlerMessage::IncomingGossip(peer_id, gossip) => { + self.handle_gossip(peer_id, gossip); + } //TODO: Handle all messages _ => {} } @@ -186,6 +190,19 @@ impl MessageHandler { } }; } + + /// Handle RPC messages + fn handle_gossip(&mut self, peer_id: PeerId, gossip_message: IncomingGossip) { + match gossip_message { + IncomingGossip::Block(message) => { + self.sync + .on_block_gossip(peer_id, message, &mut self.network_context) + } + IncomingGossip::Attestation(message) => { + // + } + } + } } pub struct NetworkContext { diff --git a/beacon_node/network/src/sync/import_queue.rs b/beacon_node/network/src/sync/import_queue.rs index 6508af89e..17cbd2f12 100644 --- a/beacon_node/network/src/sync/import_queue.rs +++ b/beacon_node/network/src/sync/import_queue.rs @@ -104,7 +104,7 @@ impl ImportQueue { } /// Returns `true` if `self.chain` has not yet processed this block. - fn is_new_block(&self, block_root: &Hash256) -> bool { + pub fn is_new_block(&self, block_root: &Hash256) -> bool { self.chain .is_new_block_root(&block_root) .unwrap_or_else(|_| { diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 4ee349043..06ccbafd3 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -204,6 +204,8 @@ impl SimpleSync { self.known_peers.insert(peer_id.clone(), remote); } + // TODO: boot peer if finalization is wrong. + match remote_status { PeerStatus::OnDifferentChain => { info!( @@ -462,6 +464,62 @@ impl SimpleSync { self.process_import_queue(network); } + /// Process a gossip message declaring a new block. + pub fn on_block_gossip( + &mut self, + peer_id: PeerId, + msg: BlockGossip, + network: &mut NetworkContext, + ) { + debug!( + self.log, + "BlockGossip"; + "peer" => format!("{:?}", peer_id), + ); + // TODO: filter out messages that a prior to the finalized slot. + // + // TODO: if the block is a few more slots ahead, try to get all block roots from then until + // now. + // + // Note: only requests the new block -- will fail if we don't have its parents. + if self.import_queue.is_new_block(&msg.root.block_root) { + self.request_block_headers( + peer_id, + BeaconBlockHeadersRequest { + start_root: msg.root.block_root, + start_slot: msg.root.slot, + max_headers: 1, + skip_slots: 0, + }, + network, + ) + } + } + + /// Process a gossip message declaring a new attestation. + /// + /// Not currently implemented. + pub fn on_attestation_gossip( + &mut self, + peer_id: PeerId, + msg: AttestationGossip, + _network: &mut NetworkContext, + ) { + debug!( + self.log, + "AttestationGossip"; + "peer" => format!("{:?}", peer_id), + ); + + // Awaiting a proper operations pool before we can import attestations. + // + // https://github.com/sigp/lighthouse/issues/281 + match self.chain.process_attestation(msg.attestation) { + Ok(_) => panic!("Impossible, method not implemented."), + Err(_) => error!(self.log, "Attestation processing not implemented!"), + } + } + /// Iterate through the `import_queue` and process any complete blocks. /// /// If a block is successfully processed it is removed from the queue, otherwise it remains in From ebb9ced0a47331980ed74c42ffe4f638ff55979a Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 25 Mar 2019 15:30:46 +1100 Subject: [PATCH 22/24] Improve peer status handling --- beacon_node/network/src/message_handler.rs | 8 +- beacon_node/network/src/sync/simple_sync.rs | 108 ++++++++++++++------ 2 files changed, 81 insertions(+), 35 deletions(-) diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index a788e83c9..5b3fe1a63 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -4,7 +4,7 @@ use crate::service::{NetworkMessage, OutgoingMessage}; use crate::sync::SimpleSync; use crossbeam_channel::{unbounded as channel, Sender}; use eth2_libp2p::{ - rpc::{IncomingGossip, RPCRequest, RPCResponse}, + rpc::{methods::GoodbyeReason, IncomingGossip, RPCRequest, RPCResponse}, PeerId, RPCEvent, }; use futures::future; @@ -199,7 +199,8 @@ impl MessageHandler { .on_block_gossip(peer_id, message, &mut self.network_context) } IncomingGossip::Attestation(message) => { - // + self.sync + .on_attestation_gossip(peer_id, message, &mut self.network_context) } } } @@ -226,7 +227,8 @@ impl NetworkContext { } } - pub fn disconnect(&self, _peer_id: PeerId) { + pub fn disconnect(&mut self, peer_id: PeerId, reason: GoodbyeReason) { + self.send_rpc_request(peer_id, RPCRequest::Goodbye(reason)) // TODO: disconnect peers. } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 06ccbafd3..05c1a0430 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -27,9 +27,9 @@ pub struct PeerSyncInfo { } impl PeerSyncInfo { - /// Returns `true` if the peer is on the same chain as `other`. - fn is_on_same_chain(&self, other: Self) -> bool { - self.network_id == other.network_id + /// Returns `true` if the has a different network ID to `other`. + fn has_different_network_id_to(&self, other: Self) -> bool { + self.network_id != other.network_id } /// Returns `true` if the peer has a higher finalized epoch than `other`. @@ -41,19 +41,6 @@ impl PeerSyncInfo { fn has_higher_best_slot_than(&self, other: Self) -> bool { self.best_slot > other.best_slot } - - /// Returns the `PeerStatus` of `self` in relation to `other`. - pub fn status_compared_to(&self, other: Self) -> PeerStatus { - if self.has_higher_finalized_epoch_than(other) { - PeerStatus::HigherFinalizedEpoch - } else if !self.is_on_same_chain(other) { - PeerStatus::OnDifferentChain - } else if self.has_higher_best_slot_than(other) { - PeerStatus::HigherBestSlot - } else { - PeerStatus::NotInteresting - } - } } /// The status of a peers view on the chain, relative to some other view of the chain (presumably @@ -61,7 +48,9 @@ impl PeerSyncInfo { #[derive(PartialEq, Clone, Copy, Debug)] pub enum PeerStatus { /// The peer is on a completely different chain. - OnDifferentChain, + DifferentNetworkId, + /// The peer lists a finalized epoch for which we have a different root. + FinalizedEpochNotInChain, /// The peer has a higher finalized epoch. HigherFinalizedEpoch, /// The peer has a higher best slot. @@ -70,6 +59,18 @@ pub enum PeerStatus { NotInteresting, } +impl PeerStatus { + pub fn should_handshake(&self) -> bool { + match self { + PeerStatus::DifferentNetworkId => false, + PeerStatus::FinalizedEpochNotInChain => false, + PeerStatus::HigherFinalizedEpoch => true, + PeerStatus::HigherBestSlot => true, + PeerStatus::NotInteresting => true, + } + } +} + impl From for PeerSyncInfo { fn from(hello: HelloMessage) -> PeerSyncInfo { PeerSyncInfo { @@ -183,6 +184,51 @@ impl SimpleSync { self.process_hello(peer_id, hello, network); } + /// Returns a `PeerStatus` for some peer. + fn peer_status(&self, peer: PeerSyncInfo) -> PeerStatus { + let local = PeerSyncInfo::from(&self.chain); + + if peer.has_different_network_id_to(local) { + return PeerStatus::DifferentNetworkId; + } + + if local.has_higher_finalized_epoch_than(peer) { + let peer_finalized_slot = peer + .latest_finalized_epoch + .start_slot(self.chain.get_spec().slots_per_epoch); + + let local_roots = self.chain.get_block_roots(peer_finalized_slot, 1, 0); + + if let Ok(local_roots) = local_roots { + if let Some(local_root) = local_roots.get(0) { + if *local_root != peer.latest_finalized_root { + return PeerStatus::FinalizedEpochNotInChain; + } + } else { + error!( + self.log, + "Cannot get root for peer finalized slot."; + "error" => "empty roots" + ); + } + } else { + error!( + self.log, + "Cannot get root for peer finalized slot."; + "error" => format!("{:?}", local_roots) + ); + } + } + + if peer.has_higher_finalized_epoch_than(local) { + PeerStatus::HigherFinalizedEpoch + } else if peer.has_higher_best_slot_than(local) { + PeerStatus::HigherBestSlot + } else { + PeerStatus::NotInteresting + } + } + /// Process a `Hello` message, requesting new blocks if appropriate. /// /// Disconnects the peer if required. @@ -196,26 +242,22 @@ impl SimpleSync { let remote = PeerSyncInfo::from(hello); let local = PeerSyncInfo::from(&self.chain); - let remote_status = remote.status_compared_to(local); + let remote_status = self.peer_status(remote); - // network id must match - if remote_status != PeerStatus::OnDifferentChain { + if remote_status.should_handshake() { info!(self.log, "HandshakeSuccess"; "peer" => format!("{:?}", peer_id)); self.known_peers.insert(peer_id.clone(), remote); + } else { + info!( + self.log, "HandshakeFailure"; + "peer" => format!("{:?}", peer_id), + "reason" => "network_id" + ); + network.disconnect(peer_id.clone(), GoodbyeReason::IrreleventNetwork); } - // TODO: boot peer if finalization is wrong. - + // If required, send requests for blocks. match remote_status { - PeerStatus::OnDifferentChain => { - info!( - self.log, "Failure"; - "peer" => format!("{:?}", peer_id), - "reason" => "network_id" - ); - - network.disconnect(peer_id); - } PeerStatus::HigherFinalizedEpoch => { let start_slot = remote .latest_finalized_epoch @@ -243,6 +285,8 @@ impl SimpleSync { network, ); } + PeerStatus::FinalizedEpochNotInChain => {} + PeerStatus::DifferentNetworkId => {} PeerStatus::NotInteresting => {} } } @@ -541,7 +585,7 @@ impl SimpleSync { "sender_peer_id" => format!("{:?}", sender), "reason" => format!("{:?}", outcome), ); - network.disconnect(sender); + network.disconnect(sender, GoodbyeReason::Fault); } // If this results to true, the item will be removed from the queue. From 32a025bdf78d818939bfa047f3389246d23eb6d8 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 25 Mar 2019 16:48:44 +1100 Subject: [PATCH 23/24] Introduced `RequestId` newtype --- beacon_node/eth2-libp2p/src/rpc/mod.rs | 2 +- beacon_node/eth2-libp2p/src/rpc/protocol.rs | 58 +++++++++++++- beacon_node/network/src/message_handler.rs | 88 ++++++++++++--------- beacon_node/network/src/sync/simple_sync.rs | 12 ++- beacon_node/network/tests/tests.rs | 16 ++-- 5 files changed, 125 insertions(+), 51 deletions(-) diff --git a/beacon_node/eth2-libp2p/src/rpc/mod.rs b/beacon_node/eth2-libp2p/src/rpc/mod.rs index 925c36616..e04540416 100644 --- a/beacon_node/eth2-libp2p/src/rpc/mod.rs +++ b/beacon_node/eth2-libp2p/src/rpc/mod.rs @@ -12,7 +12,7 @@ use libp2p::core::swarm::{ }; use libp2p::{Multiaddr, PeerId}; pub use methods::{HelloMessage, IncomingGossip, RPCMethod, RPCRequest, RPCResponse}; -pub use protocol::{RPCEvent, RPCProtocol}; +pub use protocol::{RPCEvent, RPCProtocol, RequestId}; use slog::o; use std::marker::PhantomData; use tokio::io::{AsyncRead, AsyncWrite}; diff --git a/beacon_node/eth2-libp2p/src/rpc/protocol.rs b/beacon_node/eth2-libp2p/src/rpc/protocol.rs index b328dd0dd..2c8945cb8 100644 --- a/beacon_node/eth2-libp2p/src/rpc/protocol.rs +++ b/beacon_node/eth2-libp2p/src/rpc/protocol.rs @@ -1,6 +1,7 @@ use super::methods::*; use libp2p::core::{upgrade, InboundUpgrade, OutboundUpgrade, UpgradeInfo}; -use ssz::{ssz_encode, Decodable, Encodable, SszStream}; +use ssz::{ssz_encode, Decodable, DecodeError as SSZDecodeError, Encodable, SszStream}; +use std::hash::{Hash, Hasher}; use std::io; use std::iter; use tokio::io::{AsyncRead, AsyncWrite}; @@ -29,16 +30,65 @@ impl Default for RPCProtocol { } } +/// A monotonic counter for ordering `RPCRequest`s. +#[derive(Debug, Clone, PartialEq, Default)] +pub struct RequestId(u64); + +impl RequestId { + /// Increment the request id. + pub fn increment(&mut self) { + self.0 += 1 + } + + /// Return the previous id. + pub fn previous(&self) -> Self { + Self(self.0 - 1) + } +} + +impl Eq for RequestId {} + +impl Hash for RequestId { + fn hash(&self, state: &mut H) { + self.0.hash(state); + } +} + +impl From for RequestId { + fn from(x: u64) -> RequestId { + RequestId(x) + } +} + +impl Into for RequestId { + fn into(self) -> u64 { + self.0 + } +} + +impl Encodable for RequestId { + fn ssz_append(&self, s: &mut SszStream) { + self.0.ssz_append(s); + } +} + +impl Decodable for RequestId { + fn ssz_decode(bytes: &[u8], index: usize) -> Result<(Self, usize), SSZDecodeError> { + let (id, index) = u64::ssz_decode(bytes, index)?; + Ok((Self::from(id), index)) + } +} + /// The RPC types which are sent/received in this protocol. #[derive(Debug, Clone)] pub enum RPCEvent { Request { - id: u64, + id: RequestId, method_id: u16, body: RPCRequest, }, Response { - id: u64, + id: RequestId, method_id: u16, //TODO: Remove and process decoding upstream result: RPCResponse, }, @@ -72,7 +122,7 @@ fn decode(packet: Vec) -> Result { // decode the header of the rpc // request/response let (request, index) = bool::ssz_decode(&packet, 0)?; - let (id, index) = u64::ssz_decode(&packet, index)?; + let (id, index) = RequestId::ssz_decode(&packet, index)?; let (method_id, index) = u16::ssz_decode(&packet, index)?; if request { diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 5b3fe1a63..12fb2fa6e 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -4,7 +4,7 @@ use crate::service::{NetworkMessage, OutgoingMessage}; use crate::sync::SimpleSync; use crossbeam_channel::{unbounded as channel, Sender}; use eth2_libp2p::{ - rpc::{methods::GoodbyeReason, IncomingGossip, RPCRequest, RPCResponse}, + rpc::{methods::GoodbyeReason, IncomingGossip, RPCRequest, RPCResponse, RequestId}, PeerId, RPCEvent, }; use futures::future; @@ -111,25 +111,31 @@ impl MessageHandler { } /// A new RPC request has been received from the network. - fn handle_rpc_request(&mut self, peer_id: PeerId, _id: u64, request: RPCRequest) { + fn handle_rpc_request(&mut self, peer_id: PeerId, request_id: RequestId, request: RPCRequest) { // TODO: process the `id`. match request { - RPCRequest::Hello(hello_message) => { - self.sync - .on_hello_request(peer_id, hello_message, &mut self.network_context) - } + RPCRequest::Hello(hello_message) => self.sync.on_hello_request( + peer_id, + request_id, + hello_message, + &mut self.network_context, + ), RPCRequest::Goodbye(goodbye_reason) => self.sync.on_goodbye(peer_id, goodbye_reason), - RPCRequest::BeaconBlockRoots(request) => { - self.sync - .on_beacon_block_roots_request(peer_id, request, &mut self.network_context) - } + RPCRequest::BeaconBlockRoots(request) => self.sync.on_beacon_block_roots_request( + peer_id, + request_id, + request, + &mut self.network_context, + ), RPCRequest::BeaconBlockHeaders(request) => self.sync.on_beacon_block_headers_request( peer_id, + request_id, request, &mut self.network_context, ), RPCRequest::BeaconBlockBodies(request) => self.sync.on_beacon_block_bodies_request( peer_id, + request_id, request, &mut self.network_context, ), @@ -143,17 +149,23 @@ impl MessageHandler { /// 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, id: u64, response: RPCResponse) { - // if response id is related to a request, ignore (likely RPC timeout) + fn handle_rpc_response(&mut self, peer_id: PeerId, id: RequestId, response: RPCResponse) { + // if response id is not related to a request, ignore (likely RPC timeout) if self .network_context - .requests - .remove(&(peer_id.clone(), id)) + .outstanding_outgoing_request_ids + .remove(&(peer_id.clone(), id.clone())) .is_none() { - debug!(self.log, "Unrecognised response from peer: {:?}", peer_id); + warn!( + self.log, + "Unknown ResponseId for incoming RPCRequest"; + "peer" => format!("{:?}", peer_id), + "request_id" => format!("{:?}", id) + ); return; } + match response { RPCResponse::Hello(hello_message) => { self.sync @@ -210,9 +222,9 @@ pub struct NetworkContext { /// The network channel to relay messages to the Network service. network_send: crossbeam_channel::Sender, /// A mapping of peers and the RPC id we have sent an RPC request to. - requests: HashMap<(PeerId, u64), Instant>, - /// A counter of request id for each peer. - request_ids: HashMap, + outstanding_outgoing_request_ids: HashMap<(PeerId, RequestId), Instant>, + /// Stores the next `RequestId` we should include on an outgoing `RPCRequest` to a `PeerId`. + outgoing_request_ids: HashMap, /// The `MessageHandler` logger. log: slog::Logger, } @@ -221,8 +233,8 @@ impl NetworkContext { pub fn new(network_send: crossbeam_channel::Sender, log: slog::Logger) -> Self { Self { network_send, - requests: HashMap::new(), - request_ids: HashMap::new(), + outstanding_outgoing_request_ids: HashMap::new(), + outgoing_request_ids: HashMap::new(), log, } } @@ -234,6 +246,10 @@ impl NetworkContext { pub fn send_rpc_request(&mut self, peer_id: PeerId, rpc_request: RPCRequest) { let id = self.generate_request_id(&peer_id); + + self.outstanding_outgoing_request_ids + .insert((peer_id.clone(), id.clone()), Instant::now()); + self.send_rpc_event( peer_id, RPCEvent::Request { @@ -244,12 +260,16 @@ impl NetworkContext { ); } - pub fn send_rpc_response(&mut self, peer_id: PeerId, rpc_response: RPCResponse) { - let id = self.generate_request_id(&peer_id); + pub fn send_rpc_response( + &mut self, + peer_id: PeerId, + request_id: RequestId, + rpc_response: RPCResponse, + ) { self.send_rpc_event( peer_id, RPCEvent::Response { - id, + id: request_id, method_id: rpc_response.method_id(), result: rpc_response, }, @@ -272,18 +292,14 @@ impl NetworkContext { // } - /// Generates a new request id for a peer. - fn generate_request_id(&mut self, peer_id: &PeerId) -> u64 { - // generate a unique id for the peer - let id = { - let borrowed_id = self.request_ids.entry(peer_id.clone()).or_insert_with(|| 0); - let id = borrowed_id.clone(); - //increment the counter - *borrowed_id += 1; - id - }; - // register RPC request - self.requests.insert((peer_id.clone(), id), Instant::now()); - id + /// Returns the next `RequestId` for sending an `RPCRequest` to the `peer_id`. + fn generate_request_id(&mut self, peer_id: &PeerId) -> RequestId { + let next_id = self + .outgoing_request_ids + .entry(peer_id.clone()) + .and_modify(|id| id.increment()) + .or_insert_with(|| RequestId::from(1)); + + next_id.previous() } } diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 05c1a0430..37c2c4c26 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -2,7 +2,7 @@ use super::import_queue::ImportQueue; use crate::beacon_chain::BeaconChain; use crate::message_handler::NetworkContext; use eth2_libp2p::rpc::methods::*; -use eth2_libp2p::rpc::{RPCRequest, RPCResponse}; +use eth2_libp2p::rpc::{RPCRequest, RPCResponse, RequestId}; use eth2_libp2p::PeerId; use slog::{debug, error, info, o, warn}; use std::collections::HashMap; @@ -157,6 +157,7 @@ impl SimpleSync { pub fn on_hello_request( &mut self, peer_id: PeerId, + request_id: RequestId, hello: HelloMessage, network: &mut NetworkContext, ) { @@ -165,6 +166,7 @@ impl SimpleSync { // Say hello back. network.send_rpc_response( peer_id.clone(), + request_id, RPCResponse::Hello(self.chain.hello_message()), ); @@ -256,7 +258,7 @@ impl SimpleSync { network.disconnect(peer_id.clone(), GoodbyeReason::IrreleventNetwork); } - // If required, send requests for blocks. + // If required, send additional requests. match remote_status { PeerStatus::HigherFinalizedEpoch => { let start_slot = remote @@ -295,6 +297,7 @@ impl SimpleSync { pub fn on_beacon_block_roots_request( &mut self, peer_id: PeerId, + request_id: RequestId, req: BeaconBlockRootsRequest, network: &mut NetworkContext, ) { @@ -333,6 +336,7 @@ impl SimpleSync { network.send_rpc_response( peer_id, + request_id, RPCResponse::BeaconBlockRoots(BeaconBlockRootsResponse { roots }), ) } @@ -385,6 +389,7 @@ impl SimpleSync { pub fn on_beacon_block_headers_request( &mut self, peer_id: PeerId, + request_id: RequestId, req: BeaconBlockHeadersRequest, network: &mut NetworkContext, ) { @@ -415,6 +420,7 @@ impl SimpleSync { network.send_rpc_response( peer_id, + request_id, RPCResponse::BeaconBlockHeaders(BeaconBlockHeadersResponse { headers }), ) } @@ -454,6 +460,7 @@ impl SimpleSync { pub fn on_beacon_block_bodies_request( &mut self, peer_id: PeerId, + request_id: RequestId, req: BeaconBlockBodiesRequest, network: &mut NetworkContext, ) { @@ -480,6 +487,7 @@ impl SimpleSync { network.send_rpc_response( peer_id, + request_id, RPCResponse::BeaconBlockBodies(BeaconBlockBodiesResponse { block_bodies }), ) } diff --git a/beacon_node/network/tests/tests.rs b/beacon_node/network/tests/tests.rs index 110450dc9..9cead1b55 100644 --- a/beacon_node/network/tests/tests.rs +++ b/beacon_node/network/tests/tests.rs @@ -1,6 +1,6 @@ use crossbeam_channel::{unbounded, Receiver, RecvTimeoutError, Sender}; use eth2_libp2p::rpc::methods::*; -use eth2_libp2p::rpc::{RPCMethod, RPCRequest, RPCResponse}; +use eth2_libp2p::rpc::{RPCMethod, RPCRequest, RPCResponse, RequestId}; use eth2_libp2p::{PeerId, RPCEvent}; use network::beacon_chain::BeaconChain as NetworkBeaconChain; use network::message_handler::{HandlerMessage, MessageHandler}; @@ -82,8 +82,8 @@ impl SyncNode { let network_message = self.recv().expect("Timeout on tee"); let handler_message = match network_message.clone() { - NetworkMessage::Send(peer_id, OutgoingMessage::RPC(event)) => { - HandlerMessage::RPC(peer_id, event) + NetworkMessage::Send(_to_peer_id, OutgoingMessage::RPC(event)) => { + HandlerMessage::RPC(self.peer_id.clone(), event) } _ => panic!("tee cannot parse {:?}", network_message), }; @@ -265,7 +265,7 @@ fn get_logger() -> slog::Logger { pub struct SyncMaster { harness: BeaconChainHarness, peer_id: PeerId, - response_ids: Vec, + response_ids: Vec, } impl SyncMaster { @@ -276,7 +276,7 @@ impl SyncMaster { ) -> Self { let harness = BeaconChainHarness::from_beacon_state_builder(state_builder, spec.clone()); let peer_id = PeerId::random(); - let response_ids = vec![0; node_count]; + let response_ids = vec![RequestId::from(0); node_count]; Self { harness, @@ -285,9 +285,9 @@ impl SyncMaster { } } - pub fn response_id(&mut self, node: &SyncNode) -> u64 { - let id = self.response_ids[node.id]; - self.response_ids[node.id] += 1; + pub fn response_id(&mut self, node: &SyncNode) -> RequestId { + let id = self.response_ids[node.id].clone(); + self.response_ids[node.id].increment(); id } From 098e63ac327ea8a7b64df2d4fbdbe5f794f37671 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 25 Mar 2019 18:59:50 +1100 Subject: [PATCH 24/24] Move gossip structs into behaviour --- beacon_node/eth2-libp2p/src/behaviour.rs | 21 +++++++++++++++++++++ beacon_node/eth2-libp2p/src/rpc/methods.rs | 18 ------------------ beacon_node/eth2-libp2p/src/rpc/mod.rs | 2 +- beacon_node/network/src/message_handler.rs | 3 ++- beacon_node/network/src/sync/simple_sync.rs | 1 + 5 files changed, 25 insertions(+), 20 deletions(-) diff --git a/beacon_node/eth2-libp2p/src/behaviour.rs b/beacon_node/eth2-libp2p/src/behaviour.rs index 458b32cf9..3d5b94353 100644 --- a/beacon_node/eth2-libp2p/src/behaviour.rs +++ b/beacon_node/eth2-libp2p/src/behaviour.rs @@ -1,3 +1,4 @@ +use crate::rpc::methods::BlockRootSlot; use crate::rpc::{RPCEvent, RPCMessage, Rpc}; use crate::NetworkConfig; use futures::prelude::*; @@ -13,6 +14,8 @@ use libp2p::{ NetworkBehaviour, PeerId, }; use slog::{debug, o}; +use ssz_derive::{Decode, Encode}; +use types::Attestation; use types::Topic; /// Builds the network behaviour for the libp2p Swarm. @@ -154,3 +157,21 @@ pub enum BehaviourEvent { // TODO: This is a stub at the moment Message(String), } + +#[derive(Debug, Clone)] +pub enum IncomingGossip { + Block(BlockGossip), + Attestation(AttestationGossip), +} + +/// Gossipsub message providing notification of a new block. +#[derive(Encode, Decode, Clone, Debug, PartialEq)] +pub struct BlockGossip { + pub root: BlockRootSlot, +} + +/// Gossipsub message providing notification of a new attestation. +#[derive(Encode, Decode, Clone, Debug, PartialEq)] +pub struct AttestationGossip { + pub attestation: Attestation, +} diff --git a/beacon_node/eth2-libp2p/src/rpc/methods.rs b/beacon_node/eth2-libp2p/src/rpc/methods.rs index 47e47f3eb..ad3233be7 100644 --- a/beacon_node/eth2-libp2p/src/rpc/methods.rs +++ b/beacon_node/eth2-libp2p/src/rpc/methods.rs @@ -97,12 +97,6 @@ impl RPCResponse { } } -#[derive(Debug, Clone)] -pub enum IncomingGossip { - Block(BlockGossip), - Attestation(AttestationGossip), -} - /* Request/Response data structures for RPC methods */ /// The HELLO request/response handshake message. @@ -242,15 +236,3 @@ pub struct BeaconChainStateResponse { /// The values corresponding the to the requested tree hashes. pub values: bool, //TBD - stubbed with encodeable bool } - -/// Gossipsub message providing notification of a new block. -#[derive(Encode, Decode, Clone, Debug, PartialEq)] -pub struct BlockGossip { - pub root: BlockRootSlot, -} - -/// Gossipsub message providing notification of a new attestation. -#[derive(Encode, Decode, Clone, Debug, PartialEq)] -pub struct AttestationGossip { - pub attestation: Attestation, -} diff --git a/beacon_node/eth2-libp2p/src/rpc/mod.rs b/beacon_node/eth2-libp2p/src/rpc/mod.rs index e04540416..08573aa52 100644 --- a/beacon_node/eth2-libp2p/src/rpc/mod.rs +++ b/beacon_node/eth2-libp2p/src/rpc/mod.rs @@ -11,7 +11,7 @@ use libp2p::core::swarm::{ ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters, }; use libp2p::{Multiaddr, PeerId}; -pub use methods::{HelloMessage, IncomingGossip, RPCMethod, RPCRequest, RPCResponse}; +pub use methods::{HelloMessage, RPCMethod, RPCRequest, RPCResponse}; pub use protocol::{RPCEvent, RPCProtocol, RequestId}; use slog::o; use std::marker::PhantomData; diff --git a/beacon_node/network/src/message_handler.rs b/beacon_node/network/src/message_handler.rs index 12fb2fa6e..dcfee96a0 100644 --- a/beacon_node/network/src/message_handler.rs +++ b/beacon_node/network/src/message_handler.rs @@ -4,7 +4,8 @@ use crate::service::{NetworkMessage, OutgoingMessage}; use crate::sync::SimpleSync; use crossbeam_channel::{unbounded as channel, Sender}; use eth2_libp2p::{ - rpc::{methods::GoodbyeReason, IncomingGossip, RPCRequest, RPCResponse, RequestId}, + behaviour::IncomingGossip, + rpc::{methods::GoodbyeReason, RPCRequest, RPCResponse, RequestId}, PeerId, RPCEvent, }; use futures::future; diff --git a/beacon_node/network/src/sync/simple_sync.rs b/beacon_node/network/src/sync/simple_sync.rs index 37c2c4c26..4c08a6871 100644 --- a/beacon_node/network/src/sync/simple_sync.rs +++ b/beacon_node/network/src/sync/simple_sync.rs @@ -1,6 +1,7 @@ use super::import_queue::ImportQueue; use crate::beacon_chain::BeaconChain; use crate::message_handler::NetworkContext; +use eth2_libp2p::behaviour::{AttestationGossip, BlockGossip}; use eth2_libp2p::rpc::methods::*; use eth2_libp2p::rpc::{RPCRequest, RPCResponse, RequestId}; use eth2_libp2p::PeerId;