From 3e4d96021b9952bd454e68db585695f8fb482050 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sat, 15 Aug 2020 19:13:14 +1000 Subject: [PATCH 01/20] Move gossip block processing to workers --- .../network/src/router/gossip_processor.rs | 201 +++++++++++++++++- beacon_node/network/src/router/mod.rs | 20 +- beacon_node/network/src/router/processor.rs | 92 ++------ 3 files changed, 210 insertions(+), 103 deletions(-) diff --git a/beacon_node/network/src/router/gossip_processor.rs b/beacon_node/network/src/router/gossip_processor.rs index f9f229ba77d..08ebd211638 100644 --- a/beacon_node/network/src/router/gossip_processor.rs +++ b/beacon_node/network/src/router/gossip_processor.rs @@ -38,16 +38,17 @@ use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{ attestation_verification::Error as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes, - ForkChoiceError, + BlockError, ForkChoiceError, }; use environment::TaskExecutor; use eth2_libp2p::{MessageId, NetworkGlobals, PeerId}; -use slog::{crit, debug, error, trace, warn, Logger}; +use slog::{crit, debug, error, info, trace, warn, Logger}; +use ssz::Encode; use std::collections::VecDeque; use std::sync::Arc; use std::time::{Duration, Instant}; use tokio::sync::mpsc; -use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SubnetId}; +use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId}; /// The maximum size of the channel for work events to the `GossipProcessor`. /// @@ -68,6 +69,10 @@ const MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN: usize = 16_384; /// start dropping them. const MAX_AGGREGATED_ATTESTATION_QUEUE_LEN: usize = 1_024; +/// The maximum number of queued `SignedBeaconBlock` objects that will be stored before we start +/// dropping them. +const MAX_BLOCK_QUEUE_LEN: usize = 1_024; + /// The name of the manager tokio task. const MANAGER_TASK_NAME: &str = "beacon_gossip_processor_manager"; /// The name of the worker tokio tasks. @@ -83,6 +88,40 @@ struct QueueItem { item: T, } +/// A simple first-in-first-out queue with a maximum length. +struct FifoQueue { + queue: VecDeque>, + max_length: usize, +} + +impl FifoQueue { + /// Create a new, empty queue with the given length. + pub fn new(max_length: usize) -> Self { + Self { + queue: VecDeque::default(), + max_length, + } + } + + /// Add a new item to the queue. + pub fn push(&mut self, item: QueueItem, log: &Logger) { + if self.queue.len() == self.max_length { + error!( + log, + "Gossip block queue full"; + "msg" => "the system has insufficient resources for load", + "queue_len" => self.max_length, + ) + } + self.queue.push_back(item); + } + + /// Remove the next item from the queue. + pub fn pop(&mut self) -> Option> { + self.queue.pop_front() + } +} + /// A simple last-in-first-out queue with a maximum length. struct LifoQueue { queue: VecDeque>, @@ -158,6 +197,19 @@ impl WorkEvent { work: Work::Aggregate(Box::new(aggregate)), } } + + /// Create a new `Work` event for some block. + pub fn beacon_block( + message_id: MessageId, + peer_id: PeerId, + block: Box>, + ) -> Self { + Self { + message_id, + peer_id, + work: Work::Block(block), + } + } } /// A consensus message from gossip which requires processing. @@ -165,6 +217,7 @@ impl WorkEvent { pub enum Work { Attestation(Box<(Attestation, SubnetId, bool)>), Aggregate(Box>), + Block(Box>), } /// Provides de-bounce functionality for logging. @@ -221,6 +274,8 @@ impl GossipProcessor { let mut attestation_queue = LifoQueue::new(MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN); let mut attestation_debounce = TimeLatch::default(); + let mut block_queue = FifoQueue::new(MAX_BLOCK_QUEUE_LEN); + let executor = self.executor.clone(); // The manager future will run on the non-blocking executor and delegate tasks to worker @@ -278,10 +333,19 @@ impl GossipProcessor { match work_event { // There is no new work event, but we are able to spawn a new worker. None if can_spawn => { - // Check the aggregates, *then* the unaggregates since we assume that - // aggregates are more valuable to local validators and effectively - // give us more information with less signature verification time. - if let Some(item) = aggregate_queue.pop() { + // Always check blocks first, since they might be required to process an + // attestation. After blocks, heck the aggregates, *then* the unaggregates + // since we assume that aggregates are more valuable to local validators + // and effectively give us more information with less signature + // verification time. + if let Some(item) = block_queue.pop() { + self.spawn_worker( + idle_tx.clone(), + item.message_id, + item.peer_id, + Work::Block(item.item), + ); + } else if let Some(item) = aggregate_queue.pop() { self.spawn_worker( idle_tx.clone(), item.message_id, @@ -340,6 +404,17 @@ impl GossipProcessor { peer_id, item: aggregate, }), + Work::Block(_) if can_spawn => { + self.spawn_worker(idle_tx.clone(), message_id, peer_id, work) + } + Work::Block(block) => block_queue.push( + QueueItem { + message_id, + peer_id, + item: block, + }, + &self.log, + ), }, } @@ -555,6 +630,105 @@ impl GossipProcessor { ) } } + /* + * Beacon block verification. + */ + Work::Block(boxed_block) => { + let verified_block = + match chain.verify_block_for_gossip(*boxed_block) { + Ok(verified_block) => { + info!( + log, + "New block received"; + "slot" => verified_block.block.slot(), + "hash" => verified_block.block_root.to_string() + ); + propagate_gossip_message( + network_tx, + message_id, + peer_id.clone(), + &log, + ); + verified_block + } + Err(BlockError::ParentUnknown(block)) => { + send_sync_message( + sync_tx, + SyncMessage::UnknownBlock(peer_id, block), + &log, + ); + return; + } + Err(e) => { + warn!( + log, + "Could not verify block for gossip"; + "error" => format!("{:?}", e) + ); + return; + } + }; + + + let block = Box::new(verified_block.block.clone()); + match chain.process_block(verified_block) { + Ok(_block_root) => { + trace!( + log, + "Gossipsub block processed"; + "peer_id" => peer_id.to_string() + ); + + // TODO: It would be better if we can run this _after_ we publish the block to + // reduce block propagation latency. + // + // The `MessageHandler` would be the place to put this, however it doesn't seem + // to have a reference to the `BeaconChain`. I will leave this for future + // works. + match chain.fork_choice() { + Ok(()) => trace!( + log, + "Fork choice success"; + "location" => "block gossip" + ), + Err(e) => error!( + log, + "Fork choice failed"; + "error" => format!("{:?}", e), + "location" => "block gossip" + ), + } + } + Err(BlockError::ParentUnknown { .. }) => { + // Inform the sync manager to find parents for this block + // This should not occur. It should be checked by `should_forward_block` + error!( + log, + "Block with unknown parent attempted to be processed"; + "peer_id" => peer_id.to_string() + ); + send_sync_message( + sync_tx, + SyncMessage::UnknownBlock(peer_id, block), + &log, + ); + } + other => { + warn!( + log, + "Invalid gossip beacon block"; + "outcome" => format!("{:?}", other), + "block root" => format!("{}", block.canonical_root()), + "block slot" => block.slot() + ); + trace!( + log, + "Invalid gossip beacon block ssz"; + "ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())), + ); + } + } + } }; }; handler(); @@ -596,6 +770,19 @@ fn propagate_gossip_message( }); } +/// Send a message to `sync_tx`. +/// +/// Creates a log if there is an interal error. +fn send_sync_message( + sync_tx: mpsc::UnboundedSender>, + message: SyncMessage, + log: &Logger, +) { + sync_tx + .send(message) + .unwrap_or_else(|_| error!(log, "Could not send message to the sync service")); +} + /// Handle an error whilst verifying an `Attestation` or `SignedAggregateAndProof` from the /// network. pub fn handle_attestation_verification_failure( diff --git a/beacon_node/network/src/router/mod.rs b/beacon_node/network/src/router/mod.rs index bcb02fbbee2..6359d05ea44 100644 --- a/beacon_node/network/src/router/mod.rs +++ b/beacon_node/network/src/router/mod.rs @@ -10,14 +10,14 @@ pub mod processor; use crate::error; use crate::service::NetworkMessage; -use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError}; +use beacon_chain::{BeaconChain, BeaconChainTypes}; use eth2_libp2p::{ rpc::{RPCError, RequestId}, MessageId, NetworkGlobals, PeerId, PeerRequestId, PubsubMessage, Request, Response, }; use futures::prelude::*; use processor::Processor; -use slog::{debug, info, o, trace, warn}; +use slog::{debug, o, trace, warn}; use std::sync::Arc; use tokio::sync::mpsc; use types::EthSpec; @@ -229,21 +229,7 @@ impl Router { ); } PubsubMessage::BeaconBlock(block) => { - match self.processor.should_forward_block(block) { - Ok(verified_block) => { - info!(self.log, "New block received"; "slot" => verified_block.block.slot(), "hash" => verified_block.block_root.to_string()); - self.propagate_message(id, peer_id.clone()); - self.processor.on_block_gossip(peer_id, verified_block); - } - Err(BlockError::ParentUnknown(block)) => { - self.processor.on_unknown_parent(peer_id, block); - } - Err(e) => { - // performing a parent lookup - warn!(self.log, "Could not verify block for gossip"; - "error" => format!("{:?}", e)); - } - } + self.processor.on_block_gossip(id, peer_id, block); } PubsubMessage::VoluntaryExit(exit) => { debug!(self.log, "Received a voluntary exit"; "peer_id" => format!("{}", peer_id)); diff --git a/beacon_node/network/src/router/processor.rs b/beacon_node/network/src/router/processor.rs index 52f97fff81e..ebefed94547 100644 --- a/beacon_node/network/src/router/processor.rs +++ b/beacon_node/network/src/router/processor.rs @@ -1,17 +1,13 @@ use super::gossip_processor::{GossipProcessor, WorkEvent as GossipWorkEvent}; use crate::service::NetworkMessage; use crate::sync::{PeerSyncInfo, SyncMessage}; -use beacon_chain::{ - observed_operations::ObservationOutcome, BeaconChain, BeaconChainTypes, BlockError, - GossipVerifiedBlock, -}; +use beacon_chain::{observed_operations::ObservationOutcome, BeaconChain, BeaconChainTypes}; use eth2_libp2p::rpc::*; use eth2_libp2p::{ MessageId, NetworkGlobals, PeerAction, PeerId, PeerRequestId, Request, Response, }; use itertools::process_results; use slog::{debug, error, o, trace, warn}; -use ssz::Encode; use state_processing::SigVerifiedOp; use std::cmp; use std::sync::Arc; @@ -513,23 +509,6 @@ impl Processor { } } - /// Template function to be called on a block to determine if the block should be propagated - /// across the network. - pub fn should_forward_block( - &mut self, - block: Box>, - ) -> Result, BlockError> { - self.chain.verify_block_for_gossip(*block) - } - - pub fn on_unknown_parent( - &mut self, - peer_id: PeerId, - block: Box>, - ) { - self.send_to_sync(SyncMessage::UnknownBlock(peer_id, block)); - } - /// Process a gossip message declaring a new block. /// /// Attempts to apply to block to the beacon chain. May queue the block for later processing. @@ -537,65 +516,20 @@ impl Processor { /// Returns a `bool` which, if `true`, indicates we should forward the block to our peers. pub fn on_block_gossip( &mut self, + message_id: MessageId, peer_id: PeerId, - verified_block: GossipVerifiedBlock, - ) -> bool { - let block = Box::new(verified_block.block.clone()); - match self.chain.process_block(verified_block) { - Ok(_block_root) => { - trace!( - self.log, - "Gossipsub block processed"; - "peer_id" => peer_id.to_string() - ); - - // TODO: It would be better if we can run this _after_ we publish the block to - // reduce block propagation latency. - // - // The `MessageHandler` would be the place to put this, however it doesn't seem - // to have a reference to the `BeaconChain`. I will leave this for future - // works. - match self.chain.fork_choice() { - Ok(()) => trace!( - self.log, - "Fork choice success"; - "location" => "block gossip" - ), - Err(e) => error!( - self.log, - "Fork choice failed"; - "error" => format!("{:?}", e), - "location" => "block gossip" - ), - } - } - Err(BlockError::ParentUnknown { .. }) => { - // Inform the sync manager to find parents for this block - // This should not occur. It should be checked by `should_forward_block` + block: Box>, + ) { + self.gossip_processor_send + .try_send(GossipWorkEvent::beacon_block(message_id, peer_id, block)) + .unwrap_or_else(|e| { error!( - self.log, - "Block with unknown parent attempted to be processed"; - "peer_id" => peer_id.to_string() - ); - self.send_to_sync(SyncMessage::UnknownBlock(peer_id, block)); - } - other => { - warn!( - self.log, - "Invalid gossip beacon block"; - "outcome" => format!("{:?}", other), - "block root" => format!("{}", block.canonical_root()), - "block slot" => block.slot() - ); - trace!( - self.log, - "Invalid gossip beacon block ssz"; - "ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())), - ); - } - } - // TODO: Update with correct block gossip checking - true + &self.log, + "Unable to send to gossip processor"; + "type" => "block gossip", + "error" => e.to_string(), + ) + }) } pub fn on_unaggregated_attestation_gossip( From 9427ea8f28bce9b0b4b22ab54f84858e1d1a9d5b Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sat, 15 Aug 2020 19:20:46 +1000 Subject: [PATCH 02/20] Add metrics --- beacon_node/network/src/metrics.rs | 16 ++++++++++++++++ .../network/src/router/gossip_processor.rs | 19 +++++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index b0039763c3d..de2bd302540 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -77,6 +77,22 @@ lazy_static! { "gossip_processor_worker_time", "Time taken for a worker to fully process some parcel of work." ); + pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( + "gossip_processor_gossip_block_queue_total", + "Count of unagg. attestations waiting to be processed." + ); + pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_WORKER_TIME: Result = try_create_histogram( + "gossip_processor_gossip_block_worker_time", + "Time taken for a worker to fully process an gossip block." + ); + pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( + "gossip_processor_gossip_block_verified_total", + "Total number of gossip blocks verified for gossip." + ); + pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( + "gossip_processor_gossip_block_imported_total", + "Total number of gossip blocks imported to fork choice, etc." + ); pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( "gossip_processor_unaggregated_attestation_queue_total", "Count of unagg. attestations waiting to be processed." diff --git a/beacon_node/network/src/router/gossip_processor.rs b/beacon_node/network/src/router/gossip_processor.rs index 08ebd211638..4c786381746 100644 --- a/beacon_node/network/src/router/gossip_processor.rs +++ b/beacon_node/network/src/router/gossip_processor.rs @@ -120,6 +120,11 @@ impl FifoQueue { pub fn pop(&mut self) -> Option> { self.queue.pop_front() } + + /// Returns the current length of the queue. + pub fn len(&self) -> usize { + self.queue.len() + } } /// A simple last-in-first-out queue with a maximum length. @@ -430,6 +435,10 @@ impl GossipProcessor { &metrics::GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL, aggregate_queue.len() as i64, ); + metrics::set_gauge( + &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_QUEUE_TOTAL, + block_queue.len() as i64, + ); if aggregate_queue.is_full() && aggregate_debounce.elapsed() { error!( @@ -634,6 +643,13 @@ impl GossipProcessor { * Beacon block verification. */ Work::Block(boxed_block) => { + let _block_timer = metrics::start_timer( + &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_WORKER_TIME, + ); + metrics::inc_counter( + &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_VERIFIED_TOTAL, + ); + let verified_block = match chain.verify_block_for_gossip(*boxed_block) { Ok(verified_block) => { @@ -669,6 +685,9 @@ impl GossipProcessor { } }; + metrics::inc_counter( + &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_IMPORTED_TOTAL, + ); let block = Box::new(verified_block.block.clone()); match chain.process_block(verified_block) { From b69becde82c3478117c528174fe6ca5f4f9e65e1 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sat, 15 Aug 2020 19:47:52 +1000 Subject: [PATCH 03/20] Fix metrics names --- beacon_node/network/src/metrics.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index de2bd302540..7679a9804fc 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -78,19 +78,19 @@ lazy_static! { "Time taken for a worker to fully process some parcel of work." ); pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_gossip_block_queue_total", + "gossip_processor_beacon_block_queue_total", "Count of unagg. attestations waiting to be processed." ); pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_gossip_block_worker_time", + "gossip_processor_beacon_block_worker_time", "Time taken for a worker to fully process an gossip block." ); pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( - "gossip_processor_gossip_block_verified_total", + "gossip_processor_beacon_block_verified_total", "Total number of gossip blocks verified for gossip." ); pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_gossip_block_imported_total", + "gossip_processor_beacon_block_imported_total", "Total number of gossip blocks imported to fork choice, etc." ); pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( From e3259052743165f976cee896bde7d619f16d8070 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 16 Aug 2020 11:07:45 +1000 Subject: [PATCH 04/20] Use async functions for sync processing --- Cargo.lock | 35 +++++++++++-- beacon_node/network/src/sync/manager.rs | 68 ++++++++++++++++++++++--- lighthouse/environment/src/executor.rs | 2 +- 3 files changed, 93 insertions(+), 12 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7722ecfb1ad..0dfaa9e931a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1195,9 +1195,9 @@ checksum = "212d0f5754cb6769937f4501cc0e67f4f4483c8d2c3e1e922ee9edbe4ab4c7c0" [[package]] name = "discv5" -version = "0.1.0-alpha.8" +version = "0.1.0-alpha.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90782d49541b01f9b7e34e6af5d80d01396bf7b1a81505a0035da224134b8d73" +checksum = "98bc715508160877f74d828b94238c156c50f0ca80f51271bca9a855be94c488" dependencies = [ "arrayvec", "digest 0.8.1", @@ -1213,10 +1213,10 @@ dependencies = [ "lru_time_cache", "multihash", "net2", - "openssl", "parking_lot 0.11.0", "rand 0.7.3", "rlp", + "rust-crypto", "sha2 0.8.2", "smallvec 1.4.1", "tokio 0.2.22", @@ -4020,6 +4020,16 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "def50a86306165861203e7f84ecffbbdfdea79f0e51039b33de1e952358c47ac" +[[package]] +name = "rand" +version = "0.3.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64ac302d8f83c0c1974bf758f6b041c6c8ada916fbb44a609158ca8b064cc76c" +dependencies = [ + "libc", + "rand 0.4.6", +] + [[package]] name = "rand" version = "0.4.6" @@ -4381,6 +4391,19 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "rust-crypto" +version = "0.2.36" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f76d05d3993fd5f4af9434e8e436db163a12a9d40e1a58a726f27a01dfd12a2a" +dependencies = [ + "gcc", + "libc", + "rand 0.3.23", + "rustc-serialize", + "time 0.1.43", +] + [[package]] name = "rustc-demangle" version = "0.1.16" @@ -4399,6 +4422,12 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3e75f6a532d0fd9f7f13144f392b6ad56a32696bfcd9c78f797f16bbb6f072d6" +[[package]] +name = "rustc-serialize" +version = "0.3.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dcf128d1287d2ea9d80910b5f1120d0b8eede3fbf1abe91c40d39ea7d51e6fda" + [[package]] name = "rustc_version" version = "0.2.3" diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 2f388e46111..6025076cb5d 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -40,6 +40,7 @@ use super::range_sync::{BatchId, ChainId, RangeSync, EPOCHS_PER_BATCH}; use super::RequestId; use crate::service::NetworkMessage; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError}; +use environment::TaskExecutor; use eth2_libp2p::rpc::{methods::MAX_REQUEST_BLOCKS, BlocksByRootRequest, GoodbyeReason}; use eth2_libp2p::types::NetworkGlobals; use eth2_libp2p::{PeerAction, PeerId}; @@ -134,6 +135,9 @@ pub struct SyncManager { /// A reference to the underlying beacon chain. chain: Arc>, + /// Executor for spawning tokio tasks. + executor: TaskExecutor, + /// A reference to the network globals and peer-db. network_globals: Arc>, @@ -204,6 +208,7 @@ pub fn spawn( sync_send.clone(), log.clone(), ), + executor: executor.clone(), network: SyncNetworkContext::new(network_send, network_globals.clone(), log.clone()), chain: beacon_chain, network_globals, @@ -300,7 +305,7 @@ impl SyncManager { /// There are two reasons we could have received a BlocksByRoot response /// - We requested a single hash and have received a response for the single_block_lookup /// - We are looking up parent blocks in parent lookup search - fn blocks_by_root_response( + async fn blocks_by_root_response( &mut self, peer_id: PeerId, request_id: RequestId, @@ -318,7 +323,8 @@ impl SyncManager { single_block_hash = Some(block_request.hash); } if let Some(block_hash) = single_block_hash { - self.single_block_lookup_response(peer_id, block, block_hash); + self.single_block_lookup_response(peer_id, block, block_hash) + .await; return; } @@ -340,7 +346,7 @@ impl SyncManager { // add the block to response parent_request.downloaded_blocks.push(block); // queue for processing - self.process_parent_request(parent_request); + self.process_parent_request(parent_request).await; } None => { // this is a stream termination @@ -384,12 +390,14 @@ impl SyncManager { /// Processes the response obtained from a single block lookup search. If the block is /// processed or errors, the search ends. If the blocks parent is unknown, a block parent /// lookup search is started. - fn single_block_lookup_response( + async fn single_block_lookup_response( &mut self, peer_id: PeerId, block: SignedBeaconBlock, expected_block_hash: Hash256, ) { + const FN_NAME: &str = "single_block_lookup_response"; + // verify the hash is correct and try and process the block if expected_block_hash != block.canonical_root() { // The peer that sent this, sent us the wrong block. @@ -399,8 +407,29 @@ impl SyncManager { return; } + let chain = self.chain.clone(); + let inner_block = block.clone(); + + let block_result = match self + .executor + .handle + .spawn_blocking(move || chain.process_block(inner_block)) + .await + { + Ok(block_result) => block_result, + Err(e) => { + error!( + self.log, + "Failed to spawn blocking task"; + "msg" => FN_NAME, + "error" => format!("{:?}", e) + ); + return; + } + }; + // we have the correct block, try and process it - match self.chain.process_block(block.clone()) { + match block_result { Ok(block_root) => { info!(self.log, "Processed block"; "block" => format!("{}", block_root)); @@ -599,7 +628,7 @@ impl SyncManager { // manager /// A new block has been received for a parent lookup query, process it. - fn process_parent_request(&mut self, mut parent_request: ParentRequests) { + async fn process_parent_request(&mut self, mut parent_request: ParentRequests) { // verify the last added block is the parent of the last requested block if parent_request.downloaded_blocks.len() < 2 { @@ -652,7 +681,29 @@ impl SyncManager { .downloaded_blocks .pop() .expect("There is always at least one block in the queue"); - match self.chain.process_block(newest_block.clone()) { + + let chain = self.chain.clone(); + let inner_block = newest_block.clone(); + + let block_result = match self + .executor + .handle + .spawn_blocking(move || chain.process_block(inner_block)) + .await + { + Ok(block_result) => block_result, + Err(e) => { + error!( + self.log, + "Failed to spawn blocking task"; + "msg" => "process_parent_request", + "error" => format!("{:?}", e) + ); + return; + } + }; + + match block_result { Err(BlockError::ParentUnknown { .. }) => { // need to keep looking for parents // add the block back to the queue and continue the search @@ -760,7 +811,8 @@ impl SyncManager { request_id, beacon_block, } => { - self.blocks_by_root_response(peer_id, request_id, beacon_block.map(|b| *b)); + self.blocks_by_root_response(peer_id, request_id, beacon_block.map(|b| *b)) + .await; } SyncMessage::UnknownBlock(peer_id, block) => { self.add_unknown_block(peer_id, *block); diff --git a/lighthouse/environment/src/executor.rs b/lighthouse/environment/src/executor.rs index f7d06cc516a..26567e1245d 100644 --- a/lighthouse/environment/src/executor.rs +++ b/lighthouse/environment/src/executor.rs @@ -7,7 +7,7 @@ use tokio::runtime::Handle; #[derive(Clone)] pub struct TaskExecutor { /// The handle to the runtime on which tasks are spawned - pub(crate) handle: Handle, + pub handle: Handle, /// The receiver exit future which on receiving shuts down the task pub(crate) exit: exit_future::Exit, pub(crate) log: slog::Logger, From 27b9b20359fcabe359ad9c90aaed2b435db624d2 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 16 Aug 2020 11:23:46 +1000 Subject: [PATCH 05/20] Fix disv5 issue in Cargo.lock --- Cargo.lock | 35 +++-------------------------------- 1 file changed, 3 insertions(+), 32 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0dfaa9e931a..7722ecfb1ad 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1195,9 +1195,9 @@ checksum = "212d0f5754cb6769937f4501cc0e67f4f4483c8d2c3e1e922ee9edbe4ab4c7c0" [[package]] name = "discv5" -version = "0.1.0-alpha.9" +version = "0.1.0-alpha.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98bc715508160877f74d828b94238c156c50f0ca80f51271bca9a855be94c488" +checksum = "90782d49541b01f9b7e34e6af5d80d01396bf7b1a81505a0035da224134b8d73" dependencies = [ "arrayvec", "digest 0.8.1", @@ -1213,10 +1213,10 @@ dependencies = [ "lru_time_cache", "multihash", "net2", + "openssl", "parking_lot 0.11.0", "rand 0.7.3", "rlp", - "rust-crypto", "sha2 0.8.2", "smallvec 1.4.1", "tokio 0.2.22", @@ -4020,16 +4020,6 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "def50a86306165861203e7f84ecffbbdfdea79f0e51039b33de1e952358c47ac" -[[package]] -name = "rand" -version = "0.3.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64ac302d8f83c0c1974bf758f6b041c6c8ada916fbb44a609158ca8b064cc76c" -dependencies = [ - "libc", - "rand 0.4.6", -] - [[package]] name = "rand" version = "0.4.6" @@ -4391,19 +4381,6 @@ dependencies = [ "crossbeam-utils", ] -[[package]] -name = "rust-crypto" -version = "0.2.36" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f76d05d3993fd5f4af9434e8e436db163a12a9d40e1a58a726f27a01dfd12a2a" -dependencies = [ - "gcc", - "libc", - "rand 0.3.23", - "rustc-serialize", - "time 0.1.43", -] - [[package]] name = "rustc-demangle" version = "0.1.16" @@ -4422,12 +4399,6 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3e75f6a532d0fd9f7f13144f392b6ad56a32696bfcd9c78f797f16bbb6f072d6" -[[package]] -name = "rustc-serialize" -version = "0.3.24" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dcf128d1287d2ea9d80910b5f1120d0b8eede3fbf1abe91c40d39ea7d51e6fda" - [[package]] name = "rustc_version" version = "0.2.3" From bc0b5535a30ba6132a6c9c2777aacc2a9ff6bef8 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Sun, 16 Aug 2020 12:50:36 +1000 Subject: [PATCH 06/20] Add sync blocks to gossip processor --- beacon_node/network/src/metrics.rs | 32 +- .../network/src/router/gossip_processor.rs | 361 ++++++++++-------- beacon_node/network/src/router/processor.rs | 17 +- beacon_node/network/src/sync/manager.rs | 87 ++--- 4 files changed, 275 insertions(+), 222 deletions(-) diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index 7679a9804fc..5e748cac095 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -77,22 +77,37 @@ lazy_static! { "gossip_processor_worker_time", "Time taken for a worker to fully process some parcel of work." ); - pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_beacon_block_queue_total", + // Gossip blocks. + pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( + "gossip_processor_gossip_block_queue_total", "Count of unagg. attestations waiting to be processed." ); - pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_beacon_block_worker_time", + pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_WORKER_TIME: Result = try_create_histogram( + "gossip_processor_gossip_block_worker_time", "Time taken for a worker to fully process an gossip block." ); - pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( - "gossip_processor_beacon_block_verified_total", + pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( + "gossip_processor_gossip_block_imported_total", + "Total number of gossip blocks imported to fork choice, etc." + ); + // Sync blocks. + pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( + "gossip_processor_sync_block_queue_total", + "Count of unagg. attestations waiting to be processed." + ); + pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_WORKER_TIME: Result = try_create_histogram( + "gossip_processor_sync_block_worker_time", + "Time taken for a worker to fully process an gossip block." + ); + pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( + "gossip_processor_sync_block_verified_total", "Total number of gossip blocks verified for gossip." ); - pub static ref GOSSIP_PROCESSOR_BEACON_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_beacon_block_imported_total", + pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( + "gossip_processor_sync_block_imported_total", "Total number of gossip blocks imported to fork choice, etc." ); + // Unaggregated attestations. pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( "gossip_processor_unaggregated_attestation_queue_total", "Count of unagg. attestations waiting to be processed." @@ -109,6 +124,7 @@ lazy_static! { "gossip_processor_unaggregated_attestation_imported_total", "Total number of unaggregated attestations imported to fork choice, etc." ); + // Aggregated attestations. pub static ref GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( "gossip_processor_aggregated_attestation_queue_total", "Count of agg. attestations waiting to be processed." diff --git a/beacon_node/network/src/router/gossip_processor.rs b/beacon_node/network/src/router/gossip_processor.rs index 4c786381746..75b9184c470 100644 --- a/beacon_node/network/src/router/gossip_processor.rs +++ b/beacon_node/network/src/router/gossip_processor.rs @@ -47,13 +47,13 @@ use ssz::Encode; use std::collections::VecDeque; use std::sync::Arc; use std::time::{Duration, Instant}; -use tokio::sync::mpsc; +use tokio::sync::{mpsc, oneshot}; use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId}; /// The maximum size of the channel for work events to the `GossipProcessor`. /// /// Setting this too low will cause consensus messages to be dropped. -const MAX_WORK_EVENT_QUEUE_LEN: usize = 16_384; +pub const MAX_WORK_EVENT_QUEUE_LEN: usize = 16_384; /// The maximum size of the channel for idle events to the `GossipProcessor`. /// @@ -69,9 +69,13 @@ const MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN: usize = 16_384; /// start dropping them. const MAX_AGGREGATED_ATTESTATION_QUEUE_LEN: usize = 1_024; -/// The maximum number of queued `SignedBeaconBlock` objects that will be stored before we start -/// dropping them. -const MAX_BLOCK_QUEUE_LEN: usize = 1_024; +/// The maximum number of queued `SignedBeaconBlock` objects received on gossip that will be stored +/// before we start dropping them. +const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024; + +/// The maximum number of queued `SignedBeaconBlock` objects received during syncing that will be +/// stored before we start dropping them. +const MAX_SYNC_BLOCK_QUEUE_LEN: usize = 1_024; /// The name of the manager tokio task. const MANAGER_TASK_NAME: &str = "beacon_gossip_processor_manager"; @@ -81,16 +85,13 @@ const WORKER_TASK_NAME: &str = "beacon_gossip_processor_worker"; /// The minimum interval between log messages indicating that a queue is full. const LOG_DEBOUNCE_INTERVAL: Duration = Duration::from_secs(30); -/// A queued item from gossip, awaiting processing. -struct QueueItem { - message_id: MessageId, - peer_id: PeerId, - item: T, -} +/// Used to send/receive results from a sync block import in a blocking task. +pub type BlockResultSender = oneshot::Sender>>; +pub type BlockResultReceiver = oneshot::Receiver>>; /// A simple first-in-first-out queue with a maximum length. struct FifoQueue { - queue: VecDeque>, + queue: VecDeque, max_length: usize, } @@ -104,7 +105,7 @@ impl FifoQueue { } /// Add a new item to the queue. - pub fn push(&mut self, item: QueueItem, log: &Logger) { + pub fn push(&mut self, item: T, log: &Logger) { if self.queue.len() == self.max_length { error!( log, @@ -117,7 +118,7 @@ impl FifoQueue { } /// Remove the next item from the queue. - pub fn pop(&mut self) -> Option> { + pub fn pop(&mut self) -> Option { self.queue.pop_front() } @@ -129,7 +130,7 @@ impl FifoQueue { /// A simple last-in-first-out queue with a maximum length. struct LifoQueue { - queue: VecDeque>, + queue: VecDeque, max_length: usize, } @@ -143,7 +144,7 @@ impl LifoQueue { } /// Add a new item to the queue. - pub fn push(&mut self, item: QueueItem) { + pub fn push(&mut self, item: T) { if self.queue.len() == self.max_length { self.queue.pop_back(); } @@ -151,7 +152,7 @@ impl LifoQueue { } /// Remove the next item from the queue. - pub fn pop(&mut self) -> Option> { + pub fn pop(&mut self) -> Option { self.queue.pop_front() } @@ -167,10 +168,9 @@ impl LifoQueue { } /// An event to be processed by the manager task. -#[derive(Debug, PartialEq)] +#[derive(Debug)] pub struct WorkEvent { - message_id: MessageId, - peer_id: PeerId, + drop_during_sync: bool, work: Work, } @@ -184,9 +184,14 @@ impl WorkEvent { should_import: bool, ) -> Self { Self { - message_id, - peer_id, - work: Work::Attestation(Box::new((attestation, subnet_id, should_import))), + drop_during_sync: true, + work: Work::Attestation { + message_id, + peer_id, + attestation: Box::new(attestation), + subnet_id, + should_import, + }, } } @@ -197,32 +202,71 @@ impl WorkEvent { aggregate: SignedAggregateAndProof, ) -> Self { Self { - message_id, - peer_id, - work: Work::Aggregate(Box::new(aggregate)), + drop_during_sync: true, + work: Work::Aggregate { + message_id, + peer_id, + aggregate: Box::new(aggregate), + }, } } /// Create a new `Work` event for some block. - pub fn beacon_block( + pub fn gossip_beacon_block( message_id: MessageId, peer_id: PeerId, block: Box>, ) -> Self { Self { - message_id, - peer_id, - work: Work::Block(block), + drop_during_sync: true, + work: Work::GossipBlock { + message_id, + peer_id, + block, + }, } } + + /// Create a new `Work` event for some block, where the result from computation (if any) is + /// sent to the other side of `result_tx`. + pub fn sync_beacon_block(block: Box>) -> (Self, BlockResultReceiver) { + let (result_tx, result_rx) = oneshot::channel(); + let event = Self { + drop_during_sync: false, + work: Work::SyncBlock { block, result_tx }, + }; + (event, result_rx) + } } /// A consensus message from gossip which requires processing. -#[derive(Debug, PartialEq)] +#[derive(Debug)] pub enum Work { - Attestation(Box<(Attestation, SubnetId, bool)>), - Aggregate(Box>), - Block(Box>), + // Attestation(Box<(Attestation, SubnetId, bool)>), + Attestation { + message_id: MessageId, + peer_id: PeerId, + attestation: Box>, + subnet_id: SubnetId, + should_import: bool, + }, + // Aggregate(Box>), + Aggregate { + message_id: MessageId, + peer_id: PeerId, + aggregate: Box>, + }, + // GossipBlock(Box>), + GossipBlock { + message_id: MessageId, + peer_id: PeerId, + block: Box>, + }, + // SyncBlock((Box>, Box>)), + SyncBlock { + block: Box>, + result_tx: BlockResultSender, + }, } /// Provides de-bounce functionality for logging. @@ -268,9 +312,7 @@ impl GossipProcessor { /// /// Only `self.max_workers` will ever be spawned at one time. Each worker is a `tokio` task /// started with `spawn_blocking`. - pub fn spawn_manager(mut self) -> mpsc::Sender> { - let (event_tx, mut event_rx) = - mpsc::channel::>(MAX_WORK_EVENT_QUEUE_LEN); + pub fn spawn_manager(mut self, mut event_rx: mpsc::Receiver>) { let (idle_tx, mut idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); let mut aggregate_queue = LifoQueue::new(MAX_AGGREGATED_ATTESTATION_QUEUE_LEN); @@ -279,7 +321,9 @@ impl GossipProcessor { let mut attestation_queue = LifoQueue::new(MAX_UNAGGREGATED_ATTESTATION_QUEUE_LEN); let mut attestation_debounce = TimeLatch::default(); - let mut block_queue = FifoQueue::new(MAX_BLOCK_QUEUE_LEN); + let mut gossip_block_queue = FifoQueue::new(MAX_GOSSIP_BLOCK_QUEUE_LEN); + + let mut sync_block_queue = FifoQueue::new(MAX_SYNC_BLOCK_QUEUE_LEN); let executor = self.executor.clone(); @@ -334,36 +378,27 @@ impl GossipProcessor { metrics::start_timer(&metrics::GOSSIP_PROCESSOR_EVENT_HANDLING_SECONDS); let can_spawn = self.current_workers < self.max_workers; + let drop_during_sync = work_event + .as_ref() + .map_or(false, |event| event.drop_during_sync); match work_event { // There is no new work event, but we are able to spawn a new worker. None if can_spawn => { - // Always check blocks first, since they might be required to process an - // attestation. After blocks, heck the aggregates, *then* the unaggregates + // Check sync blocks before gossip blocks, since we've already explicitly + // requested these blocks. + if let Some(item) = sync_block_queue.pop() { + self.spawn_worker(idle_tx.clone(), item); + } else if let Some(item) = gossip_block_queue.pop() { + self.spawn_worker(idle_tx.clone(), item); + // Check the aggregates, *then* the unaggregates // since we assume that aggregates are more valuable to local validators // and effectively give us more information with less signature // verification time. - if let Some(item) = block_queue.pop() { - self.spawn_worker( - idle_tx.clone(), - item.message_id, - item.peer_id, - Work::Block(item.item), - ); } else if let Some(item) = aggregate_queue.pop() { - self.spawn_worker( - idle_tx.clone(), - item.message_id, - item.peer_id, - Work::Aggregate(item.item), - ); + self.spawn_worker(idle_tx.clone(), item); } else if let Some(item) = attestation_queue.pop() { - self.spawn_worker( - idle_tx.clone(), - item.message_id, - item.peer_id, - Work::Attestation(item.item), - ); + self.spawn_worker(idle_tx.clone(), item); } } // There is no new work event and we are unable to spawn a new worker. @@ -378,7 +413,8 @@ impl GossipProcessor { } // There is a new work event, but the chain is syncing. Ignore it. Some(WorkEvent { .. }) - if self.network_globals.sync_state.read().is_syncing() => + if self.network_globals.sync_state.read().is_syncing() + && drop_during_sync => { metrics::inc_counter(&metrics::GOSSIP_PROCESSOR_WORK_EVENTS_IGNORED_TOTAL); trace!( @@ -388,38 +424,12 @@ impl GossipProcessor { ); } // There is a new work event and the chain is not syncing. Process it. - Some(WorkEvent { - message_id, - peer_id, - work, - }) => match work { - Work::Attestation(_) if can_spawn => { - self.spawn_worker(idle_tx.clone(), message_id, peer_id, work) - } - Work::Attestation(attestation) => attestation_queue.push(QueueItem { - message_id, - peer_id, - item: attestation, - }), - Work::Aggregate(_) if can_spawn => { - self.spawn_worker(idle_tx.clone(), message_id, peer_id, work) - } - Work::Aggregate(aggregate) => aggregate_queue.push(QueueItem { - message_id, - peer_id, - item: aggregate, - }), - Work::Block(_) if can_spawn => { - self.spawn_worker(idle_tx.clone(), message_id, peer_id, work) - } - Work::Block(block) => block_queue.push( - QueueItem { - message_id, - peer_id, - item: block, - }, - &self.log, - ), + Some(WorkEvent { work, .. }) => match work { + _ if can_spawn => self.spawn_worker(idle_tx.clone(), work), + Work::Attestation { .. } => attestation_queue.push(work), + Work::Aggregate { .. } => aggregate_queue.push(work), + Work::GossipBlock { .. } => gossip_block_queue.push(work, &self.log), + Work::SyncBlock { .. } => sync_block_queue.push(work, &self.log), }, } @@ -436,8 +446,12 @@ impl GossipProcessor { aggregate_queue.len() as i64, ); metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_QUEUE_TOTAL, - block_queue.len() as i64, + &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL, + gossip_block_queue.len() as i64, + ); + metrics::set_gauge( + &metrics::GOSSIP_PROCESSOR_SYNC_BLOCK_QUEUE_TOTAL, + sync_block_queue.len() as i64, ); if aggregate_queue.is_full() && aggregate_debounce.elapsed() { @@ -462,20 +476,12 @@ impl GossipProcessor { // Spawn on the non-blocking executor. executor.spawn(manager_future, MANAGER_TASK_NAME); - - event_tx } /// Spawns a blocking worker thread to process some `Work`. /// /// Sends an message on `idle_tx` when the work is complete and the task is stopping. - fn spawn_worker( - &mut self, - mut idle_tx: mpsc::Sender<()>, - message_id: MessageId, - peer_id: PeerId, - work: Work, - ) { + fn spawn_worker(&mut self, mut idle_tx: mpsc::Sender<()>, work: Work) { let worker_timer = metrics::start_timer(&metrics::GOSSIP_PROCESSOR_WORKER_TIME); metrics::inc_counter(&metrics::GOSSIP_PROCESSOR_WORKERS_SPAWNED_TOTAL); @@ -497,9 +503,13 @@ impl GossipProcessor { /* * Unaggregated attestation verification. */ - Work::Attestation(boxed_tuple) => { - let (attestation, subnet_id, should_import) = *boxed_tuple; - + Work::Attestation { + message_id, + peer_id, + attestation, + subnet_id, + should_import, + } => { let _attestation_timer = metrics::start_timer( &metrics::GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_WORKER_TIME, ); @@ -510,7 +520,7 @@ impl GossipProcessor { let beacon_block_root = attestation.data.beacon_block_root; let attestation = match chain - .verify_unaggregated_attestation_for_gossip(attestation, subnet_id) + .verify_unaggregated_attestation_for_gossip(*attestation, subnet_id) { Ok(attestation) => attestation, Err(e) => { @@ -572,7 +582,11 @@ impl GossipProcessor { /* * Aggregated attestation verification. */ - Work::Aggregate(boxed_aggregate) => { + Work::Aggregate { + message_id, + peer_id, + aggregate, + } => { let _attestation_timer = metrics::start_timer( &metrics::GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_WORKER_TIME, ); @@ -581,24 +595,23 @@ impl GossipProcessor { ); let beacon_block_root = - boxed_aggregate.message.aggregate.data.beacon_block_root; + aggregate.message.aggregate.data.beacon_block_root; - let aggregate = match chain - .verify_aggregated_attestation_for_gossip(*boxed_aggregate) - { - Ok(aggregate) => aggregate, - Err(e) => { - handle_attestation_verification_failure( - &log, - sync_tx, - peer_id.clone(), - beacon_block_root, - "aggregated", - e, - ); - return; - } - }; + let aggregate = + match chain.verify_aggregated_attestation_for_gossip(*aggregate) { + Ok(aggregate) => aggregate, + Err(e) => { + handle_attestation_verification_failure( + &log, + sync_tx, + peer_id.clone(), + beacon_block_root, + "aggregated", + e, + ); + return; + } + }; // Indicate to the `Network` service that this message is valid and can be // propagated on the gossip network. @@ -640,53 +653,56 @@ impl GossipProcessor { } } /* - * Beacon block verification. + * Verification for beacon blocks received on gossip. */ - Work::Block(boxed_block) => { + Work::GossipBlock { + message_id, + peer_id, + block, + } => { let _block_timer = metrics::start_timer( - &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_WORKER_TIME, + &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_WORKER_TIME, ); metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_VERIFIED_TOTAL, + &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, ); - let verified_block = - match chain.verify_block_for_gossip(*boxed_block) { - Ok(verified_block) => { - info!( - log, - "New block received"; - "slot" => verified_block.block.slot(), - "hash" => verified_block.block_root.to_string() - ); - propagate_gossip_message( - network_tx, - message_id, - peer_id.clone(), - &log, - ); - verified_block - } - Err(BlockError::ParentUnknown(block)) => { - send_sync_message( - sync_tx, - SyncMessage::UnknownBlock(peer_id, block), - &log, - ); - return; - } - Err(e) => { - warn!( - log, - "Could not verify block for gossip"; - "error" => format!("{:?}", e) - ); - return; - } - }; + let verified_block = match chain.verify_block_for_gossip(*block) { + Ok(verified_block) => { + info!( + log, + "New block received"; + "slot" => verified_block.block.slot(), + "hash" => verified_block.block_root.to_string() + ); + propagate_gossip_message( + network_tx, + message_id, + peer_id.clone(), + &log, + ); + verified_block + } + Err(BlockError::ParentUnknown(block)) => { + send_sync_message( + sync_tx, + SyncMessage::UnknownBlock(peer_id, block), + &log, + ); + return; + } + Err(e) => { + warn!( + log, + "Could not verify block for gossip"; + "error" => format!("{:?}", e) + ); + return; + } + }; metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_BEACON_BLOCK_IMPORTED_TOTAL, + &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, ); let block = Box::new(verified_block.block.clone()); @@ -748,6 +764,23 @@ impl GossipProcessor { } } } + /* + * Verification for beacon blocks received during syncing via RPC. + */ + Work::SyncBlock { block, result_tx } => { + let _block_timer = metrics::start_timer( + &metrics::GOSSIP_PROCESSOR_SYNC_BLOCK_WORKER_TIME, + ); + metrics::inc_counter( + &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, + ); + + let block_result = chain.process_block(*block); + + if let Err(_) = result_tx.send(block_result) { + crit!(log, "Failed return sync block result"); + } + } }; }; handler(); diff --git a/beacon_node/network/src/router/processor.rs b/beacon_node/network/src/router/processor.rs index ebefed94547..c2767ef71b5 100644 --- a/beacon_node/network/src/router/processor.rs +++ b/beacon_node/network/src/router/processor.rs @@ -1,4 +1,6 @@ -use super::gossip_processor::{GossipProcessor, WorkEvent as GossipWorkEvent}; +use super::gossip_processor::{ + GossipProcessor, WorkEvent as GossipWorkEvent, MAX_WORK_EVENT_QUEUE_LEN, +}; use crate::service::NetworkMessage; use crate::sync::{PeerSyncInfo, SyncMessage}; use beacon_chain::{observed_operations::ObservationOutcome, BeaconChain, BeaconChainTypes}; @@ -32,7 +34,7 @@ pub struct Processor { sync_send: mpsc::UnboundedSender>, /// A network context to return and handle RPC requests. network: HandlerNetworkContext, - /// A multi-threaded, non-blocking processor for consensus gossip messages. + /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. gossip_processor_send: mpsc::Sender>, /// The `RPCHandler` logger. log: slog::Logger, @@ -48,6 +50,8 @@ impl Processor { log: &slog::Logger, ) -> Self { let sync_logger = log.new(o!("service"=> "sync")); + let (gossip_processor_send, gossip_processor_receive) = + mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); // spawn the sync thread let sync_send = crate::sync::manager::spawn( @@ -55,10 +59,11 @@ impl Processor { beacon_chain.clone(), network_globals.clone(), network_send.clone(), + gossip_processor_send.clone(), sync_logger, ); - let gossip_processor_send = GossipProcessor { + GossipProcessor { beacon_chain: beacon_chain.clone(), network_tx: network_send.clone(), sync_tx: sync_send.clone(), @@ -68,7 +73,7 @@ impl Processor { current_workers: 0, log: log.clone(), } - .spawn_manager(); + .spawn_manager(gossip_processor_receive); Processor { chain: beacon_chain, @@ -521,7 +526,9 @@ impl Processor { block: Box>, ) { self.gossip_processor_send - .try_send(GossipWorkEvent::beacon_block(message_id, peer_id, block)) + .try_send(GossipWorkEvent::gossip_beacon_block( + message_id, peer_id, block, + )) .unwrap_or_else(|e| { error!( &self.log, diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 6025076cb5d..4da6fea4f0a 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -38,9 +38,9 @@ use super::network_context::SyncNetworkContext; use super::peer_sync_info::{PeerSyncInfo, PeerSyncType}; use super::range_sync::{BatchId, ChainId, RangeSync, EPOCHS_PER_BATCH}; use super::RequestId; +use crate::router::gossip_processor::WorkEvent as BeaconWorkEvent; use crate::service::NetworkMessage; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError}; -use environment::TaskExecutor; use eth2_libp2p::rpc::{methods::MAX_REQUEST_BLOCKS, BlocksByRootRequest, GoodbyeReason}; use eth2_libp2p::types::NetworkGlobals; use eth2_libp2p::{PeerAction, PeerId}; @@ -135,9 +135,6 @@ pub struct SyncManager { /// A reference to the underlying beacon chain. chain: Arc>, - /// Executor for spawning tokio tasks. - executor: TaskExecutor, - /// A reference to the network globals and peer-db. network_globals: Arc>, @@ -164,6 +161,9 @@ pub struct SyncManager { /// The sending part of input_channel sync_send: mpsc::UnboundedSender>, + + /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. + gossip_processor_send: mpsc::Sender>, } /// Object representing a single block lookup request. @@ -191,6 +191,7 @@ pub fn spawn( beacon_chain: Arc>, network_globals: Arc>, network_send: mpsc::UnboundedSender>, + gossip_processor_send: mpsc::Sender>, log: slog::Logger, ) -> mpsc::UnboundedSender> { assert!( @@ -208,7 +209,6 @@ pub fn spawn( sync_send.clone(), log.clone(), ), - executor: executor.clone(), network: SyncNetworkContext::new(network_send, network_globals.clone(), log.clone()), chain: beacon_chain, network_globals, @@ -217,6 +217,7 @@ pub fn spawn( single_block_lookups: FnvHashMap::default(), log: log.clone(), sync_send: sync_send.clone(), + gossip_processor_send, }; // spawn the sync manager thread @@ -387,6 +388,36 @@ impl SyncManager { } } + async fn process_block_async( + &mut self, + block: SignedBeaconBlock, + ) -> Option>> { + let (event, rx) = BeaconWorkEvent::sync_beacon_block(Box::new(block)); + match self.gossip_processor_send.try_send(event) { + Ok(_) => {} + Err(e) => { + error!( + self.log, + "Failed to send sync block to processor"; + "error" => format!("{:?}", e) + ); + return None; + } + } + + match rx.await { + Ok(block_result) => Some(block_result), + Err(_) => { + warn!( + self.log, + "Sync block not processed"; + "msg" => "likely due to system resource exhaustion" + ); + None + } + } + } + /// Processes the response obtained from a single block lookup search. If the block is /// processed or errors, the search ends. If the blocks parent is unknown, a block parent /// lookup search is started. @@ -396,8 +427,6 @@ impl SyncManager { block: SignedBeaconBlock, expected_block_hash: Hash256, ) { - const FN_NAME: &str = "single_block_lookup_response"; - // verify the hash is correct and try and process the block if expected_block_hash != block.canonical_root() { // The peer that sent this, sent us the wrong block. @@ -407,25 +436,9 @@ impl SyncManager { return; } - let chain = self.chain.clone(); - let inner_block = block.clone(); - - let block_result = match self - .executor - .handle - .spawn_blocking(move || chain.process_block(inner_block)) - .await - { - Ok(block_result) => block_result, - Err(e) => { - error!( - self.log, - "Failed to spawn blocking task"; - "msg" => FN_NAME, - "error" => format!("{:?}", e) - ); - return; - } + let block_result = match self.process_block_async(block.clone()).await { + Some(block_result) => block_result, + None => return, }; // we have the correct block, try and process it @@ -682,25 +695,9 @@ impl SyncManager { .pop() .expect("There is always at least one block in the queue"); - let chain = self.chain.clone(); - let inner_block = newest_block.clone(); - - let block_result = match self - .executor - .handle - .spawn_blocking(move || chain.process_block(inner_block)) - .await - { - Ok(block_result) => block_result, - Err(e) => { - error!( - self.log, - "Failed to spawn blocking task"; - "msg" => "process_parent_request", - "error" => format!("{:?}", e) - ); - return; - } + let block_result = match self.process_block_async(newest_block.clone()).await { + Some(block_result) => block_result, + None => return, }; match block_result { From 890671f0198b8a27b83eea10e37948158daba3c7 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 07:33:16 +1000 Subject: [PATCH 07/20] Tidy, rename --- beacon_node/network/src/metrics.rs | 24 +++++----- .../network/src/router/gossip_processor.rs | 44 +++++++++---------- beacon_node/network/src/sync/manager.rs | 2 +- 3 files changed, 33 insertions(+), 37 deletions(-) diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index 5e748cac095..68085fb12ca 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -80,7 +80,7 @@ lazy_static! { // Gossip blocks. pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( "gossip_processor_gossip_block_queue_total", - "Count of unagg. attestations waiting to be processed." + "Count of blocks from gossip waiting to be verified." ); pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_WORKER_TIME: Result = try_create_histogram( "gossip_processor_gossip_block_worker_time", @@ -91,20 +91,20 @@ lazy_static! { "Total number of gossip blocks imported to fork choice, etc." ); // Sync blocks. - pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_sync_block_queue_total", - "Count of unagg. attestations waiting to be processed." + pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( + "gossip_processor_rpc_block_queue_total", + "Count of blocks from the rpc waiting to be verified." ); - pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_sync_block_worker_time", - "Time taken for a worker to fully process an gossip block." + pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_WORKER_TIME: Result = try_create_histogram( + "gossip_processor_rpc_block_worker_time", + "Time taken for a worker to fully process a block from the rpc." ); - pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( - "gossip_processor_sync_block_verified_total", - "Total number of gossip blocks verified for gossip." + pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( + "gossip_processor_rpc_block_verified_total", + "Total number of verified blocks from the rpc." ); - pub static ref GOSSIP_PROCESSOR_SYNC_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_sync_block_imported_total", + pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( + "gossip_processor_rpc_block_imported_total", "Total number of gossip blocks imported to fork choice, etc." ); // Unaggregated attestations. diff --git a/beacon_node/network/src/router/gossip_processor.rs b/beacon_node/network/src/router/gossip_processor.rs index 75b9184c470..b6fa81ec294 100644 --- a/beacon_node/network/src/router/gossip_processor.rs +++ b/beacon_node/network/src/router/gossip_processor.rs @@ -75,7 +75,7 @@ const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024; /// The maximum number of queued `SignedBeaconBlock` objects received during syncing that will be /// stored before we start dropping them. -const MAX_SYNC_BLOCK_QUEUE_LEN: usize = 1_024; +const MAX_RPC_BLOCK_QUEUE_LEN: usize = 1_024; /// The name of the manager tokio task. const MANAGER_TASK_NAME: &str = "beacon_gossip_processor_manager"; @@ -85,7 +85,7 @@ const WORKER_TASK_NAME: &str = "beacon_gossip_processor_worker"; /// The minimum interval between log messages indicating that a queue is full. const LOG_DEBOUNCE_INTERVAL: Duration = Duration::from_secs(30); -/// Used to send/receive results from a sync block import in a blocking task. +/// Used to send/receive results from a rpc block import in a blocking task. pub type BlockResultSender = oneshot::Sender>>; pub type BlockResultReceiver = oneshot::Receiver>>; @@ -185,7 +185,7 @@ impl WorkEvent { ) -> Self { Self { drop_during_sync: true, - work: Work::Attestation { + work: Work::GossipAttestation { message_id, peer_id, attestation: Box::new(attestation), @@ -203,7 +203,7 @@ impl WorkEvent { ) -> Self { Self { drop_during_sync: true, - work: Work::Aggregate { + work: Work::GossipAggregate { message_id, peer_id, aggregate: Box::new(aggregate), @@ -229,11 +229,11 @@ impl WorkEvent { /// Create a new `Work` event for some block, where the result from computation (if any) is /// sent to the other side of `result_tx`. - pub fn sync_beacon_block(block: Box>) -> (Self, BlockResultReceiver) { + pub fn rpc_beacon_block(block: Box>) -> (Self, BlockResultReceiver) { let (result_tx, result_rx) = oneshot::channel(); let event = Self { drop_during_sync: false, - work: Work::SyncBlock { block, result_tx }, + work: Work::RpcBlock { block, result_tx }, }; (event, result_rx) } @@ -242,28 +242,24 @@ impl WorkEvent { /// A consensus message from gossip which requires processing. #[derive(Debug)] pub enum Work { - // Attestation(Box<(Attestation, SubnetId, bool)>), - Attestation { + GossipAttestation { message_id: MessageId, peer_id: PeerId, attestation: Box>, subnet_id: SubnetId, should_import: bool, }, - // Aggregate(Box>), - Aggregate { + GossipAggregate { message_id: MessageId, peer_id: PeerId, aggregate: Box>, }, - // GossipBlock(Box>), GossipBlock { message_id: MessageId, peer_id: PeerId, block: Box>, }, - // SyncBlock((Box>, Box>)), - SyncBlock { + RpcBlock { block: Box>, result_tx: BlockResultSender, }, @@ -323,7 +319,7 @@ impl GossipProcessor { let mut gossip_block_queue = FifoQueue::new(MAX_GOSSIP_BLOCK_QUEUE_LEN); - let mut sync_block_queue = FifoQueue::new(MAX_SYNC_BLOCK_QUEUE_LEN); + let mut rpc_block_queue = FifoQueue::new(MAX_RPC_BLOCK_QUEUE_LEN); let executor = self.executor.clone(); @@ -387,7 +383,7 @@ impl GossipProcessor { None if can_spawn => { // Check sync blocks before gossip blocks, since we've already explicitly // requested these blocks. - if let Some(item) = sync_block_queue.pop() { + if let Some(item) = rpc_block_queue.pop() { self.spawn_worker(idle_tx.clone(), item); } else if let Some(item) = gossip_block_queue.pop() { self.spawn_worker(idle_tx.clone(), item); @@ -426,10 +422,10 @@ impl GossipProcessor { // There is a new work event and the chain is not syncing. Process it. Some(WorkEvent { work, .. }) => match work { _ if can_spawn => self.spawn_worker(idle_tx.clone(), work), - Work::Attestation { .. } => attestation_queue.push(work), - Work::Aggregate { .. } => aggregate_queue.push(work), + Work::GossipAttestation { .. } => attestation_queue.push(work), + Work::GossipAggregate { .. } => aggregate_queue.push(work), Work::GossipBlock { .. } => gossip_block_queue.push(work, &self.log), - Work::SyncBlock { .. } => sync_block_queue.push(work, &self.log), + Work::RpcBlock { .. } => rpc_block_queue.push(work, &self.log), }, } @@ -450,8 +446,8 @@ impl GossipProcessor { gossip_block_queue.len() as i64, ); metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_SYNC_BLOCK_QUEUE_TOTAL, - sync_block_queue.len() as i64, + &metrics::GOSSIP_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL, + rpc_block_queue.len() as i64, ); if aggregate_queue.is_full() && aggregate_debounce.elapsed() { @@ -503,7 +499,7 @@ impl GossipProcessor { /* * Unaggregated attestation verification. */ - Work::Attestation { + Work::GossipAttestation { message_id, peer_id, attestation, @@ -582,7 +578,7 @@ impl GossipProcessor { /* * Aggregated attestation verification. */ - Work::Aggregate { + Work::GossipAggregate { message_id, peer_id, aggregate, @@ -767,9 +763,9 @@ impl GossipProcessor { /* * Verification for beacon blocks received during syncing via RPC. */ - Work::SyncBlock { block, result_tx } => { + Work::RpcBlock { block, result_tx } => { let _block_timer = metrics::start_timer( - &metrics::GOSSIP_PROCESSOR_SYNC_BLOCK_WORKER_TIME, + &metrics::GOSSIP_PROCESSOR_RPC_BLOCK_WORKER_TIME, ); metrics::inc_counter( &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 4da6fea4f0a..eb5be20059a 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -392,7 +392,7 @@ impl SyncManager { &mut self, block: SignedBeaconBlock, ) -> Option>> { - let (event, rx) = BeaconWorkEvent::sync_beacon_block(Box::new(block)); + let (event, rx) = BeaconWorkEvent::rpc_beacon_block(Box::new(block)); match self.gossip_processor_send.try_send(event) { Ok(_) => {} Err(e) => { From 1b905d9f33c89ffcbdd894d9417670c8771a7682 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 08:02:02 +1000 Subject: [PATCH 08/20] Rename gossip_processor -> beacon_processor --- .../mod.rs} | 12 ++++---- beacon_node/network/src/lib.rs | 1 + beacon_node/network/src/router/mod.rs | 1 - beacon_node/network/src/router/processor.rs | 28 +++++++++---------- beacon_node/network/src/sync/manager.rs | 2 +- 5 files changed, 22 insertions(+), 22 deletions(-) rename beacon_node/network/src/{router/gossip_processor.rs => beacon_processor/mod.rs} (99%) diff --git a/beacon_node/network/src/router/gossip_processor.rs b/beacon_node/network/src/beacon_processor/mod.rs similarity index 99% rename from beacon_node/network/src/router/gossip_processor.rs rename to beacon_node/network/src/beacon_processor/mod.rs index b6fa81ec294..8c38d83b794 100644 --- a/beacon_node/network/src/router/gossip_processor.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -1,4 +1,4 @@ -//! Provides the `GossipProcessor`, a mutli-threaded processor for messages received on the network +//! Provides the `BeaconProcessor`, a mutli-threaded processor for messages received on the network //! that need to be processed by the `BeaconChain`. //! //! Uses `tokio` tasks (instead of raw threads) to provide the following tasks: @@ -8,7 +8,7 @@ //! //! ## Purpose //! -//! The purpose of the `GossipProcessor` is to provide two things: +//! The purpose of the `BeaconProcessor` is to provide two things: //! //! 1. Moving long-running, blocking tasks off the main `tokio` executor. //! 2. A fixed-length buffer for consensus messages. @@ -50,12 +50,12 @@ use std::time::{Duration, Instant}; use tokio::sync::{mpsc, oneshot}; use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId}; -/// The maximum size of the channel for work events to the `GossipProcessor`. +/// The maximum size of the channel for work events to the `BeaconProcessor`. /// /// Setting this too low will cause consensus messages to be dropped. pub const MAX_WORK_EVENT_QUEUE_LEN: usize = 16_384; -/// The maximum size of the channel for idle events to the `GossipProcessor`. +/// The maximum size of the channel for idle events to the `BeaconProcessor`. /// /// Setting this too low will prevent new workers from being spawned. It *should* only need to be /// set to the CPU count, but we set it high to be safe. @@ -288,7 +288,7 @@ impl TimeLatch { /// that need to be processed by the `BeaconChain` /// /// See module level documentation for more information. -pub struct GossipProcessor { +pub struct BeaconProcessor { pub beacon_chain: Arc>, pub network_tx: mpsc::UnboundedSender>, pub sync_tx: mpsc::UnboundedSender>, @@ -299,7 +299,7 @@ pub struct GossipProcessor { pub log: Logger, } -impl GossipProcessor { +impl BeaconProcessor { /// Spawns the "manager" task which checks the receiver end of the returned `Sender` for /// messages which contain some new work which will be: /// diff --git a/beacon_node/network/src/lib.rs b/beacon_node/network/src/lib.rs index f7cc8051c8a..30795a63ef3 100644 --- a/beacon_node/network/src/lib.rs +++ b/beacon_node/network/src/lib.rs @@ -6,6 +6,7 @@ pub mod error; pub mod service; mod attestation_service; +mod beacon_processor; mod metrics; mod persisted_dht; mod router; diff --git a/beacon_node/network/src/router/mod.rs b/beacon_node/network/src/router/mod.rs index 6359d05ea44..c3a729ce712 100644 --- a/beacon_node/network/src/router/mod.rs +++ b/beacon_node/network/src/router/mod.rs @@ -5,7 +5,6 @@ //! syncing-related responses to the Sync manager. #![allow(clippy::unit_arg)] -pub mod gossip_processor; pub mod processor; use crate::error; diff --git a/beacon_node/network/src/router/processor.rs b/beacon_node/network/src/router/processor.rs index c2767ef71b5..0cace588f9d 100644 --- a/beacon_node/network/src/router/processor.rs +++ b/beacon_node/network/src/router/processor.rs @@ -1,5 +1,5 @@ -use super::gossip_processor::{ - GossipProcessor, WorkEvent as GossipWorkEvent, MAX_WORK_EVENT_QUEUE_LEN, +use crate::beacon_processor::{ + BeaconProcessor, WorkEvent as BeaconWorkEvent, MAX_WORK_EVENT_QUEUE_LEN, }; use crate::service::NetworkMessage; use crate::sync::{PeerSyncInfo, SyncMessage}; @@ -35,7 +35,7 @@ pub struct Processor { /// A network context to return and handle RPC requests. network: HandlerNetworkContext, /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. - gossip_processor_send: mpsc::Sender>, + beacon_processor_send: mpsc::Sender>, /// The `RPCHandler` logger. log: slog::Logger, } @@ -50,7 +50,7 @@ impl Processor { log: &slog::Logger, ) -> Self { let sync_logger = log.new(o!("service"=> "sync")); - let (gossip_processor_send, gossip_processor_receive) = + let (beacon_processor_send, beacon_processor_receive) = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); // spawn the sync thread @@ -59,11 +59,11 @@ impl Processor { beacon_chain.clone(), network_globals.clone(), network_send.clone(), - gossip_processor_send.clone(), + beacon_processor_send.clone(), sync_logger, ); - GossipProcessor { + BeaconProcessor { beacon_chain: beacon_chain.clone(), network_tx: network_send.clone(), sync_tx: sync_send.clone(), @@ -73,13 +73,13 @@ impl Processor { current_workers: 0, log: log.clone(), } - .spawn_manager(gossip_processor_receive); + .spawn_manager(beacon_processor_receive); Processor { chain: beacon_chain, sync_send, network: HandlerNetworkContext::new(network_send, log.clone()), - gossip_processor_send, + beacon_processor_send, log: log.clone(), } } @@ -525,8 +525,8 @@ impl Processor { peer_id: PeerId, block: Box>, ) { - self.gossip_processor_send - .try_send(GossipWorkEvent::gossip_beacon_block( + self.beacon_processor_send + .try_send(BeaconWorkEvent::gossip_beacon_block( message_id, peer_id, block, )) .unwrap_or_else(|e| { @@ -547,8 +547,8 @@ impl Processor { subnet_id: SubnetId, should_process: bool, ) { - self.gossip_processor_send - .try_send(GossipWorkEvent::unaggregated_attestation( + self.beacon_processor_send + .try_send(BeaconWorkEvent::unaggregated_attestation( message_id, peer_id, unaggregated_attestation, @@ -571,8 +571,8 @@ impl Processor { peer_id: PeerId, aggregate: SignedAggregateAndProof, ) { - self.gossip_processor_send - .try_send(GossipWorkEvent::aggregated_attestation( + self.beacon_processor_send + .try_send(BeaconWorkEvent::aggregated_attestation( message_id, peer_id, aggregate, )) .unwrap_or_else(|e| { diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index eb5be20059a..b15894505c4 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -38,7 +38,7 @@ use super::network_context::SyncNetworkContext; use super::peer_sync_info::{PeerSyncInfo, PeerSyncType}; use super::range_sync::{BatchId, ChainId, RangeSync, EPOCHS_PER_BATCH}; use super::RequestId; -use crate::router::gossip_processor::WorkEvent as BeaconWorkEvent; +use crate::beacon_processor::WorkEvent as BeaconWorkEvent; use crate::service::NetworkMessage; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError}; use eth2_libp2p::rpc::{methods::MAX_REQUEST_BLOCKS, BlocksByRootRequest, GoodbyeReason}; From 742cfe78543f8f089776b5dcbf12b5dab39af25a Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 08:27:17 +1000 Subject: [PATCH 09/20] Start swapping out chain segment processing --- .../chain_segment.rs} | 14 +----- .../network/src/beacon_processor/mod.rs | 9 ++++ beacon_node/network/src/sync/manager.rs | 48 ++++++++++++++----- beacon_node/network/src/sync/mod.rs | 4 +- .../network/src/sync/range_sync/chain.rs | 36 +++++++++----- .../src/sync/range_sync/chain_collection.rs | 5 ++ .../network/src/sync/range_sync/range.rs | 9 +++- 7 files changed, 85 insertions(+), 40 deletions(-) rename beacon_node/network/src/{sync/block_processor.rs => beacon_processor/chain_segment.rs} (95%) diff --git a/beacon_node/network/src/sync/block_processor.rs b/beacon_node/network/src/beacon_processor/chain_segment.rs similarity index 95% rename from beacon_node/network/src/sync/block_processor.rs rename to beacon_node/network/src/beacon_processor/chain_segment.rs index a4fe5b418ff..ed1cce981db 100644 --- a/beacon_node/network/src/sync/block_processor.rs +++ b/beacon_node/network/src/beacon_processor/chain_segment.rs @@ -1,6 +1,6 @@ use crate::router::processor::FUTURE_SLOT_TOLERANCE; use crate::sync::manager::SyncMessage; -use crate::sync::range_sync::{BatchId, ChainId}; +use crate::sync::{BatchId, BatchProcessResult, ChainId}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, ChainSegmentResult}; use eth2_libp2p::PeerId; use slog::{debug, error, trace, warn}; @@ -17,18 +17,6 @@ pub enum ProcessId { ParentLookup(PeerId), } -/// The result of a block processing request. -// TODO: When correct batch error handling occurs, we will include an error type. -#[derive(Debug)] -pub enum BatchProcessResult { - /// The batch was completed successfully. - Success, - /// The batch processing failed. - Failed, - /// The batch processing failed but managed to import at least one block. - Partial, -} - /// Spawns a thread handling the block processing of a request: range syncing or parent lookup. pub fn spawn_block_processor( chain: Weak>, diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 8c38d83b794..5d0dcfdb3a7 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -50,6 +50,10 @@ use std::time::{Duration, Instant}; use tokio::sync::{mpsc, oneshot}; use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId}; +mod chain_segment; + +pub use chain_segment::ProcessId; + /// The maximum size of the channel for work events to the `BeaconProcessor`. /// /// Setting this too low will cause consensus messages to be dropped. @@ -237,6 +241,11 @@ impl WorkEvent { }; (event, result_rx) } + + /// Create a new work event to import `blocks` as a beacon chain segment. + pub fn chain_segment(process_id: ProcessId, block: Vec>) -> Self { + todo!() + } } /// A consensus message from gossip which requires processing. diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index b15894505c4..34492d05578 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -33,12 +33,11 @@ //! if an attestation references an unknown block) this manager can search for the block and //! subsequently search for parents if needed. -use super::block_processor::{spawn_block_processor, BatchProcessResult, ProcessId}; use super::network_context::SyncNetworkContext; use super::peer_sync_info::{PeerSyncInfo, PeerSyncType}; use super::range_sync::{BatchId, ChainId, RangeSync, EPOCHS_PER_BATCH}; use super::RequestId; -use crate::beacon_processor::WorkEvent as BeaconWorkEvent; +use crate::beacon_processor::{ProcessId, WorkEvent as BeaconWorkEvent}; use crate::service::NetworkMessage; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError}; use eth2_libp2p::rpc::{methods::MAX_REQUEST_BLOCKS, BlocksByRootRequest, GoodbyeReason}; @@ -110,6 +109,18 @@ pub enum SyncMessage { ParentLookupFailed(PeerId), } +/// The result of processing a multiple blocks (a chain segment). +// TODO: When correct batch error handling occurs, we will include an error type. +#[derive(Debug)] +pub enum BatchProcessResult { + /// The batch was completed successfully. + Success, + /// The batch processing failed. + Failed, + /// The batch processing failed but managed to import at least one block. + Partial, +} + /// Maintains a sequential list of parents to lookup and the lookup's current state. struct ParentRequests { /// The blocks that have currently been downloaded. @@ -163,7 +174,7 @@ pub struct SyncManager { sync_send: mpsc::UnboundedSender>, /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. - gossip_processor_send: mpsc::Sender>, + beacon_processor_send: mpsc::Sender>, } /// Object representing a single block lookup request. @@ -191,7 +202,7 @@ pub fn spawn( beacon_chain: Arc>, network_globals: Arc>, network_send: mpsc::UnboundedSender>, - gossip_processor_send: mpsc::Sender>, + beacon_processor_send: mpsc::Sender>, log: slog::Logger, ) -> mpsc::UnboundedSender> { assert!( @@ -207,6 +218,7 @@ pub fn spawn( beacon_chain.clone(), network_globals.clone(), sync_send.clone(), + beacon_processor_send.clone(), log.clone(), ), network: SyncNetworkContext::new(network_send, network_globals.clone(), log.clone()), @@ -217,7 +229,7 @@ pub fn spawn( single_block_lookups: FnvHashMap::default(), log: log.clone(), sync_send: sync_send.clone(), - gossip_processor_send, + beacon_processor_send, }; // spawn the sync manager thread @@ -393,7 +405,7 @@ impl SyncManager { block: SignedBeaconBlock, ) -> Option>> { let (event, rx) = BeaconWorkEvent::rpc_beacon_block(Box::new(block)); - match self.gossip_processor_send.try_send(event) { + match self.beacon_processor_send.try_send(event) { Ok(_) => {} Err(e) => { error!( @@ -708,13 +720,23 @@ impl SyncManager { self.request_parent(parent_request); } Ok(_) | Err(BlockError::BlockIsAlreadyKnown { .. }) => { - spawn_block_processor( - Arc::downgrade(&self.chain), - ProcessId::ParentLookup(parent_request.last_submitted_peer.clone()), - parent_request.downloaded_blocks, - self.sync_send.clone(), - self.log.clone(), - ); + let process_id = + ProcessId::ParentLookup(parent_request.last_submitted_peer.clone()); + let blocks = parent_request.downloaded_blocks; + + match self + .beacon_processor_send + .try_send(BeaconWorkEvent::chain_segment(process_id, blocks)) + { + Ok(_) => {} + Err(e) => { + error!( + self.log, + "Failed to send chain segment to processor"; + "error" => format!("{:?}", e) + ); + } + } } Err(outcome) => { // all else we consider the chain a failure and downvote the peer that sent diff --git a/beacon_node/network/src/sync/mod.rs b/beacon_node/network/src/sync/mod.rs index 2c0fcabb287..0c0bdce3192 100644 --- a/beacon_node/network/src/sync/mod.rs +++ b/beacon_node/network/src/sync/mod.rs @@ -1,14 +1,14 @@ //! Syncing for lighthouse. //! //! Stores the various syncing methods for the beacon chain. -mod block_processor; pub mod manager; mod network_context; mod peer_sync_info; mod range_sync; -pub use manager::SyncMessage; +pub use manager::{BatchProcessResult, SyncMessage}; pub use peer_sync_info::PeerSyncInfo; +pub use range_sync::{BatchId, ChainId}; /// Type of id of rpc requests sent by sync pub type RequestId = usize; diff --git a/beacon_node/network/src/sync/range_sync/chain.rs b/beacon_node/network/src/sync/range_sync/chain.rs index b816b965670..26048ff9303 100644 --- a/beacon_node/network/src/sync/range_sync/chain.rs +++ b/beacon_node/network/src/sync/range_sync/chain.rs @@ -1,11 +1,12 @@ use super::batch::{Batch, BatchId, PendingBatches}; -use crate::sync::block_processor::{spawn_block_processor, BatchProcessResult, ProcessId}; -use crate::sync::network_context::SyncNetworkContext; +use crate::beacon_processor::ProcessId; +use crate::beacon_processor::WorkEvent as BeaconWorkEvent; +use crate::sync::{network_context::SyncNetworkContext, BatchProcessResult}; use crate::sync::{RequestId, SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes}; use eth2_libp2p::{PeerAction, PeerId}; use rand::prelude::*; -use slog::{crit, debug, warn}; +use slog::{crit, debug, error, warn}; use std::collections::HashSet; use std::sync::Arc; use tokio::sync::mpsc; @@ -88,6 +89,9 @@ pub struct SyncingChain { /// back once batch processing has completed. sync_send: mpsc::UnboundedSender>, + /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. + beacon_processor_send: mpsc::Sender>, + /// A reference to the underlying beacon chain. chain: Arc>, @@ -112,6 +116,7 @@ impl SyncingChain { target_head_root: Hash256, peer_id: PeerId, sync_send: mpsc::UnboundedSender>, + beacon_processor_send: mpsc::Sender>, chain: Arc>, log: slog::Logger, ) -> Self { @@ -132,6 +137,7 @@ impl SyncingChain { state: ChainSyncingState::Stopped, current_processing_batch: None, sync_send, + beacon_processor_send, chain, log, } @@ -257,16 +263,24 @@ impl SyncingChain { /// Sends a batch to the batch processor. fn process_batch(&mut self, mut batch: Batch) { - let downloaded_blocks = std::mem::replace(&mut batch.downloaded_blocks, Vec::new()); + let blocks = std::mem::replace(&mut batch.downloaded_blocks, Vec::new()); let process_id = ProcessId::RangeBatchId(self.id, batch.id); self.current_processing_batch = Some(batch); - spawn_block_processor( - Arc::downgrade(&self.chain.clone()), - process_id, - downloaded_blocks, - self.sync_send.clone(), - self.log.clone(), - ); + + match self + .beacon_processor_send + .try_send(BeaconWorkEvent::chain_segment(process_id, blocks)) + { + Ok(_) => {} + Err(e) => { + error!( + self.log, + "Failed to send chain segment to processor"; + "msg" => "process_batch", + "error" => format!("{:?}", e) + ); + } + } } /// The block processor has completed processing a batch. This function handles the result diff --git a/beacon_node/network/src/sync/range_sync/chain_collection.rs b/beacon_node/network/src/sync/range_sync/chain_collection.rs index 5d8083a4207..8b2b1c8618f 100644 --- a/beacon_node/network/src/sync/range_sync/chain_collection.rs +++ b/beacon_node/network/src/sync/range_sync/chain_collection.rs @@ -4,6 +4,7 @@ //! with this struct to to simplify the logic of the other layers of sync. use super::chain::{ChainSyncingState, SyncingChain}; +use crate::beacon_processor::WorkEvent as BeaconWorkEvent; use crate::sync::manager::SyncMessage; use crate::sync::network_context::SyncNetworkContext; use crate::sync::PeerSyncInfo; @@ -303,6 +304,7 @@ impl ChainCollection { target_slot: Slot, peer_id: PeerId, sync_send: mpsc::UnboundedSender>, + beacon_processor_send: mpsc::Sender>, ) { let chain_id = rand::random(); self.finalized_chains.push(SyncingChain::new( @@ -312,6 +314,7 @@ impl ChainCollection { target_head, peer_id, sync_send, + beacon_processor_send, self.beacon_chain.clone(), self.log.clone(), )); @@ -327,6 +330,7 @@ impl ChainCollection { target_slot: Slot, peer_id: PeerId, sync_send: mpsc::UnboundedSender>, + beacon_processor_send: mpsc::Sender>, ) { // remove the peer from any other head chains @@ -343,6 +347,7 @@ impl ChainCollection { target_head, peer_id, sync_send, + beacon_processor_send, self.beacon_chain.clone(), self.log.clone(), ); diff --git a/beacon_node/network/src/sync/range_sync/range.rs b/beacon_node/network/src/sync/range_sync/range.rs index f6a1d80e475..49eb06911a4 100644 --- a/beacon_node/network/src/sync/range_sync/range.rs +++ b/beacon_node/network/src/sync/range_sync/range.rs @@ -43,9 +43,10 @@ use super::chain::{ChainId, ProcessingResult}; use super::chain_collection::{ChainCollection, RangeSyncState}; use super::sync_type::RangeSyncType; use super::BatchId; -use crate::sync::block_processor::BatchProcessResult; +use crate::beacon_processor::WorkEvent as BeaconWorkEvent; use crate::sync::manager::SyncMessage; use crate::sync::network_context::SyncNetworkContext; +use crate::sync::BatchProcessResult; use crate::sync::PeerSyncInfo; use crate::sync::RequestId; use beacon_chain::{BeaconChain, BeaconChainTypes}; @@ -72,6 +73,8 @@ pub struct RangeSync { /// The sync manager channel, allowing the batch processor thread to callback the sync task /// once complete. sync_send: mpsc::UnboundedSender>, + /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. + beacon_processor_send: mpsc::Sender>, /// The syncing logger. log: slog::Logger, } @@ -81,6 +84,7 @@ impl RangeSync { beacon_chain: Arc>, network_globals: Arc>, sync_send: mpsc::UnboundedSender>, + beacon_processor_send: mpsc::Sender>, log: slog::Logger, ) -> Self { RangeSync { @@ -88,6 +92,7 @@ impl RangeSync { chains: ChainCollection::new(beacon_chain, network_globals, log.clone()), awaiting_head_peers: HashSet::new(), sync_send, + beacon_processor_send, log, } } @@ -182,6 +187,7 @@ impl RangeSync { remote_finalized_slot, peer_id, self.sync_send.clone(), + self.beacon_processor_send.clone(), ); self.chains.update_finalized(network); // update the global sync state @@ -229,6 +235,7 @@ impl RangeSync { remote_info.head_slot, peer_id, self.sync_send.clone(), + self.beacon_processor_send.clone(), ); } self.chains.update_finalized(network); From fc2a7ab2aaaf5949cb23c2fa4b4a0acf731ec35e Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 08:32:40 +1000 Subject: [PATCH 10/20] Use weak chain reference in beacon processor --- beacon_node/network/src/beacon_processor/mod.rs | 16 +++++++++++++--- beacon_node/network/src/router/processor.rs | 2 +- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 5d0dcfdb3a7..c5a09a9c3a0 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -45,7 +45,7 @@ use eth2_libp2p::{MessageId, NetworkGlobals, PeerId}; use slog::{crit, debug, error, info, trace, warn, Logger}; use ssz::Encode; use std::collections::VecDeque; -use std::sync::Arc; +use std::sync::{Arc, Weak}; use std::time::{Duration, Instant}; use tokio::sync::{mpsc, oneshot}; use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId}; @@ -298,7 +298,7 @@ impl TimeLatch { /// /// See module level documentation for more information. pub struct BeaconProcessor { - pub beacon_chain: Arc>, + pub beacon_chain: Weak>, pub network_tx: mpsc::UnboundedSender>, pub sync_tx: mpsc::UnboundedSender>, pub network_globals: Arc>, @@ -491,7 +491,17 @@ impl BeaconProcessor { metrics::inc_counter(&metrics::GOSSIP_PROCESSOR_WORKERS_SPAWNED_TOTAL); self.current_workers = self.current_workers.saturating_add(1); - let chain = self.beacon_chain.clone(); + + let chain = if let Some(chain) = self.beacon_chain.upgrade() { + chain + } else { + debug!( + self.log, + "Beacon chain dropped, shutting down"; + ); + return; + }; + let network_tx = self.network_tx.clone(); let sync_tx = self.sync_tx.clone(); let log = self.log.clone(); diff --git a/beacon_node/network/src/router/processor.rs b/beacon_node/network/src/router/processor.rs index 0cace588f9d..3a663badd3a 100644 --- a/beacon_node/network/src/router/processor.rs +++ b/beacon_node/network/src/router/processor.rs @@ -64,7 +64,7 @@ impl Processor { ); BeaconProcessor { - beacon_chain: beacon_chain.clone(), + beacon_chain: Arc::downgrade(&beacon_chain), network_tx: network_send.clone(), sync_tx: sync_send.clone(), network_globals, From 627974b063c8b4d73fdbce9c91689c57ebc32664 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 08:47:56 +1000 Subject: [PATCH 11/20] Fix compile errors with chain segement processing --- .../src/beacon_processor/chain_segment.rs | 179 +++++++++--------- .../network/src/beacon_processor/mod.rs | 35 +++- beacon_node/network/src/metrics.rs | 15 +- beacon_node/network/src/sync/manager.rs | 5 - .../network/src/sync/range_sync/chain.rs | 8 +- .../src/sync/range_sync/chain_collection.rs | 5 - .../network/src/sync/range_sync/range.rs | 8 - 7 files changed, 131 insertions(+), 124 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/chain_segment.rs b/beacon_node/network/src/beacon_processor/chain_segment.rs index ed1cce981db..f82d055f1df 100644 --- a/beacon_node/network/src/beacon_processor/chain_segment.rs +++ b/beacon_node/network/src/beacon_processor/chain_segment.rs @@ -4,7 +4,7 @@ use crate::sync::{BatchId, BatchProcessResult, ChainId}; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, ChainSegmentResult}; use eth2_libp2p::PeerId; use slog::{debug, error, trace, warn}; -use std::sync::{Arc, Weak}; +use std::sync::Arc; use tokio::sync::mpsc; use types::{EthSpec, SignedBeaconBlock}; @@ -17,73 +17,71 @@ pub enum ProcessId { ParentLookup(PeerId), } -/// Spawns a thread handling the block processing of a request: range syncing or parent lookup. -pub fn spawn_block_processor( - chain: Weak>, +pub fn handle_chain_segment( + chain: Arc>, process_id: ProcessId, downloaded_blocks: Vec>, sync_send: mpsc::UnboundedSender>, log: slog::Logger, ) { - std::thread::spawn(move || { - match process_id { - // this a request from the range sync - ProcessId::RangeBatchId(chain_id, batch_id) => { - let len = downloaded_blocks.len(); - let start_slot = if len > 0 { - downloaded_blocks[0].message.slot.as_u64() - } else { - 0 - }; - let end_slot = if len > 0 { - downloaded_blocks[len - 1].message.slot.as_u64() - } else { - 0 - }; + match process_id { + // this a request from the range sync + ProcessId::RangeBatchId(chain_id, batch_id) => { + let len = downloaded_blocks.len(); + let start_slot = if len > 0 { + downloaded_blocks[0].message.slot.as_u64() + } else { + 0 + }; + let end_slot = if len > 0 { + downloaded_blocks[len - 1].message.slot.as_u64() + } else { + 0 + }; - debug!(log, "Processing batch"; "id" => *batch_id, "blocks" => downloaded_blocks.len(), "start_slot" => start_slot, "end_slot" => end_slot); - let result = match process_blocks(chain, downloaded_blocks.iter(), &log) { - (_, Ok(_)) => { - debug!(log, "Batch processed"; "id" => *batch_id , "start_slot" => start_slot, "end_slot" => end_slot); - BatchProcessResult::Success - } - (imported_blocks, Err(e)) if imported_blocks > 0 => { - debug!(log, "Batch processing failed but imported some blocks"; + debug!(log, "Processing batch"; "id" => *batch_id, "blocks" => downloaded_blocks.len(), "start_slot" => start_slot, "end_slot" => end_slot); + let result = match process_blocks(chain, downloaded_blocks.iter(), &log) { + (_, Ok(_)) => { + debug!(log, "Batch processed"; "id" => *batch_id , "start_slot" => start_slot, "end_slot" => end_slot); + BatchProcessResult::Success + } + (imported_blocks, Err(e)) if imported_blocks > 0 => { + debug!(log, "Batch processing failed but imported some blocks"; "id" => *batch_id, "error" => e, "imported_blocks"=> imported_blocks); - BatchProcessResult::Partial - } - (_, Err(e)) => { - debug!(log, "Batch processing failed"; "id" => *batch_id, "error" => e); - BatchProcessResult::Failed - } - }; + BatchProcessResult::Partial + } + (_, Err(e)) => { + debug!(log, "Batch processing failed"; "id" => *batch_id, "error" => e); + BatchProcessResult::Failed + } + }; - let msg = SyncMessage::BatchProcessed { - chain_id, - batch_id, - downloaded_blocks, - result, - }; - sync_send.send(msg).unwrap_or_else(|_| { - debug!( - log, - "Block processor could not inform range sync result. Likely shutting down." - ); - }); - } - // this a parent lookup request from the sync manager - ProcessId::ParentLookup(peer_id) => { + let msg = SyncMessage::BatchProcessed { + chain_id, + batch_id, + downloaded_blocks, + result, + }; + sync_send.send(msg).unwrap_or_else(|_| { debug!( - log, "Processing parent lookup"; - "last_peer_id" => format!("{}", peer_id), - "blocks" => downloaded_blocks.len() + log, + "Block processor could not inform range sync result. Likely shutting down." ); - // parent blocks are ordered from highest slot to lowest, so we need to process in - // reverse - match process_blocks(chain, downloaded_blocks.iter().rev(), &log) { - (_, Err(e)) => { - warn!(log, "Parent lookup failed"; "last_peer_id" => format!("{}", peer_id), "error" => e); - sync_send + }); + } + // this a parent lookup request from the sync manager + ProcessId::ParentLookup(peer_id) => { + debug!( + log, "Processing parent lookup"; + "last_peer_id" => format!("{}", peer_id), + "blocks" => downloaded_blocks.len() + ); + // parent blocks are ordered from highest slot to lowest, so we need to process in + // reverse + match process_blocks(chain, downloaded_blocks.iter().rev(), &log) { + (_, Err(e)) => { + warn!(log, "Parent lookup failed"; "last_peer_id" => format!("{}", peer_id), "error" => e); + sync_send .send(SyncMessage::ParentLookupFailed(peer_id)) .unwrap_or_else(|_| { // on failure, inform to downvote the peer @@ -92,14 +90,13 @@ pub fn spawn_block_processor( "Block processor could not inform parent lookup result. Likely shutting down." ); }); - } - (_, Ok(_)) => { - debug!(log, "Parent lookup processed successfully"); - } + } + (_, Ok(_)) => { + debug!(log, "Parent lookup processed successfully"); } } } - }); + } } /// Helper function to process blocks batches which only consumes the chain and blocks to process. @@ -108,43 +105,37 @@ fn process_blocks< T: BeaconChainTypes, I: Iterator>, >( - chain: Weak>, + chain: Arc>, downloaded_blocks: I, log: &slog::Logger, ) -> (usize, Result<(), String>) { - if let Some(chain) = chain.upgrade() { - let blocks = downloaded_blocks.cloned().collect::>(); - let (imported_blocks, r) = match chain.process_chain_segment(blocks) { - ChainSegmentResult::Successful { imported_blocks } => { - if imported_blocks == 0 { - debug!(log, "All blocks already known"); - } else { - debug!( - log, "Imported blocks from network"; - "count" => imported_blocks, - ); - // Batch completed successfully with at least one block, run fork choice. - run_fork_choice(chain, log); - } - - (imported_blocks, Ok(())) - } - ChainSegmentResult::Failed { - imported_blocks, - error, - } => { - let r = handle_failed_chain_segment(error, log); - if imported_blocks > 0 { - run_fork_choice(chain, log); - } - (imported_blocks, r) + let blocks = downloaded_blocks.cloned().collect::>(); + match chain.process_chain_segment(blocks) { + ChainSegmentResult::Successful { imported_blocks } => { + if imported_blocks == 0 { + debug!(log, "All blocks already known"); + } else { + debug!( + log, "Imported blocks from network"; + "count" => imported_blocks, + ); + // Batch completed successfully with at least one block, run fork choice. + run_fork_choice(chain, log); } - }; - return (imported_blocks, r); + (imported_blocks, Ok(())) + } + ChainSegmentResult::Failed { + imported_blocks, + error, + } => { + let r = handle_failed_chain_segment(error, log); + if imported_blocks > 0 { + run_fork_choice(chain, log); + } + (imported_blocks, r) + } } - - (0, Ok(())) } /// Runs fork-choice on a given chain. This is used during block processing after one successful diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index c5a09a9c3a0..44ecdbd0496 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -40,6 +40,7 @@ use beacon_chain::{ attestation_verification::Error as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError, }; +use chain_segment::handle_chain_segment; use environment::TaskExecutor; use eth2_libp2p::{MessageId, NetworkGlobals, PeerId}; use slog::{crit, debug, error, info, trace, warn, Logger}; @@ -77,10 +78,14 @@ const MAX_AGGREGATED_ATTESTATION_QUEUE_LEN: usize = 1_024; /// before we start dropping them. const MAX_GOSSIP_BLOCK_QUEUE_LEN: usize = 1_024; -/// The maximum number of queued `SignedBeaconBlock` objects received during syncing that will be -/// stored before we start dropping them. +/// The maximum number of queued `SignedBeaconBlock` objects received from the network RPC that +/// will be stored before we start dropping them. const MAX_RPC_BLOCK_QUEUE_LEN: usize = 1_024; +/// The maximum number of queued `Vec` objects received during syncing that will +/// be stored before we start dropping them. +const MAX_CHAIN_SEGMENT_QUEUE_LEN: usize = 1_024; + /// The name of the manager tokio task. const MANAGER_TASK_NAME: &str = "beacon_gossip_processor_manager"; /// The name of the worker tokio tasks. @@ -243,8 +248,11 @@ impl WorkEvent { } /// Create a new work event to import `blocks` as a beacon chain segment. - pub fn chain_segment(process_id: ProcessId, block: Vec>) -> Self { - todo!() + pub fn chain_segment(process_id: ProcessId, blocks: Vec>) -> Self { + Self { + drop_during_sync: false, + work: Work::ChainSegment { process_id, blocks }, + } } } @@ -272,6 +280,10 @@ pub enum Work { block: Box>, result_tx: BlockResultSender, }, + ChainSegment { + process_id: ProcessId, + blocks: Vec>, + }, } /// Provides de-bounce functionality for logging. @@ -330,6 +342,8 @@ impl BeaconProcessor { let mut rpc_block_queue = FifoQueue::new(MAX_RPC_BLOCK_QUEUE_LEN); + let mut chain_segment_queue = FifoQueue::new(MAX_CHAIN_SEGMENT_QUEUE_LEN); + let executor = self.executor.clone(); // The manager future will run on the non-blocking executor and delegate tasks to worker @@ -435,6 +449,7 @@ impl BeaconProcessor { Work::GossipAggregate { .. } => aggregate_queue.push(work), Work::GossipBlock { .. } => gossip_block_queue.push(work, &self.log), Work::RpcBlock { .. } => rpc_block_queue.push(work, &self.log), + Work::ChainSegment { .. } => chain_segment_queue.push(work, &self.log), }, } @@ -458,6 +473,10 @@ impl BeaconProcessor { &metrics::GOSSIP_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL, rpc_block_queue.len() as i64, ); + metrics::set_gauge( + &metrics::GOSSIP_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL, + chain_segment_queue.len() as i64, + ); if aggregate_queue.is_full() && aggregate_debounce.elapsed() { error!( @@ -510,10 +529,12 @@ impl BeaconProcessor { executor.spawn_blocking( move || { let _worker_timer = worker_timer; + let inner_log = log.clone(); // We use this closure pattern to avoid using a `return` that prevents the // `idle_tx` message from sending. let handler = || { + let log = inner_log.clone(); match work { /* * Unaggregated attestation verification. @@ -796,6 +817,12 @@ impl BeaconProcessor { crit!(log, "Failed return sync block result"); } } + /* + * Verification for a chain segment (multiple blocks). + */ + Work::ChainSegment { process_id, blocks } => { + handle_chain_segment(chain, process_id, blocks, sync_tx, log) + } }; }; handler(); diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index 68085fb12ca..bc5918c445d 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -90,7 +90,7 @@ lazy_static! { "gossip_processor_gossip_block_imported_total", "Total number of gossip blocks imported to fork choice, etc." ); - // Sync blocks. + // Rpc blocks. pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( "gossip_processor_rpc_block_queue_total", "Count of blocks from the rpc waiting to be verified." @@ -107,6 +107,19 @@ lazy_static! { "gossip_processor_rpc_block_imported_total", "Total number of gossip blocks imported to fork choice, etc." ); + // Chain segments. + pub static ref GOSSIP_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL: Result = try_create_int_gauge( + "gossip_processor_chain_segment_queue_total", + "Count of chain segments from the rpc waiting to be verified." + ); + pub static ref GOSSIP_PROCESSOR_CHAIN_SEGMENT_WORKER_TIME: Result = try_create_histogram( + "gossip_processor_chain_segment_worker_time", + "Time taken for a worker to fully process a chain segment." + ); + pub static ref GOSSIP_PROCESSOR_CHAIN_SEGMENT_ATTEMPTED_TOTAL: Result = try_create_int_counter( + "gossip_processor_chain_segment_imported_total", + "Total number of chain segments attempted." + ); // Unaggregated attestations. pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( "gossip_processor_unaggregated_attestation_queue_total", diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index 34492d05578..2c9cc5add64 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -170,9 +170,6 @@ pub struct SyncManager { /// The logger for the import manager. log: Logger, - /// The sending part of input_channel - sync_send: mpsc::UnboundedSender>, - /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. beacon_processor_send: mpsc::Sender>, } @@ -217,7 +214,6 @@ pub fn spawn( range_sync: RangeSync::new( beacon_chain.clone(), network_globals.clone(), - sync_send.clone(), beacon_processor_send.clone(), log.clone(), ), @@ -228,7 +224,6 @@ pub fn spawn( parent_queue: SmallVec::new(), single_block_lookups: FnvHashMap::default(), log: log.clone(), - sync_send: sync_send.clone(), beacon_processor_send, }; diff --git a/beacon_node/network/src/sync/range_sync/chain.rs b/beacon_node/network/src/sync/range_sync/chain.rs index 26048ff9303..d6c3a965b63 100644 --- a/beacon_node/network/src/sync/range_sync/chain.rs +++ b/beacon_node/network/src/sync/range_sync/chain.rs @@ -1,8 +1,8 @@ use super::batch::{Batch, BatchId, PendingBatches}; use crate::beacon_processor::ProcessId; use crate::beacon_processor::WorkEvent as BeaconWorkEvent; +use crate::sync::RequestId; use crate::sync::{network_context::SyncNetworkContext, BatchProcessResult}; -use crate::sync::{RequestId, SyncMessage}; use beacon_chain::{BeaconChain, BeaconChainTypes}; use eth2_libp2p::{PeerAction, PeerId}; use rand::prelude::*; @@ -85,10 +85,6 @@ pub struct SyncingChain { /// The current processing batch, if any. current_processing_batch: Option>, - /// A send channel to the sync manager. This is given to the batch processor thread to report - /// back once batch processing has completed. - sync_send: mpsc::UnboundedSender>, - /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. beacon_processor_send: mpsc::Sender>, @@ -115,7 +111,6 @@ impl SyncingChain { target_head_slot: Slot, target_head_root: Hash256, peer_id: PeerId, - sync_send: mpsc::UnboundedSender>, beacon_processor_send: mpsc::Sender>, chain: Arc>, log: slog::Logger, @@ -136,7 +131,6 @@ impl SyncingChain { to_be_processed_id: BatchId(1), state: ChainSyncingState::Stopped, current_processing_batch: None, - sync_send, beacon_processor_send, chain, log, diff --git a/beacon_node/network/src/sync/range_sync/chain_collection.rs b/beacon_node/network/src/sync/range_sync/chain_collection.rs index 8b2b1c8618f..2eda6fc81ef 100644 --- a/beacon_node/network/src/sync/range_sync/chain_collection.rs +++ b/beacon_node/network/src/sync/range_sync/chain_collection.rs @@ -5,7 +5,6 @@ use super::chain::{ChainSyncingState, SyncingChain}; use crate::beacon_processor::WorkEvent as BeaconWorkEvent; -use crate::sync::manager::SyncMessage; use crate::sync::network_context::SyncNetworkContext; use crate::sync::PeerSyncInfo; use beacon_chain::{BeaconChain, BeaconChainTypes}; @@ -303,7 +302,6 @@ impl ChainCollection { target_head: Hash256, target_slot: Slot, peer_id: PeerId, - sync_send: mpsc::UnboundedSender>, beacon_processor_send: mpsc::Sender>, ) { let chain_id = rand::random(); @@ -313,7 +311,6 @@ impl ChainCollection { target_slot, target_head, peer_id, - sync_send, beacon_processor_send, self.beacon_chain.clone(), self.log.clone(), @@ -329,7 +326,6 @@ impl ChainCollection { target_head: Hash256, target_slot: Slot, peer_id: PeerId, - sync_send: mpsc::UnboundedSender>, beacon_processor_send: mpsc::Sender>, ) { // remove the peer from any other head chains @@ -346,7 +342,6 @@ impl ChainCollection { target_slot, target_head, peer_id, - sync_send, beacon_processor_send, self.beacon_chain.clone(), self.log.clone(), diff --git a/beacon_node/network/src/sync/range_sync/range.rs b/beacon_node/network/src/sync/range_sync/range.rs index 49eb06911a4..d0a76cf6f99 100644 --- a/beacon_node/network/src/sync/range_sync/range.rs +++ b/beacon_node/network/src/sync/range_sync/range.rs @@ -44,7 +44,6 @@ use super::chain_collection::{ChainCollection, RangeSyncState}; use super::sync_type::RangeSyncType; use super::BatchId; use crate::beacon_processor::WorkEvent as BeaconWorkEvent; -use crate::sync::manager::SyncMessage; use crate::sync::network_context::SyncNetworkContext; use crate::sync::BatchProcessResult; use crate::sync::PeerSyncInfo; @@ -70,9 +69,6 @@ pub struct RangeSync { /// finalized chain(s) complete, these peer's get STATUS'ed to update their head slot before /// the head chains are formed and downloaded. awaiting_head_peers: HashSet, - /// The sync manager channel, allowing the batch processor thread to callback the sync task - /// once complete. - sync_send: mpsc::UnboundedSender>, /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. beacon_processor_send: mpsc::Sender>, /// The syncing logger. @@ -83,7 +79,6 @@ impl RangeSync { pub fn new( beacon_chain: Arc>, network_globals: Arc>, - sync_send: mpsc::UnboundedSender>, beacon_processor_send: mpsc::Sender>, log: slog::Logger, ) -> Self { @@ -91,7 +86,6 @@ impl RangeSync { beacon_chain: beacon_chain.clone(), chains: ChainCollection::new(beacon_chain, network_globals, log.clone()), awaiting_head_peers: HashSet::new(), - sync_send, beacon_processor_send, log, } @@ -186,7 +180,6 @@ impl RangeSync { remote_info.finalized_root, remote_finalized_slot, peer_id, - self.sync_send.clone(), self.beacon_processor_send.clone(), ); self.chains.update_finalized(network); @@ -234,7 +227,6 @@ impl RangeSync { remote_info.head_root, remote_info.head_slot, peer_id, - self.sync_send.clone(), self.beacon_processor_send.clone(), ); } From 49d958c58371dd30a51631cc7735787be5729dec Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 09:01:15 +1000 Subject: [PATCH 12/20] Ensure chain segments are enqueued --- beacon_node/network/src/beacon_processor/mod.rs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 44ecdbd0496..8e5681e1fe6 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -403,10 +403,17 @@ impl BeaconProcessor { match work_event { // There is no new work event, but we are able to spawn a new worker. + // + // We don't check the `work.drop_during_sync` here. We assume that if it made + // it into the queue at any point then we should process it. None if can_spawn => { + // Check for chain segments first, they're the most efficient way to get + // blocks into the system. + if let Some(item) = chain_segment_queue.pop() { + self.spawn_worker(idle_tx.clone(), item); // Check sync blocks before gossip blocks, since we've already explicitly // requested these blocks. - if let Some(item) = rpc_block_queue.pop() { + } else if let Some(item) = rpc_block_queue.pop() { self.spawn_worker(idle_tx.clone(), item); } else if let Some(item) = gossip_block_queue.pop() { self.spawn_worker(idle_tx.clone(), item); @@ -430,7 +437,7 @@ impl BeaconProcessor { "msg" => "no new work and cannot spawn worker" ); } - // There is a new work event, but the chain is syncing. Ignore it. + // The chain is syncing and this event should be dropped during sync. Some(WorkEvent { .. }) if self.network_globals.sync_state.read().is_syncing() && drop_during_sync => From ff81716768062a2853c4a5d40ab220649ca38eea Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 12:29:07 +1000 Subject: [PATCH 13/20] Refactor metrics --- .../src/beacon_processor/chain_segment.rs | 3 + .../network/src/beacon_processor/mod.rs | 107 ++++++++------- beacon_node/network/src/metrics.rs | 126 ++++++++---------- common/lighthouse_metrics/src/lib.rs | 38 +++++- 4 files changed, 160 insertions(+), 114 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/chain_segment.rs b/beacon_node/network/src/beacon_processor/chain_segment.rs index f82d055f1df..5b034e4aa44 100644 --- a/beacon_node/network/src/beacon_processor/chain_segment.rs +++ b/beacon_node/network/src/beacon_processor/chain_segment.rs @@ -1,3 +1,4 @@ +use crate::metrics; use crate::router::processor::FUTURE_SLOT_TOLERANCE; use crate::sync::manager::SyncMessage; use crate::sync::{BatchId, BatchProcessResult, ChainId}; @@ -112,6 +113,7 @@ fn process_blocks< let blocks = downloaded_blocks.cloned().collect::>(); match chain.process_chain_segment(blocks) { ChainSegmentResult::Successful { imported_blocks } => { + metrics::inc_counter(&metrics::BEACON_PROCESSOR_CHAIN_SEGMENT_SUCCESS_TOTAL); if imported_blocks == 0 { debug!(log, "All blocks already known"); } else { @@ -129,6 +131,7 @@ fn process_blocks< imported_blocks, error, } => { + metrics::inc_counter(&metrics::BEACON_PROCESSOR_CHAIN_SEGMENT_FAILED_TOTAL); let r = handle_failed_chain_segment(error, log); if imported_blocks > 0 { run_fork_choice(chain, log); diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 8e5681e1fe6..d4e93790463 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -286,6 +286,19 @@ pub enum Work { }, } +impl Work { + /// Provides a `&str` that uniquely identifies each enum variant. + fn str_id(&self) -> &str { + match self { + Work::GossipAttestation { .. } => "gossip_attestation", + Work::GossipAggregate { .. } => "gossip_aggregate", + Work::GossipBlock { .. } => "gossip_block", + Work::RpcBlock { .. } => "rpc_block", + Work::ChainSegment { .. } => "chain_segment", + } + } +} + /// Provides de-bounce functionality for logging. #[derive(Default)] struct TimeLatch(Option); @@ -359,7 +372,6 @@ impl BeaconProcessor { // A worker has finished some work. new_idle_opt = idle_rx.recv() => { if new_idle_opt.is_some() { - metrics::inc_counter(&metrics::GOSSIP_PROCESSOR_IDLE_EVENTS_TOTAL); self.current_workers = self.current_workers.saturating_sub(1); None } else { @@ -377,7 +389,6 @@ impl BeaconProcessor { // There is a new piece of work to be handled. new_work_event_opt = event_rx.recv() => { if let Some(new_work_event) = new_work_event_opt { - metrics::inc_counter(&metrics::GOSSIP_PROCESSOR_WORK_EVENTS_TOTAL); Some(new_work_event) } else { // Exit if all event senders have been dropped. @@ -394,7 +405,15 @@ impl BeaconProcessor { }; let _event_timer = - metrics::start_timer(&metrics::GOSSIP_PROCESSOR_EVENT_HANDLING_SECONDS); + metrics::start_timer(&metrics::BEACON_PROCESSOR_EVENT_HANDLING_SECONDS); + if let Some(event) = &work_event { + metrics::inc_counter_vec( + &metrics::BEACON_PROCESSOR_WORK_EVENTS_RX_COUNT, + &[event.work.str_id()], + ); + } else { + metrics::inc_counter(&metrics::BEACON_PROCESSOR_IDLE_EVENTS_TOTAL); + } let can_spawn = self.current_workers < self.max_workers; let drop_during_sync = work_event @@ -438,15 +457,20 @@ impl BeaconProcessor { ); } // The chain is syncing and this event should be dropped during sync. - Some(WorkEvent { .. }) + Some(work_event) if self.network_globals.sync_state.read().is_syncing() && drop_during_sync => { - metrics::inc_counter(&metrics::GOSSIP_PROCESSOR_WORK_EVENTS_IGNORED_TOTAL); + let work_id = work_event.work.str_id(); + metrics::inc_counter_vec( + &metrics::BEACON_PROCESSOR_WORK_EVENTS_IGNORED_COUNT, + &[work_id], + ); trace!( self.log, "Gossip processor skipping work"; - "msg" => "chain is syncing" + "msg" => "chain is syncing", + "work_id" => work_id ); } // There is a new work event and the chain is not syncing. Process it. @@ -461,27 +485,27 @@ impl BeaconProcessor { } metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_WORKERS_ACTIVE_TOTAL, + &metrics::BEACON_PROCESSOR_WORKERS_ACTIVE_TOTAL, self.current_workers as i64, ); metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL, + &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL, attestation_queue.len() as i64, ); metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL, + &metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL, aggregate_queue.len() as i64, ); metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL, + &metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL, gossip_block_queue.len() as i64, ); metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL, + &metrics::BEACON_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL, rpc_block_queue.len() as i64, ); metrics::set_gauge( - &metrics::GOSSIP_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL, + &metrics::BEACON_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL, chain_segment_queue.len() as i64, ); @@ -513,8 +537,13 @@ impl BeaconProcessor { /// /// Sends an message on `idle_tx` when the work is complete and the task is stopping. fn spawn_worker(&mut self, mut idle_tx: mpsc::Sender<()>, work: Work) { - let worker_timer = metrics::start_timer(&metrics::GOSSIP_PROCESSOR_WORKER_TIME); - metrics::inc_counter(&metrics::GOSSIP_PROCESSOR_WORKERS_SPAWNED_TOTAL); + let worker_timer = + metrics::start_timer_vec(&metrics::BEACON_PROCESSOR_WORKER_TIME, &[work.str_id()]); + metrics::inc_counter(&metrics::BEACON_PROCESSOR_WORKERS_SPAWNED_TOTAL); + metrics::inc_counter_vec( + &metrics::BEACON_PROCESSOR_WORK_EVENTS_STARTED_COUNT, + &[work.str_id()], + ); self.current_workers = self.current_workers.saturating_add(1); @@ -553,13 +582,6 @@ impl BeaconProcessor { subnet_id, should_import, } => { - let _attestation_timer = metrics::start_timer( - &metrics::GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_WORKER_TIME, - ); - metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL, - ); - let beacon_block_root = attestation.data.beacon_block_root; let attestation = match chain @@ -588,7 +610,7 @@ impl BeaconProcessor { } metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_IMPORTED_TOTAL, + &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL, ); if let Err(e) = chain.apply_attestation_to_fork_choice(&attestation) { @@ -621,6 +643,10 @@ impl BeaconProcessor { "beacon_block_root" => format!("{:?}", beacon_block_root) ) } + + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_IMPORTED_TOTAL, + ); } /* * Aggregated attestation verification. @@ -630,13 +656,6 @@ impl BeaconProcessor { peer_id, aggregate, } => { - let _attestation_timer = metrics::start_timer( - &metrics::GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_WORKER_TIME, - ); - metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_VERIFIED_TOTAL, - ); - let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root; @@ -661,7 +680,7 @@ impl BeaconProcessor { propagate_gossip_message(network_tx, message_id, peer_id.clone(), &log); metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_IMPORTED_TOTAL, + &metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_VERIFIED_TOTAL, ); if let Err(e) = chain.apply_attestation_to_fork_choice(&aggregate) { @@ -694,6 +713,10 @@ impl BeaconProcessor { "beacon_block_root" => format!("{:?}", beacon_block_root) ) } + + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_IMPORTED_TOTAL, + ); } /* * Verification for beacon blocks received on gossip. @@ -703,13 +726,6 @@ impl BeaconProcessor { peer_id, block, } => { - let _block_timer = metrics::start_timer( - &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_WORKER_TIME, - ); - metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, - ); - let verified_block = match chain.verify_block_for_gossip(*block) { Ok(verified_block) => { info!( @@ -745,12 +761,16 @@ impl BeaconProcessor { }; metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, + &metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_VERIFIED_TOTAL, ); let block = Box::new(verified_block.block.clone()); match chain.process_block(verified_block) { Ok(_block_root) => { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, + ); + trace!( log, "Gossipsub block processed"; @@ -805,21 +825,18 @@ impl BeaconProcessor { "ssz" => format!("0x{}", hex::encode(block.as_ssz_bytes())), ); } - } + }; } /* * Verification for beacon blocks received during syncing via RPC. */ Work::RpcBlock { block, result_tx } => { - let _block_timer = metrics::start_timer( - &metrics::GOSSIP_PROCESSOR_RPC_BLOCK_WORKER_TIME, - ); + let block_result = chain.process_block(*block); + metrics::inc_counter( - &metrics::GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL, + &metrics::BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL, ); - let block_result = chain.process_block(*block); - if let Err(_) = result_tx.send(block_result) { crit!(log, "Failed return sync block result"); } diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index bc5918c445d..fa17d5f91fd 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -49,109 +49,101 @@ lazy_static! { /* * Gossip processor */ - pub static ref GOSSIP_PROCESSOR_WORKERS_SPAWNED_TOTAL: Result = try_create_int_counter( - "gossip_processor_workers_spawned_total", + pub static ref BEACON_PROCESSOR_WORK_EVENTS_RX_COUNT: Result = try_create_int_counter_vec( + "beacon_processor_work_events_rx_count", + "Count of work events received (but not necessarily processed)", + &["type"] + ); + pub static ref BEACON_PROCESSOR_WORK_EVENTS_IGNORED_COUNT: Result = try_create_int_counter_vec( + "beacon_processor_work_events_ignored_count", + "Count of work events purposefully ignored", + &["type"] + ); + pub static ref BEACON_PROCESSOR_WORK_EVENTS_STARTED_COUNT: Result = try_create_int_counter_vec( + "beacon_processor_work_events_started_count", + "Count of work events which have been started by a worker", + &["type"] + ); + pub static ref BEACON_PROCESSOR_WORKER_TIME: Result = try_create_histogram_vec( + "beacon_processor_worker_time", + "Time taken for a worker to fully process some parcel of work.", + &["type"] + ); + pub static ref BEACON_PROCESSOR_WORKERS_SPAWNED_TOTAL: Result = try_create_int_counter( + "beacon_processor_workers_spawned_total", "The number of workers ever spawned by the gossip processing pool." ); - pub static ref GOSSIP_PROCESSOR_WORKERS_ACTIVE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_workers_active_total", + pub static ref BEACON_PROCESSOR_WORKERS_ACTIVE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_workers_active_total", "Count of active workers in the gossip processing pool." ); - pub static ref GOSSIP_PROCESSOR_WORK_EVENTS_TOTAL: Result = try_create_int_counter( - "gossip_processor_work_events_total", - "Count of work events processed by the gossip processor manager." - ); - pub static ref GOSSIP_PROCESSOR_WORK_EVENTS_IGNORED_TOTAL: Result = try_create_int_counter( - "gossip_processor_work_events_ignored_total", - "Count of work events processed by the gossip processor manager." - ); - pub static ref GOSSIP_PROCESSOR_IDLE_EVENTS_TOTAL: Result = try_create_int_counter( - "gossip_processor_idle_events_total", + pub static ref BEACON_PROCESSOR_IDLE_EVENTS_TOTAL: Result = try_create_int_counter( + "beacon_processor_idle_events_total", "Count of idle events processed by the gossip processor manager." ); - pub static ref GOSSIP_PROCESSOR_EVENT_HANDLING_SECONDS: Result = try_create_histogram( - "gossip_processor_event_handling_seconds", + pub static ref BEACON_PROCESSOR_EVENT_HANDLING_SECONDS: Result = try_create_histogram( + "beacon_processor_event_handling_seconds", "Time spent handling a new message and allocating it to a queue or worker." ); - pub static ref GOSSIP_PROCESSOR_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_worker_time", - "Time taken for a worker to fully process some parcel of work." - ); // Gossip blocks. - pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_gossip_block_queue_total", + pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_gossip_block_queue_total", "Count of blocks from gossip waiting to be verified." ); - pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_gossip_block_worker_time", - "Time taken for a worker to fully process an gossip block." + pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( + "beacon_processor_gossip_block_verified_total", + "Total number of gossip blocks verified for propagation." ); - pub static ref GOSSIP_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_gossip_block_imported_total", + pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( + "beacon_processor_gossip_block_imported_total", "Total number of gossip blocks imported to fork choice, etc." ); // Rpc blocks. - pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_rpc_block_queue_total", + pub static ref BEACON_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_rpc_block_queue_total", "Count of blocks from the rpc waiting to be verified." ); - pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_rpc_block_worker_time", - "Time taken for a worker to fully process a block from the rpc." - ); - pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( - "gossip_processor_rpc_block_verified_total", - "Total number of verified blocks from the rpc." - ); - pub static ref GOSSIP_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_rpc_block_imported_total", + pub static ref BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL: Result = try_create_int_counter( + "beacon_processor_rpc_block_imported_total", "Total number of gossip blocks imported to fork choice, etc." ); // Chain segments. - pub static ref GOSSIP_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_chain_segment_queue_total", + pub static ref BEACON_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_chain_segment_queue_total", "Count of chain segments from the rpc waiting to be verified." ); - pub static ref GOSSIP_PROCESSOR_CHAIN_SEGMENT_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_chain_segment_worker_time", - "Time taken for a worker to fully process a chain segment." + pub static ref BEACON_PROCESSOR_CHAIN_SEGMENT_SUCCESS_TOTAL: Result = try_create_int_counter( + "beacon_processor_chain_segment_success_total", + "Total number of chain segments successfully processed." ); - pub static ref GOSSIP_PROCESSOR_CHAIN_SEGMENT_ATTEMPTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_chain_segment_imported_total", - "Total number of chain segments attempted." + pub static ref BEACON_PROCESSOR_CHAIN_SEGMENT_FAILED_TOTAL: Result = try_create_int_counter( + "beacon_processor_chain_segment_failed_total", + "Total number of chain segments that failed processing." ); // Unaggregated attestations. - pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_unaggregated_attestation_queue_total", + pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_unaggregated_attestation_queue_total", "Count of unagg. attestations waiting to be processed." ); - pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_unaggregated_attestation_worker_time", - "Time taken for a worker to fully process an unaggregated attestation." - ); - pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result = try_create_int_counter( - "gossip_processor_unaggregated_attestation_verified_total", + pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result = try_create_int_counter( + "beacon_processor_unaggregated_attestation_verified_total", "Total number of unaggregated attestations verified for gossip." ); - pub static ref GOSSIP_PROCESSOR_UNAGGREGATED_ATTESTATION_IMPORTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_unaggregated_attestation_imported_total", + pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_IMPORTED_TOTAL: Result = try_create_int_counter( + "beacon_processor_unaggregated_attestation_imported_total", "Total number of unaggregated attestations imported to fork choice, etc." ); // Aggregated attestations. - pub static ref GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( - "gossip_processor_aggregated_attestation_queue_total", + pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_aggregated_attestation_queue_total", "Count of agg. attestations waiting to be processed." ); - pub static ref GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_WORKER_TIME: Result = try_create_histogram( - "gossip_processor_aggregated_attestation_worker_time", - "Time taken for a worker to fully process an aggregated attestation." - ); - pub static ref GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result = try_create_int_counter( - "gossip_processor_aggregated_attestation_verified_total", + pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result = try_create_int_counter( + "beacon_processor_aggregated_attestation_verified_total", "Total number of aggregated attestations verified for gossip." ); - pub static ref GOSSIP_PROCESSOR_AGGREGATED_ATTESTATION_IMPORTED_TOTAL: Result = try_create_int_counter( - "gossip_processor_aggregated_attestation_imported_total", + pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_IMPORTED_TOTAL: Result = try_create_int_counter( + "beacon_processor_aggregated_attestation_imported_total", "Total number of aggregated attestations imported to fork choice, etc." ); diff --git a/common/lighthouse_metrics/src/lib.rs b/common/lighthouse_metrics/src/lib.rs index a0f59c54b54..187311b68d2 100644 --- a/common/lighthouse_metrics/src/lib.rs +++ b/common/lighthouse_metrics/src/lib.rs @@ -57,8 +57,8 @@ use prometheus::{HistogramOpts, HistogramTimer, Opts}; pub use prometheus::{ - Encoder, Gauge, GaugeVec, Histogram, HistogramVec, IntCounter, IntGauge, IntGaugeVec, Result, - TextEncoder, + Encoder, Gauge, GaugeVec, Histogram, HistogramVec, IntCounter, IntCounterVec, IntGauge, + IntGaugeVec, Result, TextEncoder, }; /// Collect all the metrics for reporting. @@ -141,6 +141,19 @@ pub fn try_create_float_gauge_vec( Ok(counter_vec) } +/// Attempts to crate a `IntGaugeVec`, returning `Err` if the registry does not accept the gauge +/// (potentially due to naming conflict). +pub fn try_create_int_counter_vec( + name: &str, + help: &str, + label_names: &[&str], +) -> Result { + let opts = Opts::new(name, help); + let counter_vec = IntCounterVec::new(opts, label_names)?; + prometheus::register(Box::new(counter_vec.clone()))?; + Ok(counter_vec) +} + pub fn get_int_gauge(int_gauge_vec: &Result, name: &[&str]) -> Option { if let Ok(int_gauge_vec) = int_gauge_vec { Some(int_gauge_vec.get_metric_with_label_values(name).ok()?) @@ -149,6 +162,23 @@ pub fn get_int_gauge(int_gauge_vec: &Result, name: &[&str]) -> Opti } } +pub fn get_int_counter( + int_counter_vec: &Result, + name: &[&str], +) -> Option { + if let Ok(int_counter_vec) = int_counter_vec { + Some(int_counter_vec.get_metric_with_label_values(name).ok()?) + } else { + None + } +} + +pub fn inc_counter_vec(int_counter_vec: &Result, name: &[&str]) { + if let Some(counter) = get_int_counter(int_counter_vec, name) { + counter.inc() + } +} + pub fn get_histogram(histogram_vec: &Result, name: &[&str]) -> Option { if let Ok(histogram_vec) = histogram_vec { Some(histogram_vec.get_metric_with_label_values(name).ok()?) @@ -157,6 +187,10 @@ pub fn get_histogram(histogram_vec: &Result, name: &[&str]) -> Opt } } +pub fn start_timer_vec(vec: &Result, name: &[&str]) -> Option { + get_histogram(vec, name).map(|h| h.start_timer()) +} + /// Starts a timer for the given `Histogram`, stopping when it gets dropped or given to `stop_timer(..)`. pub fn start_timer(histogram: &Result) -> Option { if let Ok(histogram) = histogram { From 8db86ddd4fd9b249d63895e4feb22abe4cbefcc2 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 14:20:26 +1000 Subject: [PATCH 14/20] Add logs to processor --- .../network/src/beacon_processor/mod.rs | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index d4e93790463..76a68b9e949 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -288,7 +288,7 @@ pub enum Work { impl Work { /// Provides a `&str` that uniquely identifies each enum variant. - fn str_id(&self) -> &str { + fn str_id(&self) -> &'static str { match self { Work::GossipAttestation { .. } => "gossip_attestation", Work::GossipAggregate { .. } => "gossip_aggregate", @@ -537,14 +537,16 @@ impl BeaconProcessor { /// /// Sends an message on `idle_tx` when the work is complete and the task is stopping. fn spawn_worker(&mut self, mut idle_tx: mpsc::Sender<()>, work: Work) { + let work_id = work.str_id(); let worker_timer = - metrics::start_timer_vec(&metrics::BEACON_PROCESSOR_WORKER_TIME, &[work.str_id()]); + metrics::start_timer_vec(&metrics::BEACON_PROCESSOR_WORKER_TIME, &[work_id]); metrics::inc_counter(&metrics::BEACON_PROCESSOR_WORKERS_SPAWNED_TOTAL); metrics::inc_counter_vec( &metrics::BEACON_PROCESSOR_WORK_EVENTS_STARTED_COUNT, &[work.str_id()], ); + let worker_id = self.current_workers; self.current_workers = self.current_workers.saturating_add(1); let chain = if let Some(chain) = self.beacon_chain.upgrade() { @@ -562,6 +564,13 @@ impl BeaconProcessor { let log = self.log.clone(); let executor = self.executor.clone(); + trace!( + self.log, + "Spawning beacon processor worker"; + "work" => work_id, + "worker" => worker_id, + ); + executor.spawn_blocking( move || { let _worker_timer = worker_timer; @@ -851,6 +860,13 @@ impl BeaconProcessor { }; handler(); + trace!( + log, + "Beacon processor worker done"; + "work" => work_id, + "worker" => worker_id, + ); + idle_tx.try_send(()).unwrap_or_else(|e| { crit!( log, From 6bcfd97f22e7105f8076973725bb6d71c03a853b Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 14:24:37 +1000 Subject: [PATCH 15/20] Address clippy lints --- beacon_node/network/src/beacon_processor/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 76a68b9e949..77996520317 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -601,7 +601,7 @@ impl BeaconProcessor { handle_attestation_verification_failure( &log, sync_tx, - peer_id.clone(), + peer_id, beacon_block_root, "unaggregated", e, @@ -675,7 +675,7 @@ impl BeaconProcessor { handle_attestation_verification_failure( &log, sync_tx, - peer_id.clone(), + peer_id, beacon_block_root, "aggregated", e, @@ -846,7 +846,7 @@ impl BeaconProcessor { &metrics::BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL, ); - if let Err(_) = result_tx.send(block_result) { + if result_tx.send(block_result).is_err() { crit!(log, "Failed return sync block result"); } } From e53bd7a2bb52255fefbc618dc786afef13293556 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 15:31:54 +1000 Subject: [PATCH 16/20] Add items from self review --- .../network/src/beacon_processor/mod.rs | 43 +++++++++++++------ common/lighthouse_metrics/src/lib.rs | 5 +++ lighthouse/environment/src/executor.rs | 2 +- 3 files changed, 35 insertions(+), 15 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 77996520317..71662bddbbf 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -84,7 +84,7 @@ const MAX_RPC_BLOCK_QUEUE_LEN: usize = 1_024; /// The maximum number of queued `Vec` objects received during syncing that will /// be stored before we start dropping them. -const MAX_CHAIN_SEGMENT_QUEUE_LEN: usize = 1_024; +const MAX_CHAIN_SEGMENT_QUEUE_LEN: usize = 64; /// The name of the manager tokio task. const MANAGER_TASK_NAME: &str = "beacon_gossip_processor_manager"; @@ -114,16 +114,20 @@ impl FifoQueue { } /// Add a new item to the queue. - pub fn push(&mut self, item: T, log: &Logger) { + /// + /// Drops `item` if the queue is full. + pub fn push(&mut self, item: T, item_desc: &str, log: &Logger) { if self.queue.len() == self.max_length { error!( log, - "Gossip block queue full"; + "Block queue full"; "msg" => "the system has insufficient resources for load", "queue_len" => self.max_length, + "queue" => item_desc, ) + } else { + self.queue.push_back(item); } - self.queue.push_back(item); } /// Remove the next item from the queue. @@ -152,7 +156,9 @@ impl LifoQueue { } } - /// Add a new item to the queue. + /// Add a new item to the front of the queue. + /// + /// If the queue is full, the item at the back of the queue is dropped. pub fn push(&mut self, item: T) { if self.queue.len() == self.max_length { self.queue.pop_back(); @@ -256,7 +262,7 @@ impl WorkEvent { } } -/// A consensus message from gossip which requires processing. +/// A consensus message (or multiple) from the network that requires processing. #[derive(Debug)] pub enum Work { GossipAttestation { @@ -434,6 +440,8 @@ impl BeaconProcessor { // requested these blocks. } else if let Some(item) = rpc_block_queue.pop() { self.spawn_worker(idle_tx.clone(), item); + // Check gossip blocks before gossip attestations, since a block might be + // required to verify some attestations. } else if let Some(item) = gossip_block_queue.pop() { self.spawn_worker(idle_tx.clone(), item); // Check the aggregates, *then* the unaggregates @@ -474,14 +482,21 @@ impl BeaconProcessor { ); } // There is a new work event and the chain is not syncing. Process it. - Some(WorkEvent { work, .. }) => match work { - _ if can_spawn => self.spawn_worker(idle_tx.clone(), work), - Work::GossipAttestation { .. } => attestation_queue.push(work), - Work::GossipAggregate { .. } => aggregate_queue.push(work), - Work::GossipBlock { .. } => gossip_block_queue.push(work, &self.log), - Work::RpcBlock { .. } => rpc_block_queue.push(work, &self.log), - Work::ChainSegment { .. } => chain_segment_queue.push(work, &self.log), - }, + Some(WorkEvent { work, .. }) => { + let work_id = work.str_id(); + match work { + _ if can_spawn => self.spawn_worker(idle_tx.clone(), work), + Work::GossipAttestation { .. } => attestation_queue.push(work), + Work::GossipAggregate { .. } => aggregate_queue.push(work), + Work::GossipBlock { .. } => { + gossip_block_queue.push(work, work_id, &self.log) + } + Work::RpcBlock { .. } => rpc_block_queue.push(work, work_id, &self.log), + Work::ChainSegment { .. } => { + chain_segment_queue.push(work, work_id, &self.log) + } + } + } } metrics::set_gauge( diff --git a/common/lighthouse_metrics/src/lib.rs b/common/lighthouse_metrics/src/lib.rs index 187311b68d2..2990eb8cae7 100644 --- a/common/lighthouse_metrics/src/lib.rs +++ b/common/lighthouse_metrics/src/lib.rs @@ -154,6 +154,7 @@ pub fn try_create_int_counter_vec( Ok(counter_vec) } +/// If `int_gauge_vec.is_ok()`, returns a gauge with the given `name`. pub fn get_int_gauge(int_gauge_vec: &Result, name: &[&str]) -> Option { if let Ok(int_gauge_vec) = int_gauge_vec { Some(int_gauge_vec.get_metric_with_label_values(name).ok()?) @@ -162,6 +163,7 @@ pub fn get_int_gauge(int_gauge_vec: &Result, name: &[&str]) -> Opti } } +/// If `int_counter_vec.is_ok()`, returns a counter with the given `name`. pub fn get_int_counter( int_counter_vec: &Result, name: &[&str], @@ -173,12 +175,14 @@ pub fn get_int_counter( } } +/// Increments the `int_counter_vec` with the given `name`. pub fn inc_counter_vec(int_counter_vec: &Result, name: &[&str]) { if let Some(counter) = get_int_counter(int_counter_vec, name) { counter.inc() } } +/// If `histogram_vec.is_ok()`, returns a histogram with the given `name`. pub fn get_histogram(histogram_vec: &Result, name: &[&str]) -> Option { if let Ok(histogram_vec) = histogram_vec { Some(histogram_vec.get_metric_with_label_values(name).ok()?) @@ -187,6 +191,7 @@ pub fn get_histogram(histogram_vec: &Result, name: &[&str]) -> Opt } } +/// Starts a timer on `vec` with the given `name`. pub fn start_timer_vec(vec: &Result, name: &[&str]) -> Option { get_histogram(vec, name).map(|h| h.start_timer()) } diff --git a/lighthouse/environment/src/executor.rs b/lighthouse/environment/src/executor.rs index 26567e1245d..f7d06cc516a 100644 --- a/lighthouse/environment/src/executor.rs +++ b/lighthouse/environment/src/executor.rs @@ -7,7 +7,7 @@ use tokio::runtime::Handle; #[derive(Clone)] pub struct TaskExecutor { /// The handle to the runtime on which tasks are spawned - pub handle: Handle, + pub(crate) handle: Handle, /// The receiver exit future which on receiving shuts down the task pub(crate) exit: exit_future::Exit, pub(crate) log: slog::Logger, From 17501e5eddc82f6a8519d683557f7138f3df1101 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 17:39:59 +1000 Subject: [PATCH 17/20] Addresses most of Michael's comments --- .../network/src/beacon_processor/mod.rs | 2 +- .../network/src/sync/range_sync/chain.rs | 19 ++++++++----------- common/lighthouse_metrics/src/lib.rs | 16 ++++++++-------- 3 files changed, 17 insertions(+), 20 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 71662bddbbf..91d68aeaf32 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -836,7 +836,7 @@ impl BeaconProcessor { ); } other => { - warn!( + debug!( log, "Invalid gossip beacon block"; "outcome" => format!("{:?}", other), diff --git a/beacon_node/network/src/sync/range_sync/chain.rs b/beacon_node/network/src/sync/range_sync/chain.rs index d6c3a965b63..b86ef5e158f 100644 --- a/beacon_node/network/src/sync/range_sync/chain.rs +++ b/beacon_node/network/src/sync/range_sync/chain.rs @@ -255,25 +255,22 @@ impl SyncingChain { } } - /// Sends a batch to the batch processor. + /// Sends a batch to the beacon processor for async processing in a queue. fn process_batch(&mut self, mut batch: Batch) { let blocks = std::mem::replace(&mut batch.downloaded_blocks, Vec::new()); let process_id = ProcessId::RangeBatchId(self.id, batch.id); self.current_processing_batch = Some(batch); - match self + if let Err(e) = self .beacon_processor_send .try_send(BeaconWorkEvent::chain_segment(process_id, blocks)) { - Ok(_) => {} - Err(e) => { - error!( - self.log, - "Failed to send chain segment to processor"; - "msg" => "process_batch", - "error" => format!("{:?}", e) - ); - } + error!( + self.log, + "Failed to send chain segment to processor"; + "msg" => "process_batch", + "error" => format!("{:?}", e) + ); } } diff --git a/common/lighthouse_metrics/src/lib.rs b/common/lighthouse_metrics/src/lib.rs index 2990eb8cae7..d785e0b5669 100644 --- a/common/lighthouse_metrics/src/lib.rs +++ b/common/lighthouse_metrics/src/lib.rs @@ -66,7 +66,7 @@ pub fn gather() -> Vec { prometheus::gather() } -/// Attempts to crate an `IntCounter`, returning `Err` if the registry does not accept the counter +/// Attempts to create an `IntCounter`, returning `Err` if the registry does not accept the counter /// (potentially due to naming conflict). pub fn try_create_int_counter(name: &str, help: &str) -> Result { let opts = Opts::new(name, help); @@ -75,7 +75,7 @@ pub fn try_create_int_counter(name: &str, help: &str) -> Result { Ok(counter) } -/// Attempts to crate an `IntGauge`, returning `Err` if the registry does not accept the counter +/// Attempts to create an `IntGauge`, returning `Err` if the registry does not accept the counter /// (potentially due to naming conflict). pub fn try_create_int_gauge(name: &str, help: &str) -> Result { let opts = Opts::new(name, help); @@ -84,7 +84,7 @@ pub fn try_create_int_gauge(name: &str, help: &str) -> Result { Ok(gauge) } -/// Attempts to crate a `Gauge`, returning `Err` if the registry does not accept the counter +/// Attempts to create a `Gauge`, returning `Err` if the registry does not accept the counter /// (potentially due to naming conflict). pub fn try_create_float_gauge(name: &str, help: &str) -> Result { let opts = Opts::new(name, help); @@ -93,7 +93,7 @@ pub fn try_create_float_gauge(name: &str, help: &str) -> Result { Ok(gauge) } -/// Attempts to crate a `Histogram`, returning `Err` if the registry does not accept the counter +/// Attempts to create a `Histogram`, returning `Err` if the registry does not accept the counter /// (potentially due to naming conflict). pub fn try_create_histogram(name: &str, help: &str) -> Result { let opts = HistogramOpts::new(name, help); @@ -102,7 +102,7 @@ pub fn try_create_histogram(name: &str, help: &str) -> Result { Ok(histogram) } -/// Attempts to crate a `HistogramVec`, returning `Err` if the registry does not accept the counter +/// Attempts to create a `HistogramVec`, returning `Err` if the registry does not accept the counter /// (potentially due to naming conflict). pub fn try_create_histogram_vec( name: &str, @@ -115,7 +115,7 @@ pub fn try_create_histogram_vec( Ok(histogram_vec) } -/// Attempts to crate a `IntGaugeVec`, returning `Err` if the registry does not accept the gauge +/// Attempts to create a `IntGaugeVec`, returning `Err` if the registry does not accept the gauge /// (potentially due to naming conflict). pub fn try_create_int_gauge_vec( name: &str, @@ -128,7 +128,7 @@ pub fn try_create_int_gauge_vec( Ok(counter_vec) } -/// Attempts to crate a `GaugeVec`, returning `Err` if the registry does not accept the gauge +/// Attempts to create a `GaugeVec`, returning `Err` if the registry does not accept the gauge /// (potentially due to naming conflict). pub fn try_create_float_gauge_vec( name: &str, @@ -141,7 +141,7 @@ pub fn try_create_float_gauge_vec( Ok(counter_vec) } -/// Attempts to crate a `IntGaugeVec`, returning `Err` if the registry does not accept the gauge +/// Attempts to create a `IntGaugeVec`, returning `Err` if the registry does not accept the gauge /// (potentially due to naming conflict). pub fn try_create_int_counter_vec( name: &str, From 764ac484b696b29f0acc471e3acbd0bae7837ff6 Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 18:24:56 +1000 Subject: [PATCH 18/20] Add Age's suggestion --- .../network/src/beacon_processor/mod.rs | 32 +++++++++++++++++-- beacon_node/network/src/metrics.rs | 4 +++ 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 91d68aeaf32..b14db18ebaf 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -35,7 +35,11 @@ //! checks the queues to see if there are more parcels of work that can be spawned in a new worker //! task. -use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; +use crate::{ + metrics, + service::NetworkMessage, + sync::{manager::SLOT_IMPORT_TOLERANCE, SyncMessage}, +}; use beacon_chain::{ attestation_verification::Error as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError, @@ -46,10 +50,13 @@ use eth2_libp2p::{MessageId, NetworkGlobals, PeerId}; use slog::{crit, debug, error, info, trace, warn, Logger}; use ssz::Encode; use std::collections::VecDeque; +use std::ops::Sub; use std::sync::{Arc, Weak}; use std::time::{Duration, Instant}; use tokio::sync::{mpsc, oneshot}; -use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId}; +use types::{ + Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, Slot, SubnetId, +}; mod chain_segment; @@ -233,7 +240,7 @@ impl WorkEvent { block: Box>, ) -> Self { Self { - drop_during_sync: true, + drop_during_sync: false, work: Work::GossipBlock { message_id, peer_id, @@ -750,6 +757,25 @@ impl BeaconProcessor { peer_id, block, } => { + let head_slot = chain + .head_info() + .map(|info| info.slot) + .unwrap_or_else(|_| Slot::from(0u64)); + let block_slot = block.message.slot; + + // If the block is far in the future, ignore it. If its within the slot tolerance of + // our current head, regardless of the syncing state, fetch it. + if (head_slot >= block_slot + && head_slot.sub(block_slot).as_usize() > SLOT_IMPORT_TOLERANCE) + || (head_slot < block_slot + && block_slot.sub(head_slot).as_usize() > SLOT_IMPORT_TOLERANCE) + { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IGNORED_TOTAL, + ); + return; + } + let verified_block = match chain.verify_block_for_gossip(*block) { Ok(verified_block) => { info!( diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index fa17d5f91fd..99757e570e8 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -90,6 +90,10 @@ lazy_static! { "beacon_processor_gossip_block_queue_total", "Count of blocks from gossip waiting to be verified." ); + pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_IGNORED_TOTAL: Result = try_create_int_counter( + "beacon_processor_gossip_block_ignored_total", + "Total number of gossip blocks ignored in the worker thread." + ); pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_gossip_block_verified_total", "Total number of gossip blocks verified for propagation." From d768c4ea89a6cd551d6f0302fa29834c36745c8c Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 18:31:17 +1000 Subject: [PATCH 19/20] Revert "Add Age's suggestion" This reverts commit 764ac484b696b29f0acc471e3acbd0bae7837ff6. --- .../network/src/beacon_processor/mod.rs | 32 ++----------------- beacon_node/network/src/metrics.rs | 4 --- 2 files changed, 3 insertions(+), 33 deletions(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index b14db18ebaf..91d68aeaf32 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -35,11 +35,7 @@ //! checks the queues to see if there are more parcels of work that can be spawned in a new worker //! task. -use crate::{ - metrics, - service::NetworkMessage, - sync::{manager::SLOT_IMPORT_TOLERANCE, SyncMessage}, -}; +use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; use beacon_chain::{ attestation_verification::Error as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes, BlockError, ForkChoiceError, @@ -50,13 +46,10 @@ use eth2_libp2p::{MessageId, NetworkGlobals, PeerId}; use slog::{crit, debug, error, info, trace, warn, Logger}; use ssz::Encode; use std::collections::VecDeque; -use std::ops::Sub; use std::sync::{Arc, Weak}; use std::time::{Duration, Instant}; use tokio::sync::{mpsc, oneshot}; -use types::{ - Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, Slot, SubnetId, -}; +use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, SignedBeaconBlock, SubnetId}; mod chain_segment; @@ -240,7 +233,7 @@ impl WorkEvent { block: Box>, ) -> Self { Self { - drop_during_sync: false, + drop_during_sync: true, work: Work::GossipBlock { message_id, peer_id, @@ -757,25 +750,6 @@ impl BeaconProcessor { peer_id, block, } => { - let head_slot = chain - .head_info() - .map(|info| info.slot) - .unwrap_or_else(|_| Slot::from(0u64)); - let block_slot = block.message.slot; - - // If the block is far in the future, ignore it. If its within the slot tolerance of - // our current head, regardless of the syncing state, fetch it. - if (head_slot >= block_slot - && head_slot.sub(block_slot).as_usize() > SLOT_IMPORT_TOLERANCE) - || (head_slot < block_slot - && block_slot.sub(head_slot).as_usize() > SLOT_IMPORT_TOLERANCE) - { - metrics::inc_counter( - &metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IGNORED_TOTAL, - ); - return; - } - let verified_block = match chain.verify_block_for_gossip(*block) { Ok(verified_block) => { info!( diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index 99757e570e8..fa17d5f91fd 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -90,10 +90,6 @@ lazy_static! { "beacon_processor_gossip_block_queue_total", "Count of blocks from gossip waiting to be verified." ); - pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_IGNORED_TOTAL: Result = try_create_int_counter( - "beacon_processor_gossip_block_ignored_total", - "Total number of gossip blocks ignored in the worker thread." - ); pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_gossip_block_verified_total", "Total number of gossip blocks verified for propagation." From 16e52d9ea0e44660cda2b3b3026d9e4d4d1cfe0d Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 17 Aug 2020 18:31:43 +1000 Subject: [PATCH 20/20] Don't drop gossip blocks during sync --- beacon_node/network/src/beacon_processor/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/network/src/beacon_processor/mod.rs index 91d68aeaf32..33022c0ad7d 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/network/src/beacon_processor/mod.rs @@ -233,7 +233,7 @@ impl WorkEvent { block: Box>, ) -> Self { Self { - drop_during_sync: true, + drop_during_sync: false, work: Work::GossipBlock { message_id, peer_id,