From c25825a5393aca2cde6e33dd673f8c074c2b543b Mon Sep 17 00:00:00 2001 From: Paul Hauner Date: Mon, 10 Jul 2023 07:45:54 +0000 Subject: [PATCH] Move the `BeaconProcessor` into a new crate (#4435) *Replaces #4434. It is identical, but this PR has a smaller diff due to a curated commit history.* ## Issue Addressed NA ## Proposed Changes This PR moves the scheduling logic for the `BeaconProcessor` into a new crate in `beacon_node/beacon_processor`. Previously it existed in the `beacon_node/network` crate. This addresses a circular-dependency problem where it's not possible to use the `BeaconProcessor` from the `beacon_chain` crate. The `network` crate depends on the `beacon_chain` crate (`network -> beacon_chain`), but importing the `BeaconProcessor` into the `beacon_chain` crate would create a circular dependancy of `beacon_chain -> network`. The `BeaconProcessor` was designed to provide queuing and prioritized scheduling for messages from the network. It has proven to be quite valuable and I believe we'd make Lighthouse more stable and effective by using it elsewhere. In particular, I think we should use the `BeaconProcessor` for: 1. HTTP API requests. 1. Scheduled tasks in the `BeaconChain` (e.g., state advance). Using the `BeaconProcessor` for these tasks would help prevent the BN from becoming overwhelmed and would also help it to prioritize operations (e.g., choosing to process blocks from gossip before responding to low-priority HTTP API requests). ## Additional Info This PR is intended to have zero impact on runtime behaviour. It aims to simply separate the *scheduling* code (i.e., the `BeaconProcessor`) from the *business logic* in the `network` crate (i.e., the `Worker` impls). Future PRs (see #4462) can build upon these works to actually use the `BeaconProcessor` for more operations. I've gone to some effort to use `git mv` to make the diff look more like "file was moved and modified" rather than "file was deleted and a new one added". This should reduce review burden and help maintain commit attribution. --- Cargo.lock | 28 + Cargo.toml | 1 + beacon_node/Cargo.toml | 2 +- beacon_node/beacon_processor/Cargo.toml | 24 + .../mod.rs => beacon_processor/src/lib.rs} | 1264 ++++------------- beacon_node/beacon_processor/src/metrics.rs | 141 ++ .../src}/work_reprocessing_queue.rs | 243 ++-- beacon_node/client/Cargo.toml | 2 + beacon_node/client/src/builder.rs | 42 +- beacon_node/network/Cargo.toml | 6 +- .../src/beacon_processor/worker/mod.rs | 51 - beacon_node/network/src/lib.rs | 2 +- beacon_node/network/src/metrics.rs | 117 +- .../gossip_methods.rs | 202 ++- .../src/network_beacon_processor/mod.rs | 590 ++++++++ .../rpc_methods.rs | 14 +- .../sync_methods.rs | 109 +- .../tests.rs | 139 +- beacon_node/network/src/router.rs | 258 ++-- beacon_node/network/src/service.rs | 7 +- beacon_node/network/src/service/tests.rs | 23 +- .../network/src/sync/backfill_sync/mod.rs | 6 +- .../network/src/sync/block_lookups/mod.rs | 20 +- .../network/src/sync/block_lookups/tests.rs | 21 +- beacon_node/network/src/sync/manager.rs | 45 +- .../network/src/sync/network_context.rs | 29 +- .../network/src/sync/range_sync/chain.rs | 10 +- .../network/src/sync/range_sync/range.rs | 17 +- 28 files changed, 1741 insertions(+), 1672 deletions(-) create mode 100644 beacon_node/beacon_processor/Cargo.toml rename beacon_node/{network/src/beacon_processor/mod.rs => beacon_processor/src/lib.rs} (58%) create mode 100644 beacon_node/beacon_processor/src/metrics.rs rename beacon_node/{network/src/beacon_processor => beacon_processor/src}/work_reprocessing_queue.rs (85%) delete mode 100644 beacon_node/network/src/beacon_processor/worker/mod.rs rename beacon_node/network/src/{beacon_processor/worker => network_beacon_processor}/gossip_methods.rs (95%) create mode 100644 beacon_node/network/src/network_beacon_processor/mod.rs rename beacon_node/network/src/{beacon_processor/worker => network_beacon_processor}/rpc_methods.rs (98%) rename beacon_node/network/src/{beacon_processor/worker => network_beacon_processor}/sync_methods.rs (90%) rename beacon_node/network/src/{beacon_processor => network_beacon_processor}/tests.rs (89%) diff --git a/Cargo.lock b/Cargo.lock index e360bdd62e2..532f7ff2042 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -700,6 +700,30 @@ dependencies = [ "unused_port", ] +[[package]] +name = "beacon_processor" +version = "0.1.0" +dependencies = [ + "derivative", + "ethereum_ssz", + "fnv", + "futures", + "hex", + "itertools", + "lazy_static", + "lighthouse_metrics", + "lighthouse_network", + "logging", + "parking_lot 0.12.1", + "slog", + "slot_clock", + "strum", + "task_executor", + "tokio", + "tokio-util 0.6.10", + "types", +] + [[package]] name = "bincode" version = "1.3.3" @@ -1148,6 +1172,7 @@ name = "client" version = "0.2.0" dependencies = [ "beacon_chain", + "beacon_processor", "directory", "dirs", "environment", @@ -1165,6 +1190,7 @@ dependencies = [ "logging", "monitoring_api", "network", + "num_cpus", "operation_pool", "parking_lot 0.12.1", "sensitive_url", @@ -5405,6 +5431,7 @@ name = "network" version = "0.2.0" dependencies = [ "beacon_chain", + "beacon_processor", "delay_map", "derivative", "environment", @@ -5428,6 +5455,7 @@ dependencies = [ "matches", "num_cpus", "operation_pool", + "parking_lot 0.12.1", "rand 0.8.5", "rlp", "slog", diff --git a/Cargo.toml b/Cargo.toml index 5c39e01ed13..775842a8c7a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,6 +4,7 @@ members = [ "beacon_node", "beacon_node/beacon_chain", + "beacon_node/beacon_processor", "beacon_node/builder_client", "beacon_node/client", "beacon_node/eth1", diff --git a/beacon_node/Cargo.toml b/beacon_node/Cargo.toml index 7c74365418b..f55c724dc39 100644 --- a/beacon_node/Cargo.toml +++ b/beacon_node/Cargo.toml @@ -41,4 +41,4 @@ monitoring_api = { path = "../common/monitoring_api" } sensitive_url = { path = "../common/sensitive_url" } http_api = { path = "http_api" } unused_port = { path = "../common/unused_port" } -strum = "0.24.1" +strum = "0.24.1" \ No newline at end of file diff --git a/beacon_node/beacon_processor/Cargo.toml b/beacon_node/beacon_processor/Cargo.toml new file mode 100644 index 00000000000..5c5200e101e --- /dev/null +++ b/beacon_node/beacon_processor/Cargo.toml @@ -0,0 +1,24 @@ +[package] +name = "beacon_processor" +version = "0.1.0" +edition = "2021" + +[dependencies] +slog = { version = "2.5.2", features = ["max_level_trace"] } +itertools = "0.10.0" +logging = { path = "../../common/logging" } +tokio = { version = "1.14.0", features = ["full"] } +tokio-util = { version = "0.6.3", features = ["time"] } +futures = "0.3.7" +fnv = "1.0.7" +strum = "0.24.0" +task_executor = { path = "../../common/task_executor" } +slot_clock = { path = "../../common/slot_clock" } +lighthouse_network = { path = "../lighthouse_network" } +hex = "0.4.2" +derivative = "2.2.0" +types = { path = "../../consensus/types" } +ethereum_ssz = "0.5.0" +lazy_static = "1.4.0" +lighthouse_metrics = { path = "../../common/lighthouse_metrics" } +parking_lot = "0.12.0" \ No newline at end of file diff --git a/beacon_node/network/src/beacon_processor/mod.rs b/beacon_node/beacon_processor/src/lib.rs similarity index 58% rename from beacon_node/network/src/beacon_processor/mod.rs rename to beacon_node/beacon_processor/src/lib.rs index 84d8e1b07a8..88066f2a305 100644 --- a/beacon_node/network/src/beacon_processor/mod.rs +++ b/beacon_node/beacon_processor/src/lib.rs @@ -38,51 +38,34 @@ //! checks the queues to see if there are more parcels of work that can be spawned in a new worker //! task. -use crate::sync::manager::BlockProcessType; -use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; -use beacon_chain::parking_lot::Mutex; -use beacon_chain::{BeaconChain, BeaconChainTypes, GossipVerifiedBlock, NotifyExecutionLayer}; -use derivative::Derivative; +use crate::work_reprocessing_queue::{ + spawn_reprocess_scheduler, QueuedAggregate, QueuedBackfillBatch, QueuedGossipBlock, + QueuedLightClientUpdate, QueuedRpcBlock, QueuedUnaggregate, ReadyWork, ReprocessQueueMessage, +}; use futures::stream::{Stream, StreamExt}; use futures::task::Poll; -use lighthouse_network::rpc::LightClientBootstrapRequest; -use lighthouse_network::{ - rpc::{BlocksByRangeRequest, BlocksByRootRequest, StatusMessage}, - Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, -}; +use lighthouse_network::NetworkGlobals; +use lighthouse_network::{MessageId, PeerId}; use logging::TimeLatch; +use parking_lot::Mutex; use slog::{crit, debug, error, trace, warn, Logger}; -use std::collections::VecDeque; +use slot_clock::SlotClock; +use std::cmp; +use std::collections::{HashSet, VecDeque}; +use std::fmt; use std::future::Future; -use std::path::PathBuf; use std::pin::Pin; -use std::sync::{Arc, Weak}; +use std::sync::Arc; use std::task::Context; use std::time::Duration; -use std::{cmp, collections::HashSet}; use task_executor::TaskExecutor; use tokio::sync::mpsc; use tokio::sync::mpsc::error::TrySendError; -use types::{ - Attestation, AttesterSlashing, Hash256, LightClientFinalityUpdate, LightClientOptimisticUpdate, - ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange, - SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId, -}; -use work_reprocessing_queue::{ - spawn_reprocess_scheduler, QueuedAggregate, QueuedLightClientUpdate, QueuedRpcBlock, - QueuedUnaggregate, ReadyWork, -}; - -use worker::{Toolbox, Worker}; - -mod tests; -mod work_reprocessing_queue; -mod worker; +use types::{Attestation, EthSpec, Hash256, SignedAggregateAndProof, Slot, SubnetId}; +use work_reprocessing_queue::IgnoredRpcBlock; -use crate::beacon_processor::work_reprocessing_queue::{ - QueuedBackfillBatch, QueuedGossipBlock, ReprocessQueueMessage, -}; -pub use worker::{ChainSegmentProcessId, GossipAggregatePackage, GossipAttestationPackage}; +mod metrics; +pub mod work_reprocessing_queue; /// The maximum size of the channel for work events to the `BeaconProcessor`. /// @@ -96,7 +79,7 @@ pub const MAX_WORK_EVENT_QUEUE_LEN: usize = 16_384; const MAX_IDLE_QUEUE_LEN: usize = 16_384; /// The maximum size of the channel for re-processing work events. -const MAX_SCHEDULED_WORK_QUEUE_LEN: usize = 3 * MAX_WORK_EVENT_QUEUE_LEN / 4; +pub const MAX_SCHEDULED_WORK_QUEUE_LEN: usize = 3 * MAX_WORK_EVENT_QUEUE_LEN / 4; /// The maximum number of queued `Attestation` objects that will be stored before we start dropping /// them. @@ -221,6 +204,7 @@ pub const GOSSIP_SYNC_CONTRIBUTION: &str = "gossip_sync_contribution"; pub const GOSSIP_LIGHT_CLIENT_FINALITY_UPDATE: &str = "light_client_finality_update"; pub const GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE: &str = "light_client_optimistic_update"; pub const RPC_BLOCK: &str = "rpc_block"; +pub const IGNORED_RPC_BLOCK: &str = "ignored_rpc_block"; pub const CHAIN_SEGMENT: &str = "chain_segment"; pub const CHAIN_SEGMENT_BACKFILL: &str = "chain_segment_backfill"; pub const STATUS_PROCESSING: &str = "status_processing"; @@ -366,394 +350,104 @@ impl DuplicateCache { } /// An event to be processed by the manager task. -#[derive(Derivative)] -#[derivative(Debug(bound = "T: BeaconChainTypes"))] -pub struct WorkEvent { - drop_during_sync: bool, - work: Work, +#[derive(Debug)] +pub struct WorkEvent { + pub drop_during_sync: bool, + pub work: Work, } -impl WorkEvent { - /// Create a new `Work` event for some unaggregated attestation. - pub fn unaggregated_attestation( - message_id: MessageId, - peer_id: PeerId, - attestation: Attestation, - subnet_id: SubnetId, - should_import: bool, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: true, - work: Work::GossipAttestation { - message_id, - peer_id, - attestation: Box::new(attestation), - subnet_id, - should_import, - seen_timestamp, - }, - } - } - - /// Create a new `Work` event for some aggregated attestation. - pub fn aggregated_attestation( - message_id: MessageId, - peer_id: PeerId, - aggregate: SignedAggregateAndProof, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: true, - work: Work::GossipAggregate { - message_id, - peer_id, - aggregate: Box::new(aggregate), - seen_timestamp, - }, - } - } - - /// Create a new `Work` event for some block. - pub fn gossip_beacon_block( - message_id: MessageId, - peer_id: PeerId, - peer_client: Client, - block: Arc>, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::GossipBlock { - message_id, - peer_id, - peer_client, - block, - seen_timestamp, - }, - } - } - - /// Create a new `Work` event for some sync committee signature. - pub fn gossip_sync_signature( - message_id: MessageId, - peer_id: PeerId, - sync_signature: SyncCommitteeMessage, - subnet_id: SyncSubnetId, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: true, - work: Work::GossipSyncSignature { - message_id, - peer_id, - sync_signature: Box::new(sync_signature), - subnet_id, - seen_timestamp, - }, - } - } - - /// Create a new `Work` event for some sync committee contribution. - pub fn gossip_sync_contribution( - message_id: MessageId, - peer_id: PeerId, - sync_contribution: SignedContributionAndProof, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: true, - work: Work::GossipSyncContribution { - message_id, - peer_id, - sync_contribution: Box::new(sync_contribution), - seen_timestamp, - }, - } - } - - /// Create a new `Work` event for some exit. - pub fn gossip_voluntary_exit( - message_id: MessageId, - peer_id: PeerId, - voluntary_exit: Box, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::GossipVoluntaryExit { - message_id, - peer_id, - voluntary_exit, - }, - } - } - - /// Create a new `Work` event for some proposer slashing. - pub fn gossip_proposer_slashing( - message_id: MessageId, - peer_id: PeerId, - proposer_slashing: Box, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::GossipProposerSlashing { - message_id, - peer_id, - proposer_slashing, - }, - } - } - - /// Create a new `Work` event for some light client finality update. - pub fn gossip_light_client_finality_update( - message_id: MessageId, - peer_id: PeerId, - light_client_finality_update: Box>, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: true, - work: Work::GossipLightClientFinalityUpdate { - message_id, - peer_id, - light_client_finality_update, - seen_timestamp, - }, - } - } - - /// Create a new `Work` event for some light client optimistic update. - pub fn gossip_light_client_optimistic_update( - message_id: MessageId, - peer_id: PeerId, - light_client_optimistic_update: Box>, - seen_timestamp: Duration, - ) -> Self { - Self { - drop_during_sync: true, - work: Work::GossipLightClientOptimisticUpdate { - message_id, - peer_id, - light_client_optimistic_update, - seen_timestamp, - }, - } - } - - /// Create a new `Work` event for some attester slashing. - pub fn gossip_attester_slashing( - message_id: MessageId, - peer_id: PeerId, - attester_slashing: Box>, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::GossipAttesterSlashing { - message_id, - peer_id, - attester_slashing, - }, - } - } - - /// Create a new `Work` event for some BLS to execution change. - pub fn gossip_bls_to_execution_change( - message_id: MessageId, - peer_id: PeerId, - bls_to_execution_change: Box, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::GossipBlsToExecutionChange { - message_id, - peer_id, - bls_to_execution_change, - }, - } - } - - /// 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 rpc_beacon_block( - block_root: Hash256, - block: Arc>, - seen_timestamp: Duration, - process_type: BlockProcessType, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::RpcBlock { - block_root, - block, - seen_timestamp, - process_type, - should_process: true, - }, - } - } - - /// Create a new work event to import `blocks` as a beacon chain segment. - pub fn chain_segment( - process_id: ChainSegmentProcessId, - blocks: Vec>>, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::ChainSegment { process_id, blocks }, - } - } - - /// Create a new work event to process `StatusMessage`s from the RPC network. - pub fn status_message(peer_id: PeerId, message: StatusMessage) -> Self { - Self { - drop_during_sync: false, - work: Work::Status { peer_id, message }, - } - } - - /// Create a new work event to process `BlocksByRangeRequest`s from the RPC network. - pub fn blocks_by_range_request( - peer_id: PeerId, - request_id: PeerRequestId, - request: BlocksByRangeRequest, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::BlocksByRangeRequest { - peer_id, - request_id, - request, - }, - } - } - - /// Create a new work event to process `BlocksByRootRequest`s from the RPC network. - pub fn blocks_by_roots_request( - peer_id: PeerId, - request_id: PeerRequestId, - request: BlocksByRootRequest, - ) -> Self { - Self { - drop_during_sync: false, - work: Work::BlocksByRootsRequest { - peer_id, - request_id, - request, - }, - } - } - - /// Create a new work event to process `LightClientBootstrap`s from the RPC network. - pub fn lightclient_bootstrap_request( - peer_id: PeerId, - request_id: PeerRequestId, - request: LightClientBootstrapRequest, - ) -> Self { - Self { - drop_during_sync: true, - work: Work::LightClientBootstrapRequest { - peer_id, - request_id, - request, - }, - } - } - +impl WorkEvent { /// Get a `str` representation of the type of work this `WorkEvent` contains. pub fn work_type(&self) -> &'static str { self.work.str_id() } } -impl std::convert::From> for WorkEvent { - fn from(ready_work: ReadyWork) -> Self { +impl std::convert::From for WorkEvent { + fn from(ready_work: ReadyWork) -> Self { match ready_work { ReadyWork::Block(QueuedGossipBlock { - peer_id, - block, - seen_timestamp, + beacon_block_slot, + beacon_block_root, + process_fn, }) => Self { drop_during_sync: false, work: Work::DelayedImportBlock { - peer_id, - block, - seen_timestamp, + beacon_block_slot, + beacon_block_root, + process_fn, }, }, ReadyWork::RpcBlock(QueuedRpcBlock { - block_root, - block, - seen_timestamp, - process_type, - should_process, + beacon_block_root: _, + process_fn, + ignore_fn: _, }) => Self { drop_during_sync: false, - work: Work::RpcBlock { - block_root, - block, - seen_timestamp, - process_type, - should_process, - }, + work: Work::RpcBlock { process_fn }, + }, + ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock { process_fn }) => Self { + drop_during_sync: false, + work: Work::IgnoredRpcBlock { process_fn }, }, ReadyWork::Unaggregate(QueuedUnaggregate { - peer_id, - message_id, - attestation, - subnet_id, - should_import, - seen_timestamp, + beacon_block_root: _, + process_fn, }) => Self { drop_during_sync: true, - work: Work::UnknownBlockAttestation { - message_id, - peer_id, - attestation, - subnet_id, - should_import, - seen_timestamp, - }, + work: Work::UnknownBlockAttestation { process_fn }, }, ReadyWork::Aggregate(QueuedAggregate { - peer_id, - message_id, - attestation, - seen_timestamp, + process_fn, + beacon_block_root: _, }) => Self { drop_during_sync: true, - work: Work::UnknownBlockAggregate { - message_id, - peer_id, - aggregate: attestation, - seen_timestamp, - }, + work: Work::UnknownBlockAggregate { process_fn }, }, ReadyWork::LightClientUpdate(QueuedLightClientUpdate { - peer_id, - message_id, - light_client_optimistic_update, - seen_timestamp, - .. + parent_root, + process_fn, }) => Self { drop_during_sync: true, work: Work::UnknownLightClientOptimisticUpdate { - message_id, - peer_id, - light_client_optimistic_update, - seen_timestamp, + parent_root, + process_fn, }, }, - ReadyWork::BackfillSync(QueuedBackfillBatch { process_id, blocks }) => { - WorkEvent::chain_segment(process_id, blocks) - } + ReadyWork::BackfillSync(QueuedBackfillBatch(process_fn)) => Self { + drop_during_sync: false, + work: Work::ChainSegmentBackfill(process_fn), + }, } } } -pub struct BeaconProcessorSend(pub mpsc::Sender>); +/// Items required to verify a batch of unaggregated gossip attestations. +#[derive(Debug)] +pub struct GossipAttestationPackage { + pub message_id: MessageId, + pub peer_id: PeerId, + pub attestation: Box>, + pub subnet_id: SubnetId, + pub should_import: bool, + pub seen_timestamp: Duration, +} + +/// Items required to verify a batch of aggregated gossip attestations. +#[derive(Debug)] +pub struct GossipAggregatePackage { + pub message_id: MessageId, + pub peer_id: PeerId, + pub aggregate: Box>, + pub beacon_block_root: Hash256, + pub seen_timestamp: Duration, +} -impl BeaconProcessorSend { - pub fn try_send(&self, message: WorkEvent) -> Result<(), Box>>> { +#[derive(Clone)] +pub struct BeaconProcessorSend(pub mpsc::Sender>); + +impl BeaconProcessorSend { + pub fn try_send(&self, message: WorkEvent) -> Result<(), TrySendError>> { let work_type = message.work_type(); match self.0.try_send(message) { Ok(res) => Ok(res), @@ -762,146 +456,82 @@ impl BeaconProcessorSend { &metrics::BEACON_PROCESSOR_SEND_ERROR_PER_WORK_TYPE, &[work_type], ); - Err(Box::new(e)) + Err(e) } } } } -/// A consensus message (or multiple) from the network that requires processing. -#[derive(Derivative)] -#[derivative(Debug(bound = "T: BeaconChainTypes"))] -pub enum Work { +pub type AsyncFn = Pin + Send + Sync>>; +pub type BlockingFn = Box; +pub type BlockingFnWithManualSendOnIdle = Box; + +/// Indicates the type of work to be performed and therefore its priority and +/// queuing specifics. +pub enum Work { GossipAttestation { - message_id: MessageId, - peer_id: PeerId, - attestation: Box>, - subnet_id: SubnetId, - should_import: bool, - seen_timestamp: Duration, + attestation: GossipAttestationPackage, + process_individual: Box) + Send + Sync>, + process_batch: Box>) + Send + Sync>, }, UnknownBlockAttestation { - message_id: MessageId, - peer_id: PeerId, - attestation: Box>, - subnet_id: SubnetId, - should_import: bool, - seen_timestamp: Duration, + process_fn: BlockingFn, }, GossipAttestationBatch { - packages: Vec>, + attestations: Vec>, + process_batch: Box>) + Send + Sync>, }, GossipAggregate { - message_id: MessageId, - peer_id: PeerId, - aggregate: Box>, - seen_timestamp: Duration, + aggregate: GossipAggregatePackage, + process_individual: Box) + Send + Sync>, + process_batch: Box>) + Send + Sync>, }, UnknownBlockAggregate { - message_id: MessageId, - peer_id: PeerId, - aggregate: Box>, - seen_timestamp: Duration, + process_fn: BlockingFn, }, UnknownLightClientOptimisticUpdate { - message_id: MessageId, - peer_id: PeerId, - light_client_optimistic_update: Box>, - seen_timestamp: Duration, + parent_root: Hash256, + process_fn: BlockingFn, }, GossipAggregateBatch { - packages: Vec>, - }, - GossipBlock { - message_id: MessageId, - peer_id: PeerId, - peer_client: Client, - block: Arc>, - seen_timestamp: Duration, + aggregates: Vec>, + process_batch: Box>) + Send + Sync>, }, + GossipBlock(AsyncFn), DelayedImportBlock { - peer_id: PeerId, - block: Box>, - seen_timestamp: Duration, - }, - GossipVoluntaryExit { - message_id: MessageId, - peer_id: PeerId, - voluntary_exit: Box, - }, - GossipProposerSlashing { - message_id: MessageId, - peer_id: PeerId, - proposer_slashing: Box, - }, - GossipAttesterSlashing { - message_id: MessageId, - peer_id: PeerId, - attester_slashing: Box>, - }, - GossipSyncSignature { - message_id: MessageId, - peer_id: PeerId, - sync_signature: Box, - subnet_id: SyncSubnetId, - seen_timestamp: Duration, - }, - GossipSyncContribution { - message_id: MessageId, - peer_id: PeerId, - sync_contribution: Box>, - seen_timestamp: Duration, - }, - GossipLightClientFinalityUpdate { - message_id: MessageId, - peer_id: PeerId, - light_client_finality_update: Box>, - seen_timestamp: Duration, - }, - GossipLightClientOptimisticUpdate { - message_id: MessageId, - peer_id: PeerId, - light_client_optimistic_update: Box>, - seen_timestamp: Duration, + beacon_block_slot: Slot, + beacon_block_root: Hash256, + process_fn: AsyncFn, }, + GossipVoluntaryExit(BlockingFn), + GossipProposerSlashing(BlockingFn), + GossipAttesterSlashing(BlockingFn), + GossipSyncSignature(BlockingFn), + GossipSyncContribution(BlockingFn), + GossipLightClientFinalityUpdate(BlockingFn), + GossipLightClientOptimisticUpdate(BlockingFn), RpcBlock { - block_root: Hash256, - block: Arc>, - seen_timestamp: Duration, - process_type: BlockProcessType, - should_process: bool, + process_fn: AsyncFn, }, - ChainSegment { - process_id: ChainSegmentProcessId, - blocks: Vec>>, - }, - Status { - peer_id: PeerId, - message: StatusMessage, - }, - BlocksByRangeRequest { - peer_id: PeerId, - request_id: PeerRequestId, - request: BlocksByRangeRequest, - }, - BlocksByRootsRequest { - peer_id: PeerId, - request_id: PeerRequestId, - request: BlocksByRootRequest, - }, - GossipBlsToExecutionChange { - message_id: MessageId, - peer_id: PeerId, - bls_to_execution_change: Box, - }, - LightClientBootstrapRequest { - peer_id: PeerId, - request_id: PeerRequestId, - request: LightClientBootstrapRequest, + IgnoredRpcBlock { + process_fn: BlockingFn, }, + ChainSegment(AsyncFn), + ChainSegmentBackfill(AsyncFn), + Status(BlockingFn), + BlocksByRangeRequest(BlockingFnWithManualSendOnIdle), + BlocksByRootsRequest(BlockingFnWithManualSendOnIdle), + GossipBlsToExecutionChange(BlockingFn), + LightClientBootstrapRequest(BlockingFn), } -impl Work { +impl fmt::Debug for Work { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.str_id()) + } +} + +impl Work { /// Provides a `&str` that uniquely identifies each enum variant. fn str_id(&self) -> &'static str { match self { @@ -909,58 +539,56 @@ impl Work { Work::GossipAttestationBatch { .. } => GOSSIP_ATTESTATION_BATCH, Work::GossipAggregate { .. } => GOSSIP_AGGREGATE, Work::GossipAggregateBatch { .. } => GOSSIP_AGGREGATE_BATCH, - Work::GossipBlock { .. } => GOSSIP_BLOCK, + Work::GossipBlock(_) => GOSSIP_BLOCK, Work::DelayedImportBlock { .. } => DELAYED_IMPORT_BLOCK, - Work::GossipVoluntaryExit { .. } => GOSSIP_VOLUNTARY_EXIT, - Work::GossipProposerSlashing { .. } => GOSSIP_PROPOSER_SLASHING, - Work::GossipAttesterSlashing { .. } => GOSSIP_ATTESTER_SLASHING, - Work::GossipSyncSignature { .. } => GOSSIP_SYNC_SIGNATURE, - Work::GossipSyncContribution { .. } => GOSSIP_SYNC_CONTRIBUTION, - Work::GossipLightClientFinalityUpdate { .. } => GOSSIP_LIGHT_CLIENT_FINALITY_UPDATE, - Work::GossipLightClientOptimisticUpdate { .. } => GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE, + Work::GossipVoluntaryExit(_) => GOSSIP_VOLUNTARY_EXIT, + Work::GossipProposerSlashing(_) => GOSSIP_PROPOSER_SLASHING, + Work::GossipAttesterSlashing(_) => GOSSIP_ATTESTER_SLASHING, + Work::GossipSyncSignature(_) => GOSSIP_SYNC_SIGNATURE, + Work::GossipSyncContribution(_) => GOSSIP_SYNC_CONTRIBUTION, + Work::GossipLightClientFinalityUpdate(_) => GOSSIP_LIGHT_CLIENT_FINALITY_UPDATE, + Work::GossipLightClientOptimisticUpdate(_) => GOSSIP_LIGHT_CLIENT_OPTIMISTIC_UPDATE, Work::RpcBlock { .. } => RPC_BLOCK, - Work::ChainSegment { - process_id: ChainSegmentProcessId::BackSyncBatchId { .. }, - .. - } => CHAIN_SEGMENT_BACKFILL, + Work::IgnoredRpcBlock { .. } => IGNORED_RPC_BLOCK, Work::ChainSegment { .. } => CHAIN_SEGMENT, - Work::Status { .. } => STATUS_PROCESSING, - Work::BlocksByRangeRequest { .. } => BLOCKS_BY_RANGE_REQUEST, - Work::BlocksByRootsRequest { .. } => BLOCKS_BY_ROOTS_REQUEST, - Work::LightClientBootstrapRequest { .. } => LIGHT_CLIENT_BOOTSTRAP_REQUEST, + Work::ChainSegmentBackfill(_) => CHAIN_SEGMENT_BACKFILL, + Work::Status(_) => STATUS_PROCESSING, + Work::BlocksByRangeRequest(_) => BLOCKS_BY_RANGE_REQUEST, + Work::BlocksByRootsRequest(_) => BLOCKS_BY_ROOTS_REQUEST, + Work::LightClientBootstrapRequest(_) => LIGHT_CLIENT_BOOTSTRAP_REQUEST, Work::UnknownBlockAttestation { .. } => UNKNOWN_BLOCK_ATTESTATION, Work::UnknownBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE, - Work::GossipBlsToExecutionChange { .. } => GOSSIP_BLS_TO_EXECUTION_CHANGE, + Work::GossipBlsToExecutionChange(_) => GOSSIP_BLS_TO_EXECUTION_CHANGE, Work::UnknownLightClientOptimisticUpdate { .. } => UNKNOWN_LIGHT_CLIENT_UPDATE, } } } /// Unifies all the messages processed by the `BeaconProcessor`. -enum InboundEvent { +enum InboundEvent { /// A worker has completed a task and is free. WorkerIdle, /// There is new work to be done. - WorkEvent(WorkEvent), + WorkEvent(WorkEvent), /// A work event that was queued for re-processing has become ready. - ReprocessingWork(WorkEvent), + ReprocessingWork(WorkEvent), } /// Combines the various incoming event streams for the `BeaconProcessor` into a single stream. /// /// This struct has a similar purpose to `tokio::select!`, however it allows for more fine-grained /// control (specifically in the ordering of event processing). -struct InboundEvents { +struct InboundEvents { /// Used by workers when they finish a task. idle_rx: mpsc::Receiver<()>, /// Used by upstream processes to send new work to the `BeaconProcessor`. - event_rx: mpsc::Receiver>, + event_rx: mpsc::Receiver>, /// Used internally for queuing work ready to be re-processed. - reprocess_work_rx: mpsc::Receiver>, + reprocess_work_rx: mpsc::Receiver, } -impl Stream for InboundEvents { - type Item = InboundEvent; +impl Stream for InboundEvents { + type Item = InboundEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { // Always check for idle workers before anything else. This allows us to ensure that a big @@ -1001,31 +629,20 @@ impl Stream for InboundEvents { } } -/// Defines if and where we will store the SSZ files of invalid blocks. -#[derive(Clone)] -pub enum InvalidBlockStorage { - Enabled(PathBuf), - Disabled, -} - /// A mutli-threaded processor for messages received on the network /// that need to be processed by the `BeaconChain` /// /// See module level documentation for more information. -pub struct BeaconProcessor { - pub beacon_chain: Weak>, - pub network_tx: mpsc::UnboundedSender>, - pub sync_tx: mpsc::UnboundedSender>, - pub network_globals: Arc>, +pub struct BeaconProcessor { + pub network_globals: Arc>, pub executor: TaskExecutor, pub max_workers: usize, pub current_workers: usize, - pub importing_blocks: DuplicateCache, - pub invalid_block_storage: InvalidBlockStorage, + pub enable_backfill_rate_limiting: bool, pub log: Logger, } -impl BeaconProcessor { +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: /// @@ -1037,10 +654,13 @@ impl BeaconProcessor { /// /// The optional `work_journal_tx` allows for an outside process to receive a log of all work /// events processed by `self`. This should only be used during testing. - pub fn spawn_manager( + pub fn spawn_manager( mut self, - event_rx: mpsc::Receiver>, + event_rx: mpsc::Receiver>, + work_reprocessing_tx: mpsc::Sender, + work_reprocessing_rx: mpsc::Receiver, work_journal_tx: Option>, + slot_clock: S, ) { // Used by workers to communicate that they are finished a task. let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); @@ -1093,20 +713,15 @@ impl BeaconProcessor { let mut lcbootstrap_queue = FifoQueue::new(MAX_LIGHT_CLIENT_BOOTSTRAP_QUEUE_LEN); - let chain = match self.beacon_chain.upgrade() { - Some(chain) => chain, - // No need to proceed any further if the beacon chain has been dropped, the client - // is shutting down. - None => return, - }; - // Channels for sending work to the re-process scheduler (`work_reprocessing_tx`) and to // receive them back once they are ready (`ready_work_rx`). - let (ready_work_tx, ready_work_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); - let work_reprocessing_tx = spawn_reprocess_scheduler( + let (ready_work_tx, ready_work_rx) = + mpsc::channel::(MAX_SCHEDULED_WORK_QUEUE_LEN); + spawn_reprocess_scheduler( ready_work_tx, + work_reprocessing_rx, &self.executor, - chain.slot_clock.clone(), + slot_clock, self.log.clone(), ); @@ -1121,7 +736,7 @@ impl BeaconProcessor { reprocess_work_rx: ready_work_rx, }; - let enable_backfill_rate_limiting = chain.config.enable_backfill_rate_limiting; + let enable_backfill_rate_limiting = self.enable_backfill_rate_limiting; loop { let work_event = match inbound_events.next().await { @@ -1209,33 +824,29 @@ impl BeaconProcessor { .as_ref() .map_or(false, |event| event.drop_during_sync); + let idle_tx = idle_tx.clone(); 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 => { - let toolbox = Toolbox { - idle_tx: idle_tx.clone(), - work_reprocessing_tx: work_reprocessing_tx.clone(), - }; - // 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(item, toolbox); + self.spawn_worker(item, idle_tx); // Check sync blocks before gossip blocks, since we've already explicitly // requested these blocks. } else if let Some(item) = rpc_block_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // Check delayed blocks before gossip blocks, the gossip blocks might rely // on the delayed ones. } else if let Some(item) = delayed_block_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // 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(item, toolbox); + self.spawn_worker(item, idle_tx); // 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. @@ -1246,7 +857,7 @@ impl BeaconProcessor { if batch_size < 2 { // One single aggregate is in the queue, process it individually. if let Some(item) = aggregate_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } } else { // Collect two or more aggregates into a batch, so they can take @@ -1254,32 +865,45 @@ impl BeaconProcessor { // // Note: this will convert the `Work::GossipAggregate` item into a // `Work::GossipAggregateBatch` item. - let mut packages = Vec::with_capacity(batch_size); + let mut aggregates = Vec::with_capacity(batch_size); + let mut process_batch_opt = None; for _ in 0..batch_size { if let Some(item) = aggregate_queue.pop() { match item { Work::GossipAggregate { - message_id, - peer_id, aggregate, - seen_timestamp, + process_individual: _, + process_batch, } => { - packages.push(GossipAggregatePackage::new( - message_id, - peer_id, - aggregate, - seen_timestamp, - )); + aggregates.push(aggregate); + if process_batch_opt.is_none() { + process_batch_opt = Some(process_batch); + } } _ => { - error!(self.log, "Invalid item in aggregate queue") + error!(self.log, "Invalid item in aggregate queue"); } } } } - // Process all aggregates with a single worker. - self.spawn_worker(Work::GossipAggregateBatch { packages }, toolbox) + if let Some(process_batch) = process_batch_opt { + // Process all aggregates with a single worker. + self.spawn_worker( + Work::GossipAggregateBatch { + aggregates, + process_batch, + }, + idle_tx, + ) + } else { + // There is no good reason for this to + // happen, it is a serious logic error. + // Since we only form batches when multiple + // work items exist, we should always have a + // work closure at this point. + crit!(self.log, "Missing aggregate work"); + } } // Check the unaggregated attestation queue. // @@ -1293,7 +917,7 @@ impl BeaconProcessor { if batch_size < 2 { // One single attestation is in the queue, process it individually. if let Some(item) = attestation_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } } else { // Collect two or more attestations into a batch, so they can take @@ -1301,26 +925,20 @@ impl BeaconProcessor { // // Note: this will convert the `Work::GossipAttestation` item into a // `Work::GossipAttestationBatch` item. - let mut packages = Vec::with_capacity(batch_size); + let mut attestations = Vec::with_capacity(batch_size); + let mut process_batch_opt = None; for _ in 0..batch_size { if let Some(item) = attestation_queue.pop() { match item { Work::GossipAttestation { - message_id, - peer_id, attestation, - subnet_id, - should_import, - seen_timestamp, + process_individual: _, + process_batch, } => { - packages.push(GossipAttestationPackage::new( - message_id, - peer_id, - attestation, - subnet_id, - should_import, - seen_timestamp, - )); + attestations.push(attestation); + if process_batch_opt.is_none() { + process_batch_opt = Some(process_batch); + } } _ => error!( self.log, @@ -1330,54 +948,66 @@ impl BeaconProcessor { } } - // Process all attestations with a single worker. - self.spawn_worker( - Work::GossipAttestationBatch { packages }, - toolbox, - ) + if let Some(process_batch) = process_batch_opt { + // Process all attestations with a single worker. + self.spawn_worker( + Work::GossipAttestationBatch { + attestations, + process_batch, + }, + idle_tx, + ) + } else { + // There is no good reason for this to + // happen, it is a serious logic error. + // Since we only form batches when multiple + // work items exist, we should always have a + // work closure at this point. + crit!(self.log, "Missing attestations work"); + } } // Check sync committee messages after attestations as their rewards are lesser // and they don't influence fork choice. } else if let Some(item) = sync_contribution_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } else if let Some(item) = sync_message_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // Aggregates and unaggregates queued for re-processing are older and we // care about fresher ones, so check those first. } else if let Some(item) = unknown_block_aggregate_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } else if let Some(item) = unknown_block_attestation_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // Check RPC methods next. Status messages are needed for sync so // prioritize them over syncing requests from other peers (BlocksByRange // and BlocksByRoot) } else if let Some(item) = status_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } else if let Some(item) = bbrange_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } else if let Some(item) = bbroots_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // Check slashings after all other consensus messages so we prioritize // following head. // // Check attester slashings before proposer slashings since they have the // potential to slash multiple validators at once. } else if let Some(item) = gossip_attester_slashing_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } else if let Some(item) = gossip_proposer_slashing_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // Check exits and address changes late since our validators don't get // rewards from them. } else if let Some(item) = gossip_voluntary_exit_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } else if let Some(item) = gossip_bls_to_execution_change_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // Handle backfill sync chain segments. } else if let Some(item) = backfill_chain_segment.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); // This statement should always be the final else statement. } else if let Some(item) = lcbootstrap_queue.pop() { - self.spawn_worker(item, toolbox); + self.spawn_worker(item, idle_tx); } else { // Let the journal know that a worker is freed and there's nothing else // for it to do. @@ -1419,13 +1049,9 @@ impl BeaconProcessor { // it. Some(WorkEvent { work, .. }) => { let work_id = work.str_id(); - let toolbox = Toolbox { - idle_tx: idle_tx.clone(), - work_reprocessing_tx: work_reprocessing_tx.clone(), - }; match work { - _ if can_spawn => self.spawn_worker(work, toolbox), + _ if can_spawn => self.spawn_worker(work, idle_tx), Work::GossipAttestation { .. } => attestation_queue.push(work), // Attestation batches are formed internally within the // `BeaconProcessor`, they are not sent from external services. @@ -1467,16 +1093,15 @@ impl BeaconProcessor { Work::GossipLightClientOptimisticUpdate { .. } => { optimistic_update_queue.push(work, work_id, &self.log) } - Work::RpcBlock { .. } => rpc_block_queue.push(work, work_id, &self.log), - Work::ChainSegment { ref process_id, .. } => match process_id { - ChainSegmentProcessId::RangeBatchId { .. } - | ChainSegmentProcessId::ParentLookup { .. } => { - chain_segment_queue.push(work, work_id, &self.log) - } - ChainSegmentProcessId::BackSyncBatchId { .. } => { - backfill_chain_segment.push(work, work_id, &self.log) - } - }, + Work::RpcBlock { .. } | Work::IgnoredRpcBlock { .. } => { + rpc_block_queue.push(work, work_id, &self.log) + } + Work::ChainSegment { .. } => { + chain_segment_queue.push(work, work_id, &self.log) + } + Work::ChainSegmentBackfill { .. } => { + backfill_chain_segment.push(work, work_id, &self.log) + } Work::Status { .. } => status_queue.push(work, work_id, &self.log), Work::BlocksByRangeRequest { .. } => { bbrange_queue.push(work, work_id, &self.log) @@ -1583,10 +1208,7 @@ impl BeaconProcessor { /// 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, work: Work, toolbox: Toolbox) { - let idle_tx = toolbox.idle_tx; - let work_reprocessing_tx = toolbox.work_reprocessing_tx; - + fn spawn_worker(&mut self, work: Work, idle_tx: mpsc::Sender<()>) { let work_id = work.str_id(); let worker_timer = metrics::start_timer_vec(&metrics::BEACON_PROCESSOR_WORKER_TIME, &[work_id]); @@ -1609,27 +1231,8 @@ impl BeaconProcessor { 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() { - chain - } else { - debug!( - self.log, - "Beacon chain dropped, shutting down"; - ); - return; - }; - let executor = self.executor.clone(); - let worker = Worker { - chain, - network_tx: self.network_tx.clone(), - sync_tx: self.sync_tx.clone(), - log: self.log.clone(), - }; - - let duplicate_cache = self.importing_blocks.clone(); - trace!( self.log, "Spawning beacon processor worker"; @@ -1638,349 +1241,72 @@ impl BeaconProcessor { ); let task_spawner = TaskSpawner { - executor: executor.clone(), + executor, send_idle_on_drop, }; - let sub_executor = executor; match work { - /* - * Individual unaggregated attestation verification. - */ Work::GossipAttestation { - message_id, - peer_id, attestation, - subnet_id, - should_import, - seen_timestamp, + process_individual, + process_batch: _, } => task_spawner.spawn_blocking(move || { - worker.process_gossip_attestation( - message_id, - peer_id, - attestation, - subnet_id, - should_import, - Some(work_reprocessing_tx), - seen_timestamp, - ) + process_individual(attestation); }), - /* - * Batched unaggregated attestation verification. - */ - Work::GossipAttestationBatch { packages } => task_spawner.spawn_blocking(|| { - worker.process_gossip_attestation_batch(packages, Some(work_reprocessing_tx)) + Work::GossipAttestationBatch { + attestations, + process_batch, + } => task_spawner.spawn_blocking(move || { + process_batch(attestations); }), - /* - * Individual aggregated attestation verification. - */ Work::GossipAggregate { - message_id, - peer_id, aggregate, - seen_timestamp, + process_individual, + process_batch: _, } => task_spawner.spawn_blocking(move || { - worker.process_gossip_aggregate( - message_id, - peer_id, - aggregate, - Some(work_reprocessing_tx), - seen_timestamp, - ) - }), - /* - * Batched aggregated attestation verification. - */ - Work::GossipAggregateBatch { packages } => task_spawner.spawn_blocking(|| { - worker.process_gossip_aggregate_batch(packages, Some(work_reprocessing_tx)) + process_individual(aggregate); }), - /* - * Verification for beacon blocks received on gossip. - */ - Work::GossipBlock { - message_id, - peer_id, - peer_client, - block, - seen_timestamp, - } => { - let invalid_block_storage = self.invalid_block_storage.clone(); - task_spawner.spawn_async(async move { - worker - .process_gossip_block( - message_id, - peer_id, - peer_client, - block, - work_reprocessing_tx, - duplicate_cache, - invalid_block_storage, - seen_timestamp, - ) - .await - }) - } - /* - * Import for blocks that we received earlier than their intended slot. - */ - Work::DelayedImportBlock { - peer_id, - block, - seen_timestamp, - } => { - let invalid_block_storage = self.invalid_block_storage.clone(); - task_spawner.spawn_async(worker.process_gossip_verified_block( - peer_id, - *block, - work_reprocessing_tx, - invalid_block_storage, - seen_timestamp, - )) - } - /* - * Voluntary exits received on gossip. - */ - Work::GossipVoluntaryExit { - message_id, - peer_id, - voluntary_exit, + Work::GossipAggregateBatch { + aggregates, + process_batch, } => task_spawner.spawn_blocking(move || { - worker.process_gossip_voluntary_exit(message_id, peer_id, *voluntary_exit) + process_batch(aggregates); }), - /* - * Proposer slashings received on gossip. - */ - Work::GossipProposerSlashing { - message_id, - peer_id, - proposer_slashing, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_proposer_slashing(message_id, peer_id, *proposer_slashing) - }), - /* - * Attester slashings received on gossip. - */ - Work::GossipAttesterSlashing { - message_id, - peer_id, - attester_slashing, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_attester_slashing(message_id, peer_id, *attester_slashing) + Work::ChainSegment(process_fn) => task_spawner.spawn_async(async move { + process_fn.await; }), - /* - * Sync committee message verification. - */ - Work::GossipSyncSignature { - message_id, - peer_id, - sync_signature, - subnet_id, - seen_timestamp, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_sync_committee_signature( - message_id, - peer_id, - *sync_signature, - subnet_id, - seen_timestamp, - ) - }), - /* - * Sync contribution verification. - */ - Work::GossipSyncContribution { - message_id, - peer_id, - sync_contribution, - seen_timestamp, - } => task_spawner.spawn_blocking(move || { - worker.process_sync_committee_contribution( - message_id, - peer_id, - *sync_contribution, - seen_timestamp, - ) - }), - /* - * BLS to execution change verification. - */ - Work::GossipBlsToExecutionChange { - message_id, - peer_id, - bls_to_execution_change, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_bls_to_execution_change( - message_id, - peer_id, - *bls_to_execution_change, - ) - }), - /* - * Light client finality update verification. - */ - Work::GossipLightClientFinalityUpdate { - message_id, - peer_id, - light_client_finality_update, - seen_timestamp, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_finality_update( - message_id, - peer_id, - *light_client_finality_update, - seen_timestamp, - ) - }), - /* - * Light client optimistic update verification. - */ - Work::GossipLightClientOptimisticUpdate { - message_id, - peer_id, - light_client_optimistic_update, - seen_timestamp, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_optimistic_update( - message_id, - peer_id, - *light_client_optimistic_update, - Some(work_reprocessing_tx), - seen_timestamp, - ) + Work::UnknownBlockAttestation { process_fn } => task_spawner.spawn_blocking(process_fn), + Work::UnknownBlockAggregate { process_fn } => task_spawner.spawn_blocking(process_fn), + Work::UnknownLightClientOptimisticUpdate { + parent_root: _, + process_fn, + } => task_spawner.spawn_blocking(process_fn), + Work::DelayedImportBlock { + beacon_block_slot: _, + beacon_block_root: _, + process_fn, + } => task_spawner.spawn_async(process_fn), + Work::RpcBlock { process_fn } => task_spawner.spawn_async(process_fn), + Work::IgnoredRpcBlock { process_fn } => task_spawner.spawn_blocking(process_fn), + Work::GossipBlock(work) => task_spawner.spawn_async(async move { + work.await; }), - /* - * Verification for beacon blocks received during syncing via RPC. - */ - Work::RpcBlock { - block_root, - block, - seen_timestamp, - process_type, - should_process, - } => task_spawner.spawn_async(worker.process_rpc_block( - block_root, - block, - seen_timestamp, - process_type, - work_reprocessing_tx, - duplicate_cache, - should_process, - )), - /* - * Verification for a chain segment (multiple blocks). - */ - Work::ChainSegment { process_id, blocks } => { - let notify_execution_layer = if self - .network_globals - .sync_state - .read() - .is_syncing_finalized() - { - NotifyExecutionLayer::No - } else { - NotifyExecutionLayer::Yes - }; - - task_spawner.spawn_async(async move { - worker - .process_chain_segment(process_id, blocks, notify_execution_layer) - .await - }) + Work::BlocksByRangeRequest(work) | Work::BlocksByRootsRequest(work) => { + task_spawner.spawn_blocking_with_manual_send_idle(work) } - /* - * Processing of Status Messages. - */ - Work::Status { peer_id, message } => { - task_spawner.spawn_blocking(move || worker.process_status(peer_id, message)) + Work::ChainSegmentBackfill(process_fn) => task_spawner.spawn_async(process_fn), + Work::GossipVoluntaryExit(process_fn) + | Work::GossipProposerSlashing(process_fn) + | Work::GossipAttesterSlashing(process_fn) + | Work::GossipSyncSignature(process_fn) + | Work::GossipSyncContribution(process_fn) + | Work::GossipLightClientFinalityUpdate(process_fn) + | Work::GossipLightClientOptimisticUpdate(process_fn) + | Work::Status(process_fn) + | Work::GossipBlsToExecutionChange(process_fn) + | Work::LightClientBootstrapRequest(process_fn) => { + task_spawner.spawn_blocking(process_fn) } - /* - * Processing of range syncing requests from other peers. - */ - Work::BlocksByRangeRequest { - peer_id, - request_id, - request, - } => task_spawner.spawn_blocking_with_manual_send_idle(move |send_idle_on_drop| { - worker.handle_blocks_by_range_request( - sub_executor, - send_idle_on_drop, - peer_id, - request_id, - request, - ) - }), - /* - * Processing of blocks by roots requests from other peers. - */ - Work::BlocksByRootsRequest { - peer_id, - request_id, - request, - } => task_spawner.spawn_blocking_with_manual_send_idle(move |send_idle_on_drop| { - worker.handle_blocks_by_root_request( - sub_executor, - send_idle_on_drop, - peer_id, - request_id, - request, - ) - }), - /* - * Processing of lightclient bootstrap requests from other peers. - */ - Work::LightClientBootstrapRequest { - peer_id, - request_id, - request, - } => task_spawner.spawn_blocking(move || { - worker.handle_light_client_bootstrap(peer_id, request_id, request) - }), - Work::UnknownBlockAttestation { - message_id, - peer_id, - attestation, - subnet_id, - should_import, - seen_timestamp, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_attestation( - message_id, - peer_id, - attestation, - subnet_id, - should_import, - None, // Do not allow this attestation to be re-processed beyond this point. - seen_timestamp, - ) - }), - Work::UnknownBlockAggregate { - message_id, - peer_id, - aggregate, - seen_timestamp, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_aggregate( - message_id, - peer_id, - aggregate, - None, - seen_timestamp, - ) - }), - Work::UnknownLightClientOptimisticUpdate { - message_id, - peer_id, - light_client_optimistic_update, - seen_timestamp, - } => task_spawner.spawn_blocking(move || { - worker.process_gossip_optimistic_update( - message_id, - peer_id, - *light_client_optimistic_update, - None, - seen_timestamp, - ) - }), }; } } diff --git a/beacon_node/beacon_processor/src/metrics.rs b/beacon_node/beacon_processor/src/metrics.rs new file mode 100644 index 00000000000..65ab0bd8fc5 --- /dev/null +++ b/beacon_node/beacon_processor/src/metrics.rs @@ -0,0 +1,141 @@ +pub use lighthouse_metrics::*; + +lazy_static::lazy_static! { + + /* + * Gossip processor + */ + 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 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 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 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." + ); + // Gossip blocks. + 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." + ); + // Gossip Exits. + pub static ref BEACON_PROCESSOR_EXIT_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_exit_queue_total", + "Count of exits from gossip waiting to be verified." + ); + // Gossip proposer slashings. + pub static ref BEACON_PROCESSOR_PROPOSER_SLASHING_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_proposer_slashing_queue_total", + "Count of proposer slashings from gossip waiting to be verified." + ); + // Gossip attester slashings. + pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_attester_slashing_queue_total", + "Count of attester slashings from gossip waiting to be verified." + ); + // Gossip BLS to execution changes. + pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_bls_to_execution_change_queue_total", + "Count of address changes from gossip waiting to be verified." + ); + // Rpc blocks. + 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." + ); + // Chain segments. + 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 BEACON_PROCESSOR_BACKFILL_CHAIN_SEGMENT_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_backfill_chain_segment_queue_total", + "Count of backfill chain segments from the rpc waiting to be verified." + ); + // Unaggregated attestations. + 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." + ); + // Aggregated attestations. + 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." + ); + // Sync committee messages. + pub static ref BEACON_PROCESSOR_SYNC_MESSAGE_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_sync_message_queue_total", + "Count of sync committee messages waiting to be processed." + ); + // Sync contribution. + pub static ref BEACON_PROCESSOR_SYNC_CONTRIBUTION_QUEUE_TOTAL: Result = try_create_int_gauge( + "beacon_processor_sync_contribution_queue_total", + "Count of sync committee contributions waiting to be processed." + ); + + /* + * Attestation reprocessing queue metrics. + */ + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL: Result = + try_create_int_gauge_vec( + "beacon_processor_reprocessing_queue_total", + "Count of items in a reprocessing queue.", + &["type"] + ); + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS: Result = try_create_int_counter( + "beacon_processor_reprocessing_queue_expired_attestations", + "Number of queued attestations which have expired before a matching block has been found." + ); + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_ATTESTATIONS: Result = try_create_int_counter( + "beacon_processor_reprocessing_queue_matched_attestations", + "Number of queued attestations where as matching block has been imported." + ); + + /* + * Light client update reprocessing queue metrics. + */ + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_OPTIMISTIC_UPDATES: Result = try_create_int_counter( + "beacon_processor_reprocessing_queue_expired_optimistic_updates", + "Number of queued light client optimistic updates which have expired before a matching block has been found." + ); + pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_OPTIMISTIC_UPDATES: Result = try_create_int_counter( + "beacon_processor_reprocessing_queue_matched_optimistic_updates", + "Number of queued light client optimistic updates where as matching block has been imported." + ); + + /// Errors and Debugging Stats + pub static ref BEACON_PROCESSOR_SEND_ERROR_PER_WORK_TYPE: Result = + try_create_int_counter_vec( + "beacon_processor_send_error_per_work_type", + "Total number of beacon processor send error per work type", + &["type"] + ); +} diff --git a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs similarity index 85% rename from beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs rename to beacon_node/beacon_processor/src/work_reprocessing_queue.rs index 427be6d5138..608f634d537 100644 --- a/beacon_node/network/src/beacon_processor/work_reprocessing_queue.rs +++ b/beacon_node/beacon_processor/src/work_reprocessing_queue.rs @@ -10,23 +10,18 @@ //! //! Aggregated and unaggregated attestations that failed verification due to referencing an unknown //! block will be re-queued until their block is imported, or until they expire. -use super::MAX_SCHEDULED_WORK_QUEUE_LEN; -use crate::beacon_processor::{ChainSegmentProcessId, Work, WorkEvent}; use crate::metrics; -use crate::sync::manager::BlockProcessType; -use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; +use crate::{AsyncFn, BlockingFn, Work, WorkEvent}; use fnv::FnvHashMap; use futures::task::Poll; use futures::{Stream, StreamExt}; use itertools::Itertools; -use lighthouse_network::{MessageId, PeerId}; use logging::TimeLatch; use slog::{crit, debug, error, trace, warn, Logger}; use slot_clock::SlotClock; use std::collections::{HashMap, HashSet}; use std::future::Future; use std::pin::Pin; -use std::sync::Arc; use std::task::Context; use std::time::Duration; use strum::AsRefStr; @@ -34,10 +29,7 @@ use task_executor::TaskExecutor; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::time::error::Error as TimeError; use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; -use types::{ - Attestation, EthSpec, Hash256, LightClientOptimisticUpdate, SignedAggregateAndProof, - SignedBeaconBlock, SubnetId, -}; +use types::{EthSpec, Hash256, Slot}; const TASK_NAME: &str = "beacon_processor_reprocess_queue"; const GOSSIP_BLOCKS: &str = "gossip_blocks"; @@ -47,7 +39,7 @@ const LIGHT_CLIENT_UPDATES: &str = "lc_updates"; /// Queue blocks for re-processing with an `ADDITIONAL_QUEUED_BLOCK_DELAY` after the slot starts. /// This is to account for any slight drift in the system clock. -const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5); +pub const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5); /// For how long to queue aggregated and unaggregated attestations for re-processing. pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12); @@ -84,12 +76,12 @@ pub const BACKFILL_SCHEDULE_IN_SLOT: [(u32, u32); 3] = [ /// Messages that the scheduler can receive. #[derive(AsRefStr)] -pub enum ReprocessQueueMessage { +pub enum ReprocessQueueMessage { /// A block that has been received early and we should queue for later processing. - EarlyBlock(QueuedGossipBlock), + EarlyBlock(QueuedGossipBlock), /// A gossip block for hash `X` is being imported, we should queue the rpc block for the same /// hash until the gossip block is imported. - RpcBlock(QueuedRpcBlock), + RpcBlock(QueuedRpcBlock), /// A block that was successfully processed. We use this to handle attestations and light client updates /// for unknown blocks. BlockImported { @@ -97,139 +89,127 @@ pub enum ReprocessQueueMessage { parent_root: Hash256, }, /// An unaggregated attestation that references an unknown block. - UnknownBlockUnaggregate(QueuedUnaggregate), + UnknownBlockUnaggregate(QueuedUnaggregate), /// An aggregated attestation that references an unknown block. - UnknownBlockAggregate(QueuedAggregate), + UnknownBlockAggregate(QueuedAggregate), /// A light client optimistic update that references a parent root that has not been seen as a parent. - UnknownLightClientOptimisticUpdate(QueuedLightClientUpdate), + UnknownLightClientOptimisticUpdate(QueuedLightClientUpdate), /// A new backfill batch that needs to be scheduled for processing. - BackfillSync(QueuedBackfillBatch), + BackfillSync(QueuedBackfillBatch), } /// Events sent by the scheduler once they are ready for re-processing. -pub enum ReadyWork { - Block(QueuedGossipBlock), - RpcBlock(QueuedRpcBlock), - Unaggregate(QueuedUnaggregate), - Aggregate(QueuedAggregate), - LightClientUpdate(QueuedLightClientUpdate), - BackfillSync(QueuedBackfillBatch), +pub enum ReadyWork { + Block(QueuedGossipBlock), + RpcBlock(QueuedRpcBlock), + IgnoredRpcBlock(IgnoredRpcBlock), + Unaggregate(QueuedUnaggregate), + Aggregate(QueuedAggregate), + LightClientUpdate(QueuedLightClientUpdate), + BackfillSync(QueuedBackfillBatch), } /// An Attestation for which the corresponding block was not seen while processing, queued for /// later. -pub struct QueuedUnaggregate { - pub peer_id: PeerId, - pub message_id: MessageId, - pub attestation: Box>, - pub subnet_id: SubnetId, - pub should_import: bool, - pub seen_timestamp: Duration, +pub struct QueuedUnaggregate { + pub beacon_block_root: Hash256, + pub process_fn: BlockingFn, } /// An aggregated attestation for which the corresponding block was not seen while processing, queued for /// later. -pub struct QueuedAggregate { - pub peer_id: PeerId, - pub message_id: MessageId, - pub attestation: Box>, - pub seen_timestamp: Duration, +pub struct QueuedAggregate { + pub beacon_block_root: Hash256, + pub process_fn: BlockingFn, } /// A light client update for which the corresponding parent block was not seen while processing, /// queued for later. -pub struct QueuedLightClientUpdate { - pub peer_id: PeerId, - pub message_id: MessageId, - pub light_client_optimistic_update: Box>, +pub struct QueuedLightClientUpdate { pub parent_root: Hash256, - pub seen_timestamp: Duration, + pub process_fn: BlockingFn, } /// A block that arrived early and has been queued for later import. -pub struct QueuedGossipBlock { - pub peer_id: PeerId, - pub block: Box>, - pub seen_timestamp: Duration, +pub struct QueuedGossipBlock { + pub beacon_block_slot: Slot, + pub beacon_block_root: Hash256, + pub process_fn: AsyncFn, } /// A block that arrived for processing when the same block was being imported over gossip. /// It is queued for later import. -pub struct QueuedRpcBlock { - pub block_root: Hash256, - pub block: Arc>, - pub process_type: BlockProcessType, - pub seen_timestamp: Duration, - /// Indicates if the beacon chain should process this block or not. - /// We use this to ignore block processing when rpc block queues are full. - pub should_process: bool, +pub struct QueuedRpcBlock { + pub beacon_block_root: Hash256, + /// Processes/imports the block. + pub process_fn: AsyncFn, + /// Ignores the block. + pub ignore_fn: BlockingFn, } -/// A backfill batch work that has been queued for processing later. -#[derive(Clone)] -pub struct QueuedBackfillBatch { - pub process_id: ChainSegmentProcessId, - pub blocks: Vec>>, +/// A block that arrived for processing when the same block was being imported over gossip. +/// It is queued for later import. +pub struct IgnoredRpcBlock { + pub process_fn: BlockingFn, } -impl TryFrom> for QueuedBackfillBatch { +/// A backfill batch work that has been queued for processing later. +pub struct QueuedBackfillBatch(pub AsyncFn); + +impl TryFrom> for QueuedBackfillBatch { type Error = WorkEvent; fn try_from(event: WorkEvent) -> Result> { match event { WorkEvent { - work: - Work::ChainSegment { - process_id: process_id @ ChainSegmentProcessId::BackSyncBatchId(_), - blocks, - }, + work: Work::ChainSegmentBackfill(process_fn), .. - } => Ok(QueuedBackfillBatch { process_id, blocks }), + } => Ok(QueuedBackfillBatch(process_fn)), _ => Err(event), } } } -impl From> for WorkEvent { - fn from(queued_backfill_batch: QueuedBackfillBatch) -> WorkEvent { - WorkEvent::chain_segment( - queued_backfill_batch.process_id, - queued_backfill_batch.blocks, - ) +impl From for WorkEvent { + fn from(queued_backfill_batch: QueuedBackfillBatch) -> WorkEvent { + WorkEvent { + drop_during_sync: false, + work: Work::ChainSegmentBackfill(queued_backfill_batch.0), + } } } /// Unifies the different messages processed by the block delay queue. -enum InboundEvent { +enum InboundEvent { /// A gossip block that was queued for later processing and is ready for import. - ReadyGossipBlock(QueuedGossipBlock), + ReadyGossipBlock(QueuedGossipBlock), /// A rpc block that was queued because the same gossip block was being imported /// will now be retried for import. - ReadyRpcBlock(QueuedRpcBlock), + ReadyRpcBlock(QueuedRpcBlock), /// An aggregated or unaggregated attestation is ready for re-processing. ReadyAttestation(QueuedAttestationId), /// A light client update that is ready for re-processing. ReadyLightClientUpdate(QueuedLightClientUpdateId), /// A backfill batch that was queued is ready for processing. - ReadyBackfillSync(QueuedBackfillBatch), + ReadyBackfillSync(QueuedBackfillBatch), /// A `DelayQueue` returned an error. DelayQueueError(TimeError, &'static str), /// A message sent to the `ReprocessQueue` - Msg(ReprocessQueueMessage), + Msg(ReprocessQueueMessage), } /// Manages scheduling works that need to be later re-processed. -struct ReprocessQueue { +struct ReprocessQueue { /// Receiver of messages relevant to schedule works for reprocessing. - work_reprocessing_rx: Receiver>, + work_reprocessing_rx: Receiver, /// Sender of works once they become ready - ready_work_tx: Sender>, + ready_work_tx: Sender, /* Queues */ /// Queue to manage scheduled early blocks. - gossip_block_delay_queue: DelayQueue>, + gossip_block_delay_queue: DelayQueue, /// Queue to manage scheduled early blocks. - rpc_block_delay_queue: DelayQueue>, + rpc_block_delay_queue: DelayQueue, /// Queue to manage scheduled attestations. attestations_delay_queue: DelayQueue, /// Queue to manage scheduled light client updates. @@ -239,17 +219,17 @@ struct ReprocessQueue { /// Queued blocks. queued_gossip_block_roots: HashSet, /// Queued aggregated attestations. - queued_aggregates: FnvHashMap, DelayKey)>, + queued_aggregates: FnvHashMap, /// Queued attestations. - queued_unaggregates: FnvHashMap, DelayKey)>, + queued_unaggregates: FnvHashMap, /// Attestations (aggregated and unaggregated) per root. awaiting_attestations_per_root: HashMap>, /// Queued Light Client Updates. - queued_lc_updates: FnvHashMap, DelayKey)>, + queued_lc_updates: FnvHashMap, /// Light Client Updates per parent_root. awaiting_lc_updates_per_parent_root: HashMap>, /// Queued backfill batches - queued_backfill_batches: Vec>, + queued_backfill_batches: Vec, /* Aux */ /// Next attestation id, used for both aggregated and unaggregated attestations @@ -260,7 +240,7 @@ struct ReprocessQueue { attestation_delay_debounce: TimeLatch, lc_update_delay_debounce: TimeLatch, next_backfill_batch_event: Option>>, - slot_clock: Pin>, + slot_clock: Pin>, } pub type QueuedLightClientUpdateId = usize; @@ -271,20 +251,20 @@ enum QueuedAttestationId { Unaggregate(usize), } -impl QueuedAggregate { +impl QueuedAggregate { pub fn beacon_block_root(&self) -> &Hash256 { - &self.attestation.message.aggregate.data.beacon_block_root + &self.beacon_block_root } } -impl QueuedUnaggregate { +impl QueuedUnaggregate { pub fn beacon_block_root(&self) -> &Hash256 { - &self.attestation.data.beacon_block_root + &self.beacon_block_root } } -impl Stream for ReprocessQueue { - type Item = InboundEvent; +impl Stream for ReprocessQueue { + type Item = InboundEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { // NOTE: implementing `Stream` is not necessary but allows to maintain the future selection @@ -375,16 +355,13 @@ impl Stream for ReprocessQueue { /// Starts the job that manages scheduling works that need re-processing. The returned `Sender` /// gives the communicating channel to receive those works. Once a work is ready, it is sent back /// via `ready_work_tx`. -pub fn spawn_reprocess_scheduler( - ready_work_tx: Sender>, +pub fn spawn_reprocess_scheduler( + ready_work_tx: Sender, + work_reprocessing_rx: Receiver, executor: &TaskExecutor, - slot_clock: T::SlotClock, + slot_clock: S, log: Logger, -) -> Sender> { - let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); - // Basic sanity check. - assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < MAXIMUM_GOSSIP_CLOCK_DISPARITY); - +) { let mut queue = ReprocessQueue { work_reprocessing_rx, ready_work_tx, @@ -423,19 +400,17 @@ pub fn spawn_reprocess_scheduler( }, TASK_NAME, ); - - work_reprocessing_tx } -impl ReprocessQueue { - fn handle_message(&mut self, msg: InboundEvent, slot_clock: &T::SlotClock, log: &Logger) { +impl ReprocessQueue { + fn handle_message(&mut self, msg: InboundEvent, slot_clock: &S, log: &Logger) { use ReprocessQueueMessage::*; match msg { // Some block has been indicated as "early" and should be processed when the // appropriate slot arrives. InboundEvent::Msg(EarlyBlock(early_block)) => { - let block_slot = early_block.block.block.slot(); - let block_root = early_block.block.block_root; + let block_slot = early_block.beacon_block_slot; + let block_root = early_block.beacon_block_root; // Don't add the same block to the queue twice. This prevents DoS attacks. if self.queued_gossip_block_roots.contains(&block_root) { @@ -494,7 +469,7 @@ impl ReprocessQueue { // for the same block hash is being imported. We wait for `QUEUED_RPC_BLOCK_DELAY` // and then send the rpc block back for processing assuming the gossip import // has completed by then. - InboundEvent::Msg(RpcBlock(mut rpc_block)) => { + InboundEvent::Msg(RpcBlock(rpc_block)) => { // Check to ensure this won't over-fill the queue. if self.rpc_block_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS { if self.rpc_block_debounce.elapsed() { @@ -507,10 +482,11 @@ impl ReprocessQueue { } // Return the block to the beacon processor signalling to // ignore processing for this block - rpc_block.should_process = false; if self .ready_work_tx - .try_send(ReadyWork::RpcBlock(rpc_block)) + .try_send(ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock { + process_fn: rpc_block.ignore_fn, + })) .is_err() { error!( @@ -529,7 +505,7 @@ impl ReprocessQueue { debug!( log, "Sending rpc block for reprocessing"; - "block_root" => %queued_rpc_block.block.canonical_root() + "block_root" => %queued_rpc_block.beacon_block_root ); if self .ready_work_tx @@ -767,7 +743,7 @@ impl ReprocessQueue { } // A block that was queued for later processing is now ready to be processed. InboundEvent::ReadyGossipBlock(ready_block) => { - let block_root = ready_block.block.block_root; + let block_root = ready_block.beacon_block_root; if !self.queued_gossip_block_roots.remove(&block_root) { // Log an error to alert that we've made a bad assumption about how this @@ -885,18 +861,28 @@ impl ReprocessQueue { "millis_from_slot_start" => millis_from_slot_start ); - if self + match self .ready_work_tx - .try_send(ReadyWork::BackfillSync(queued_backfill_batch.clone())) - .is_err() + .try_send(ReadyWork::BackfillSync(queued_backfill_batch)) { - error!( + // The message was sent successfully. + Ok(()) => (), + // The message was not sent, recover it from the returned `Err`. + Err(mpsc::error::TrySendError::Full(ReadyWork::BackfillSync(batch))) + | Err(mpsc::error::TrySendError::Closed(ReadyWork::BackfillSync(batch))) => { + error!( + log, + "Failed to send scheduled backfill work"; + "info" => "sending work back to queue" + ); + self.queued_backfill_batches.insert(0, batch) + } + // The message was not sent and we didn't get the correct + // return result. This is a logic error. + _ => crit!( log, - "Failed to send scheduled backfill work"; - "info" => "sending work back to queue" - ); - self.queued_backfill_batches - .insert(0, queued_backfill_batch); + "Unexpected return from try_send error"; + ), } } } @@ -927,7 +913,7 @@ impl ReprocessQueue { // only recompute the `next_backfill_batch_event` if there are backfill batches in the queue if !self.queued_backfill_batches.is_empty() { self.next_backfill_batch_event = Some(Box::pin(tokio::time::sleep( - ReprocessQueue::::duration_until_next_backfill_batch_event(&self.slot_clock), + ReprocessQueue::::duration_until_next_backfill_batch_event(&self.slot_clock), ))); } else { self.next_backfill_batch_event = None @@ -936,7 +922,7 @@ impl ReprocessQueue { /// Returns duration until the next scheduled processing time. The schedule ensure that backfill /// processing is done in windows of time that aren't critical - fn duration_until_next_backfill_batch_event(slot_clock: &T::SlotClock) -> Duration { + fn duration_until_next_backfill_batch_event(slot_clock: &S) -> Duration { let slot_duration = slot_clock.slot_duration(); slot_clock .millis_from_current_slot_start() @@ -966,16 +952,9 @@ impl ReprocessQueue { #[cfg(test)] mod tests { use super::*; - use beacon_chain::builder::Witness; - use beacon_chain::eth1_chain::CachingEth1Backend; use slot_clock::TestingSlotClock; - use store::MemoryStore; - use types::MainnetEthSpec as E; use types::Slot; - type TestBeaconChainType = - Witness, E, MemoryStore, MemoryStore>; - #[test] fn backfill_processing_schedule_calculation() { let slot_duration = Duration::from_secs(12); @@ -988,7 +967,7 @@ mod tests { for &event_duration_from_slot_start in event_times.iter() { let duration_to_next_event = - ReprocessQueue::::duration_until_next_backfill_batch_event( + ReprocessQueue::::duration_until_next_backfill_batch_event( &slot_clock, ); @@ -1005,7 +984,7 @@ mod tests { // check for next event beyond the current slot let duration_to_next_slot = slot_clock.duration_to_next_slot().unwrap(); let duration_to_next_event = - ReprocessQueue::::duration_until_next_backfill_batch_event( + ReprocessQueue::::duration_until_next_backfill_batch_event( &slot_clock, ); assert_eq!( diff --git a/beacon_node/client/Cargo.toml b/beacon_node/client/Cargo.toml index 64c79ea668b..72b6a6c7d47 100644 --- a/beacon_node/client/Cargo.toml +++ b/beacon_node/client/Cargo.toml @@ -43,3 +43,5 @@ slasher = { path = "../../slasher" } slasher_service = { path = "../../slasher/service" } monitoring_api = {path = "../../common/monitoring_api"} execution_layer = { path = "../execution_layer" } +beacon_processor = { path = "../beacon_processor" } +num_cpus = "1.13.0" diff --git a/beacon_node/client/src/builder.rs b/beacon_node/client/src/builder.rs index e05b92a2779..f3ed7f65a90 100644 --- a/beacon_node/client/src/builder.rs +++ b/beacon_node/client/src/builder.rs @@ -13,6 +13,10 @@ use beacon_chain::{ store::{HotColdDB, ItemStore, LevelDB, StoreConfig}, BeaconChain, BeaconChainTypes, Eth1ChainBackend, ServerSentEventHandler, }; +use beacon_processor::{ + work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessor, BeaconProcessorSend, + WorkEvent, MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN, +}; use environment::RuntimeContext; use eth1::{Config as Eth1Config, Service as Eth1Service}; use eth2::{ @@ -27,12 +31,13 @@ use network::{NetworkConfig, NetworkSenders, NetworkService}; use slasher::Slasher; use slasher_service::SlasherService; use slog::{debug, info, warn, Logger}; +use std::cmp; use std::net::TcpListener; use std::path::{Path, PathBuf}; use std::sync::Arc; use std::time::Duration; use timer::spawn_timer; -use tokio::sync::oneshot; +use tokio::sync::{mpsc, oneshot}; use types::{ test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec, ExecutionBlockHash, Hash256, SignedBeaconBlock, @@ -72,6 +77,10 @@ pub struct ClientBuilder { http_metrics_config: http_metrics::Config, slasher: Option>>, eth_spec_instance: T::EthSpec, + beacon_processor_send: BeaconProcessorSend, + beacon_processor_receive: mpsc::Receiver>, + work_reprocessing_tx: mpsc::Sender, + work_reprocessing_rx: mpsc::Receiver, } impl @@ -87,6 +96,10 @@ where /// /// The `eth_spec_instance` parameter is used to concretize `TEthSpec`. pub fn new(eth_spec_instance: TEthSpec) -> Self { + let (beacon_processor_send, beacon_processor_receive) = + mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); + let (work_reprocessing_tx, work_reprocessing_rx) = + mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); Self { slot_clock: None, store: None, @@ -104,6 +117,10 @@ where http_metrics_config: <_>::default(), slasher: None, eth_spec_instance, + beacon_processor_send: BeaconProcessorSend(beacon_processor_send), + beacon_processor_receive, + work_reprocessing_tx, + work_reprocessing_rx, } } @@ -568,6 +585,8 @@ where gossipsub_registry .as_mut() .map(|registry| registry.sub_registry_with_prefix("gossipsub")), + self.beacon_processor_send.clone(), + self.work_reprocessing_tx.clone(), ) .await .map_err(|e| format!("Failed to start network: {:?}", e))?; @@ -755,6 +774,27 @@ where } if let Some(beacon_chain) = self.beacon_chain.as_ref() { + if let Some(network_globals) = &self.network_globals { + let beacon_processor_context = runtime_context.service_context("bproc".into()); + BeaconProcessor { + network_globals: network_globals.clone(), + executor: beacon_processor_context.executor.clone(), + max_workers: cmp::max(1, num_cpus::get()), + current_workers: 0, + enable_backfill_rate_limiting: beacon_chain + .config + .enable_backfill_rate_limiting, + log: beacon_processor_context.log().clone(), + } + .spawn_manager( + self.beacon_processor_receive, + self.work_reprocessing_tx, + self.work_reprocessing_rx, + None, + beacon_chain.slot_clock.clone(), + ); + } + let state_advance_context = runtime_context.service_context("state_advance".into()); let state_advance_log = state_advance_context.log().clone(); spawn_state_advance_timer( diff --git a/beacon_node/network/Cargo.toml b/beacon_node/network/Cargo.toml index aa1827787c6..a5cc12bbc55 100644 --- a/beacon_node/network/Cargo.toml +++ b/beacon_node/network/Cargo.toml @@ -11,7 +11,6 @@ matches = "0.1.8" exit-future = "0.2.0" slog-term = "2.6.0" slog-async = "2.5.0" -environment = { path = "../../lighthouse/environment" } [dependencies] beacon_chain = { path = "../beacon_chain" } @@ -46,4 +45,7 @@ derivative = "2.2.0" delay_map = "0.3.0" ethereum-types = { version = "0.14.1", optional = true } operation_pool = { path = "../operation_pool" } -execution_layer = { path = "../execution_layer" } \ No newline at end of file +execution_layer = { path = "../execution_layer" } +beacon_processor = { path = "../beacon_processor" } +parking_lot = "0.12.0" +environment = { path = "../../lighthouse/environment" } \ No newline at end of file diff --git a/beacon_node/network/src/beacon_processor/worker/mod.rs b/beacon_node/network/src/beacon_processor/worker/mod.rs deleted file mode 100644 index 1cbc64b6329..00000000000 --- a/beacon_node/network/src/beacon_processor/worker/mod.rs +++ /dev/null @@ -1,51 +0,0 @@ -use super::work_reprocessing_queue::ReprocessQueueMessage; -use crate::{service::NetworkMessage, sync::SyncMessage}; -use beacon_chain::{BeaconChain, BeaconChainTypes}; -use slog::{debug, Logger}; -use std::sync::Arc; -use tokio::sync::mpsc; - -mod gossip_methods; -mod rpc_methods; -mod sync_methods; - -pub use gossip_methods::{GossipAggregatePackage, GossipAttestationPackage}; -pub use sync_methods::ChainSegmentProcessId; - -pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1; - -/// Contains the context necessary to import blocks, attestations, etc to the beacon chain. -pub struct Worker { - pub chain: Arc>, - pub network_tx: mpsc::UnboundedSender>, - pub sync_tx: mpsc::UnboundedSender>, - pub log: Logger, -} - -impl Worker { - /// Send a message to `sync_tx`. - /// - /// Creates a log if there is an internal error. - fn send_sync_message(&self, message: SyncMessage) { - self.sync_tx.send(message).unwrap_or_else(|e| { - debug!(self.log, "Could not send message to the sync service"; - "error" => %e) - }); - } - - /// Send a message to `network_tx`. - /// - /// Creates a log if there is an internal error. - fn send_network_message(&self, message: NetworkMessage) { - self.network_tx.send(message).unwrap_or_else(|e| { - debug!(self.log, "Could not send message to the network service. Likely shutdown"; - "error" => %e) - }); - } -} - -/// Contains the necessary items for a worker to do their job. -pub struct Toolbox { - pub idle_tx: mpsc::Sender<()>, - pub work_reprocessing_tx: mpsc::Sender>, -} diff --git a/beacon_node/network/src/lib.rs b/beacon_node/network/src/lib.rs index 648c636acca..da64368b16d 100644 --- a/beacon_node/network/src/lib.rs +++ b/beacon_node/network/src/lib.rs @@ -6,10 +6,10 @@ pub mod error; #[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy pub mod service; -mod beacon_processor; #[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy mod metrics; mod nat; +mod network_beacon_processor; mod persisted_dht; mod router; mod status; diff --git a/beacon_node/network/src/metrics.rs b/beacon_node/network/src/metrics.rs index 27d7dc9625d..0144824861d 100644 --- a/beacon_node/network/src/metrics.rs +++ b/beacon_node/network/src/metrics.rs @@ -49,47 +49,8 @@ lazy_static! { /* * Gossip processor */ - 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 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 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 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." - ); + // Gossip blocks. - 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 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." @@ -107,10 +68,6 @@ lazy_static! { "Whenever a gossip block is received early this metrics is set to how early that block was." ); // Gossip Exits. - pub static ref BEACON_PROCESSOR_EXIT_QUEUE_TOTAL: Result = try_create_int_gauge( - "beacon_processor_exit_queue_total", - "Count of exits from gossip waiting to be verified." - ); pub static ref BEACON_PROCESSOR_EXIT_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_exit_verified_total", "Total number of voluntary exits verified for propagation." @@ -120,10 +77,6 @@ lazy_static! { "Total number of voluntary exits imported to the op pool." ); // Gossip proposer slashings. - pub static ref BEACON_PROCESSOR_PROPOSER_SLASHING_QUEUE_TOTAL: Result = try_create_int_gauge( - "beacon_processor_proposer_slashing_queue_total", - "Count of proposer slashings from gossip waiting to be verified." - ); pub static ref BEACON_PROCESSOR_PROPOSER_SLASHING_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_proposer_slashing_verified_total", "Total number of proposer slashings verified for propagation." @@ -133,10 +86,6 @@ lazy_static! { "Total number of proposer slashings imported to the op pool." ); // Gossip attester slashings. - pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_QUEUE_TOTAL: Result = try_create_int_gauge( - "beacon_processor_attester_slashing_queue_total", - "Count of attester slashings from gossip waiting to be verified." - ); pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_attester_slashing_verified_total", "Total number of attester slashings verified for propagation." @@ -146,10 +95,6 @@ lazy_static! { "Total number of attester slashings imported to the op pool." ); // Gossip BLS to execution changes. - pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_QUEUE_TOTAL: Result = try_create_int_gauge( - "beacon_processor_bls_to_execution_change_queue_total", - "Count of address changes from gossip waiting to be verified." - ); pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_bls_to_execution_change_verified_total", "Total number of address changes verified for propagation." @@ -159,23 +104,11 @@ lazy_static! { "Total number of address changes imported to the op pool." ); // Rpc blocks. - 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 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 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 BEACON_PROCESSOR_BACKFILL_CHAIN_SEGMENT_QUEUE_TOTAL: Result = try_create_int_gauge( - "beacon_processor_backfill_chain_segment_queue_total", - "Count of backfill chain segments from the rpc waiting to be verified." - ); 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." @@ -193,10 +126,6 @@ lazy_static! { "Total number of backfill chain segments that failed processing." ); // Unaggregated attestations. - 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 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." @@ -210,10 +139,6 @@ lazy_static! { "Total number of unaggregated attestations that referenced an unknown block and were re-queued." ); // Aggregated attestations. - 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 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." @@ -227,10 +152,6 @@ lazy_static! { "Total number of aggregated attestations that referenced an unknown block and were re-queued." ); // Sync committee messages. - pub static ref BEACON_PROCESSOR_SYNC_MESSAGE_QUEUE_TOTAL: Result = try_create_int_gauge( - "beacon_processor_sync_message_queue_total", - "Count of sync committee messages waiting to be processed." - ); pub static ref BEACON_PROCESSOR_SYNC_MESSAGE_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_sync_message_verified_total", "Total number of sync committee messages verified for gossip." @@ -240,10 +161,6 @@ lazy_static! { "Total number of sync committee messages imported to fork choice, etc." ); // Sync contribution. - pub static ref BEACON_PROCESSOR_SYNC_CONTRIBUTION_QUEUE_TOTAL: Result = try_create_int_gauge( - "beacon_processor_sync_contribution_queue_total", - "Count of sync committee contributions waiting to be processed." - ); pub static ref BEACON_PROCESSOR_SYNC_CONTRIBUTION_VERIFIED_TOTAL: Result = try_create_int_counter( "beacon_processor_sync_contribution_verified_total", "Total number of sync committee contributions verified for gossip." @@ -279,12 +196,6 @@ lazy_static! { "Gossipsub light_client_optimistic_update errors per error type", &["type"] ); - pub static ref BEACON_PROCESSOR_SEND_ERROR_PER_WORK_TYPE: Result = - try_create_int_counter_vec( - "beacon_processor_send_error_per_work_type", - "Total number of beacon processor send error per work type", - &["type"] - ); /* @@ -371,35 +282,9 @@ lazy_static! { "Count of times when a gossip block arrived from the network later than the attestation deadline.", ); - /* - * Attestation reprocessing queue metrics. - */ - pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL: Result = - try_create_int_gauge_vec( - "beacon_processor_reprocessing_queue_total", - "Count of items in a reprocessing queue.", - &["type"] - ); - pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS: Result = try_create_int_counter( - "beacon_processor_reprocessing_queue_expired_attestations", - "Number of queued attestations which have expired before a matching block has been found." - ); - pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_ATTESTATIONS: Result = try_create_int_counter( - "beacon_processor_reprocessing_queue_matched_attestations", - "Number of queued attestations where as matching block has been imported." - ); - /* * Light client update reprocessing queue metrics. */ - pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_OPTIMISTIC_UPDATES: Result = try_create_int_counter( - "beacon_processor_reprocessing_queue_expired_optimistic_updates", - "Number of queued light client optimistic updates which have expired before a matching block has been found." - ); - pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_OPTIMISTIC_UPDATES: Result = try_create_int_counter( - "beacon_processor_reprocessing_queue_matched_optimistic_updates", - "Number of queued light client optimistic updates where as matching block has been imported." - ); pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_SENT_OPTIMISTIC_UPDATES: Result = try_create_int_counter( "beacon_processor_reprocessing_queue_sent_optimistic_updates", "Number of queued light client optimistic updates where as matching block has been imported." diff --git a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs similarity index 95% rename from beacon_node/network/src/beacon_processor/worker/gossip_methods.rs rename to beacon_node/network/src/network_beacon_processor/gossip_methods.rs index 91ec81b18d3..cde4da9ffcc 100644 --- a/beacon_node/network/src/beacon_processor/worker/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -1,4 +1,9 @@ -use crate::{metrics, service::NetworkMessage, sync::SyncMessage}; +use crate::{ + metrics, + network_beacon_processor::{InvalidBlockStorage, NetworkBeaconProcessor}, + service::NetworkMessage, + sync::SyncMessage, +}; use beacon_chain::store::Error; use beacon_chain::{ @@ -30,14 +35,13 @@ use types::{ SyncCommitteeMessage, SyncSubnetId, }; -use super::{ - super::work_reprocessing_queue::{ +use beacon_processor::{ + work_reprocessing_queue::{ QueuedAggregate, QueuedGossipBlock, QueuedLightClientUpdate, QueuedUnaggregate, ReprocessQueueMessage, }, - Worker, + DuplicateCache, GossipAggregatePackage, GossipAttestationPackage, }; -use crate::beacon_processor::{DuplicateCache, InvalidBlockStorage}; /// Set to `true` to introduce stricter penalties for peers who send some types of late consensus /// messages. @@ -144,65 +148,7 @@ impl FailedAtt { } } -/// Items required to verify a batch of unaggregated gossip attestations. -#[derive(Debug)] -pub struct GossipAttestationPackage { - message_id: MessageId, - peer_id: PeerId, - attestation: Box>, - subnet_id: SubnetId, - should_import: bool, - seen_timestamp: Duration, -} - -impl GossipAttestationPackage { - pub fn new( - message_id: MessageId, - peer_id: PeerId, - attestation: Box>, - subnet_id: SubnetId, - should_import: bool, - seen_timestamp: Duration, - ) -> Self { - Self { - message_id, - peer_id, - attestation, - subnet_id, - should_import, - seen_timestamp, - } - } -} - -/// Items required to verify a batch of aggregated gossip attestations. -#[derive(Debug)] -pub struct GossipAggregatePackage { - message_id: MessageId, - peer_id: PeerId, - aggregate: Box>, - beacon_block_root: Hash256, - seen_timestamp: Duration, -} - -impl GossipAggregatePackage { - pub fn new( - message_id: MessageId, - peer_id: PeerId, - aggregate: Box>, - seen_timestamp: Duration, - ) -> Self { - Self { - message_id, - peer_id, - beacon_block_root: aggregate.message.aggregate.data.beacon_block_root, - aggregate, - seen_timestamp, - } - } -} - -impl Worker { +impl NetworkBeaconProcessor { /* Auxiliary functions */ /// Penalizes a peer for misbehaviour. @@ -245,13 +191,13 @@ impl Worker { /// Raises a log if there are errors. #[allow(clippy::too_many_arguments)] pub fn process_gossip_attestation( - self, + self: Arc, message_id: MessageId, peer_id: PeerId, attestation: Box>, subnet_id: SubnetId, should_import: bool, - reprocess_tx: Option>>, + reprocess_tx: Option>, seen_timestamp: Duration, ) { let result = match self @@ -277,9 +223,9 @@ impl Worker { } pub fn process_gossip_attestation_batch( - self, + self: Arc, packages: Vec>, - reprocess_tx: Option>>, + reprocess_tx: Option>, ) { let attestations_and_subnets = packages .iter() @@ -348,12 +294,12 @@ impl Worker { // cant' be mixed-up) and creating a struct would result in more complexity. #[allow(clippy::too_many_arguments)] fn process_gossip_attestation_result( - &self, + self: &Arc, result: Result, RejectedUnaggregate>, message_id: MessageId, peer_id: PeerId, subnet_id: SubnetId, - reprocess_tx: Option>>, + reprocess_tx: Option>, should_import: bool, seen_timestamp: Duration, ) { @@ -456,11 +402,11 @@ impl Worker { /// /// Raises a log if there are errors. pub fn process_gossip_aggregate( - self, + self: Arc, message_id: MessageId, peer_id: PeerId, aggregate: Box>, - reprocess_tx: Option>>, + reprocess_tx: Option>, seen_timestamp: Duration, ) { let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root; @@ -490,9 +436,9 @@ impl Worker { } pub fn process_gossip_aggregate_batch( - self, + self: Arc, packages: Vec>, - reprocess_tx: Option>>, + reprocess_tx: Option>, ) { let aggregates = packages.iter().map(|package| package.aggregate.as_ref()); @@ -555,12 +501,12 @@ impl Worker { } fn process_gossip_aggregate_result( - &self, + self: &Arc, result: Result, RejectedAggregate>, beacon_block_root: Hash256, message_id: MessageId, peer_id: PeerId, - reprocess_tx: Option>>, + reprocess_tx: Option>, seen_timestamp: Duration, ) { match result { @@ -659,12 +605,12 @@ impl Worker { /// Raises a log if there are errors. #[allow(clippy::too_many_arguments)] pub async fn process_gossip_block( - self, + self: Arc, message_id: MessageId, peer_id: PeerId, peer_client: Client, block: Arc>, - reprocess_tx: mpsc::Sender>, + reprocess_tx: mpsc::Sender, duplicate_cache: DuplicateCache, invalid_block_storage: InvalidBlockStorage, seen_duration: Duration, @@ -708,12 +654,12 @@ impl Worker { /// /// Returns the `GossipVerifiedBlock` if verification passes and raises a log if there are errors. pub async fn process_gossip_unverified_block( - &self, + self: &Arc, message_id: MessageId, peer_id: PeerId, peer_client: Client, block: Arc>, - reprocess_tx: mpsc::Sender>, + reprocess_tx: mpsc::Sender, seen_duration: Duration, ) -> Option> { let block_delay = @@ -911,11 +857,25 @@ impl Worker { metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL); + let inner_self = self.clone(); + let process_fn = Box::pin(async move { + let reprocess_tx = inner_self.reprocess_tx.clone(); + let invalid_block_storage = inner_self.invalid_block_storage.clone(); + inner_self + .process_gossip_verified_block( + peer_id, + verified_block, + reprocess_tx, + invalid_block_storage, + seen_duration, + ) + .await; + }); if reprocess_tx .try_send(ReprocessQueueMessage::EarlyBlock(QueuedGossipBlock { - peer_id, - block: Box::new(verified_block), - seen_timestamp: seen_duration, + beacon_block_slot: block_slot, + beacon_block_root: block_root, + process_fn, })) .is_err() { @@ -948,10 +908,10 @@ impl Worker { /// /// Raises a log if there are errors. pub async fn process_gossip_verified_block( - self, + self: Arc, peer_id: PeerId, verified_block: GossipVerifiedBlock, - reprocess_tx: mpsc::Sender>, + reprocess_tx: mpsc::Sender, invalid_block_storage: InvalidBlockStorage, // This value is not used presently, but it might come in handy for debugging. _seen_duration: Duration, @@ -1051,7 +1011,7 @@ impl Worker { } pub fn process_gossip_voluntary_exit( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, voluntary_exit: SignedVoluntaryExit, @@ -1109,7 +1069,7 @@ impl Worker { } pub fn process_gossip_proposer_slashing( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, proposer_slashing: ProposerSlashing, @@ -1171,7 +1131,7 @@ impl Worker { } pub fn process_gossip_attester_slashing( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, attester_slashing: AttesterSlashing, @@ -1225,7 +1185,7 @@ impl Worker { } pub fn process_gossip_bls_to_execution_change( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, bls_to_execution_change: SignedBlsToExecutionChange, @@ -1308,7 +1268,7 @@ impl Worker { /// /// Raises a log if there are errors. pub fn process_gossip_sync_committee_signature( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, sync_signature: SyncCommitteeMessage, @@ -1371,7 +1331,7 @@ impl Worker { /// /// Raises a log if there are errors. pub fn process_sync_committee_contribution( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, sync_contribution: SignedContributionAndProof, @@ -1426,7 +1386,7 @@ impl Worker { } pub fn process_gossip_finality_update( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, light_client_finality_update: LightClientFinalityUpdate, @@ -1492,11 +1452,11 @@ impl Worker { } pub fn process_gossip_optimistic_update( - self, + self: &Arc, message_id: MessageId, peer_id: PeerId, light_client_optimistic_update: LightClientOptimisticUpdate, - reprocess_tx: Option>>, + reprocess_tx: Option>, seen_timestamp: Duration, ) { match self.chain.verify_optimistic_update_for_gossip( @@ -1527,15 +1487,19 @@ impl Worker { ); if let Some(sender) = reprocess_tx { + let processor = self.clone(); let msg = ReprocessQueueMessage::UnknownLightClientOptimisticUpdate( QueuedLightClientUpdate { - peer_id, - message_id, - light_client_optimistic_update: Box::new( - light_client_optimistic_update, - ), parent_root, - seen_timestamp, + process_fn: Box::new(move || { + processor.process_gossip_optimistic_update( + message_id, + peer_id, + light_client_optimistic_update, + None, // Do not reprocess this message again. + seen_timestamp, + ) + }), }, ); @@ -1624,11 +1588,11 @@ impl Worker { /// Handle an error whilst verifying an `Attestation` or `SignedAggregateAndProof` from the /// network. fn handle_attestation_verification_failure( - &self, + self: &Arc, peer_id: PeerId, message_id: MessageId, failed_att: FailedAtt, - reprocess_tx: Option>>, + reprocess_tx: Option>, error: AttnError, seen_timestamp: Duration, ) { @@ -1860,11 +1824,18 @@ impl Worker { metrics::inc_counter( &metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_REQUEUED_TOTAL, ); + let processor = self.clone(); ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate { - peer_id, - message_id, - attestation, - seen_timestamp, + beacon_block_root: *beacon_block_root, + process_fn: Box::new(move || { + processor.process_gossip_aggregate( + message_id, + peer_id, + attestation, + None, // Do not allow this attestation to be re-processed beyond this point. + seen_timestamp, + ) + }), }) } FailedAtt::Unaggregate { @@ -1876,13 +1847,20 @@ impl Worker { metrics::inc_counter( &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL, ); + let processor = self.clone(); ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { - peer_id, - message_id, - attestation, - subnet_id, - should_import, - seen_timestamp, + beacon_block_root: *beacon_block_root, + process_fn: Box::new(move || { + processor.process_gossip_attestation( + message_id, + peer_id, + attestation, + subnet_id, + should_import, + None, // Do not allow this attestation to be re-processed beyond this point. + seen_timestamp, + ) + }), }) } }; diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs new file mode 100644 index 00000000000..7f0ef1fb817 --- /dev/null +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -0,0 +1,590 @@ +use crate::{ + service::NetworkMessage, + sync::{manager::BlockProcessType, SyncMessage}, +}; +use beacon_chain::{ + builder::Witness, eth1_chain::CachingEth1Backend, test_utils::BeaconChainHarness, BeaconChain, +}; +use beacon_chain::{BeaconChainTypes, NotifyExecutionLayer}; +use beacon_processor::{ + work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend, DuplicateCache, + GossipAggregatePackage, GossipAttestationPackage, Work, WorkEvent as BeaconWorkEvent, + MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN, +}; +use environment::null_logger; +use lighthouse_network::{ + rpc::{BlocksByRangeRequest, BlocksByRootRequest, LightClientBootstrapRequest, StatusMessage}, + Client, MessageId, NetworkGlobals, PeerId, PeerRequestId, +}; +use slog::{debug, Logger}; +use slot_clock::ManualSlotClock; +use std::path::PathBuf; +use std::sync::Arc; +use std::time::Duration; +use store::MemoryStore; +use task_executor::test_utils::TestRuntime; +use task_executor::TaskExecutor; +use tokio::sync::mpsc::{self, error::TrySendError}; +use types::*; + +pub use sync_methods::ChainSegmentProcessId; + +pub type Error = TrySendError>; + +mod gossip_methods; +mod rpc_methods; +mod sync_methods; +mod tests; + +pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1; + +/// Defines if and where we will store the SSZ files of invalid blocks. +#[derive(Clone)] +pub enum InvalidBlockStorage { + Enabled(PathBuf), + Disabled, +} + +/// Provides an interface to a `BeaconProcessor` running in some other thread. +/// The wider `networking` crate should use this struct to interface with the +/// beacon processor. +pub struct NetworkBeaconProcessor { + pub beacon_processor_send: BeaconProcessorSend, + pub duplicate_cache: DuplicateCache, + pub chain: Arc>, + pub network_tx: mpsc::UnboundedSender>, + pub sync_tx: mpsc::UnboundedSender>, + pub reprocess_tx: mpsc::Sender, + pub network_globals: Arc>, + pub invalid_block_storage: InvalidBlockStorage, + pub executor: TaskExecutor, + pub log: Logger, +} + +impl NetworkBeaconProcessor { + fn try_send(&self, event: BeaconWorkEvent) -> Result<(), Error> { + self.beacon_processor_send + .try_send(event) + .map_err(Into::into) + } + + /// Create a new `Work` event for some unaggregated attestation. + pub fn send_unaggregated_attestation( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + attestation: Attestation, + subnet_id: SubnetId, + should_import: bool, + seen_timestamp: Duration, + ) -> Result<(), Error> { + // Define a closure for processing individual attestations. + let processor = self.clone(); + let process_individual = move |package: GossipAttestationPackage| { + let reprocess_tx = processor.reprocess_tx.clone(); + processor.process_gossip_attestation( + package.message_id, + package.peer_id, + package.attestation, + package.subnet_id, + package.should_import, + Some(reprocess_tx), + package.seen_timestamp, + ) + }; + + // Define a closure for processing batches of attestations. + let processor = self.clone(); + let process_batch = move |attestations| { + let reprocess_tx = processor.reprocess_tx.clone(); + processor.process_gossip_attestation_batch(attestations, Some(reprocess_tx)) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: true, + work: Work::GossipAttestation { + attestation: GossipAttestationPackage { + message_id, + peer_id, + attestation: Box::new(attestation), + subnet_id, + should_import, + seen_timestamp, + }, + process_individual: Box::new(process_individual), + process_batch: Box::new(process_batch), + }, + }) + } + + /// Create a new `Work` event for some aggregated attestation. + pub fn send_aggregated_attestation( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + aggregate: SignedAggregateAndProof, + seen_timestamp: Duration, + ) -> Result<(), Error> { + // Define a closure for processing individual attestations. + let processor = self.clone(); + let process_individual = move |package: GossipAggregatePackage| { + let reprocess_tx = processor.reprocess_tx.clone(); + processor.process_gossip_aggregate( + package.message_id, + package.peer_id, + package.aggregate, + Some(reprocess_tx), + package.seen_timestamp, + ) + }; + + // Define a closure for processing batches of attestations. + let processor = self.clone(); + let process_batch = move |aggregates| { + let reprocess_tx = processor.reprocess_tx.clone(); + processor.process_gossip_aggregate_batch(aggregates, Some(reprocess_tx)) + }; + + let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root; + self.try_send(BeaconWorkEvent { + drop_during_sync: true, + work: Work::GossipAggregate { + aggregate: GossipAggregatePackage { + message_id, + peer_id, + aggregate: Box::new(aggregate), + beacon_block_root, + seen_timestamp, + }, + process_individual: Box::new(process_individual), + process_batch: Box::new(process_batch), + }, + }) + } + + /// Create a new `Work` event for some block. + pub fn send_gossip_beacon_block( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + peer_client: Client, + block: Arc>, + seen_timestamp: Duration, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = async move { + let reprocess_tx = processor.reprocess_tx.clone(); + let invalid_block_storage = processor.invalid_block_storage.clone(); + let duplicate_cache = processor.duplicate_cache.clone(); + processor + .process_gossip_block( + message_id, + peer_id, + peer_client, + block, + reprocess_tx, + duplicate_cache, + invalid_block_storage, + seen_timestamp, + ) + .await + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::GossipBlock(Box::pin(process_fn)), + }) + } + + /// Create a new `Work` event for some sync committee signature. + pub fn send_gossip_sync_signature( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + sync_signature: SyncCommitteeMessage, + subnet_id: SyncSubnetId, + seen_timestamp: Duration, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || { + processor.process_gossip_sync_committee_signature( + message_id, + peer_id, + sync_signature, + subnet_id, + seen_timestamp, + ) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: true, + work: Work::GossipSyncSignature(Box::new(process_fn)), + }) + } + + /// Create a new `Work` event for some sync committee contribution. + pub fn send_gossip_sync_contribution( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + sync_contribution: SignedContributionAndProof, + seen_timestamp: Duration, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || { + processor.process_sync_committee_contribution( + message_id, + peer_id, + sync_contribution, + seen_timestamp, + ) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: true, + work: Work::GossipSyncContribution(Box::new(process_fn)), + }) + } + + /// Create a new `Work` event for some exit. + pub fn send_gossip_voluntary_exit( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + voluntary_exit: Box, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = + move || processor.process_gossip_voluntary_exit(message_id, peer_id, *voluntary_exit); + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::GossipVoluntaryExit(Box::new(process_fn)), + }) + } + + /// Create a new `Work` event for some proposer slashing. + pub fn send_gossip_proposer_slashing( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + proposer_slashing: Box, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || { + processor.process_gossip_proposer_slashing(message_id, peer_id, *proposer_slashing) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::GossipProposerSlashing(Box::new(process_fn)), + }) + } + + /// Create a new `Work` event for some light client finality update. + pub fn send_gossip_light_client_finality_update( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + light_client_finality_update: LightClientFinalityUpdate, + seen_timestamp: Duration, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || { + processor.process_gossip_finality_update( + message_id, + peer_id, + light_client_finality_update, + seen_timestamp, + ) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: true, + work: Work::GossipLightClientFinalityUpdate(Box::new(process_fn)), + }) + } + + /// Create a new `Work` event for some light client optimistic update. + pub fn send_gossip_light_client_optimistic_update( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + light_client_optimistic_update: LightClientOptimisticUpdate, + seen_timestamp: Duration, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || { + let reprocess_tx = processor.reprocess_tx.clone(); + processor.process_gossip_optimistic_update( + message_id, + peer_id, + light_client_optimistic_update, + Some(reprocess_tx), + seen_timestamp, + ) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: true, + work: Work::GossipLightClientOptimisticUpdate(Box::new(process_fn)), + }) + } + + /// Create a new `Work` event for some attester slashing. + pub fn send_gossip_attester_slashing( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + attester_slashing: Box>, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || { + processor.process_gossip_attester_slashing(message_id, peer_id, *attester_slashing) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::GossipAttesterSlashing(Box::new(process_fn)), + }) + } + + /// Create a new `Work` event for some BLS to execution change. + pub fn send_gossip_bls_to_execution_change( + self: &Arc, + message_id: MessageId, + peer_id: PeerId, + bls_to_execution_change: Box, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || { + processor.process_gossip_bls_to_execution_change( + message_id, + peer_id, + *bls_to_execution_change, + ) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::GossipBlsToExecutionChange(Box::new(process_fn)), + }) + } + + /// 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 send_rpc_beacon_block( + self: &Arc, + block_root: Hash256, + block: Arc>, + seen_timestamp: Duration, + process_type: BlockProcessType, + ) -> Result<(), Error> { + let process_fn = self.clone().generate_rpc_beacon_block_process_fn( + block_root, + block, + seen_timestamp, + process_type, + ); + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::RpcBlock { process_fn }, + }) + } + + /// Create a new work event to import `blocks` as a beacon chain segment. + pub fn send_chain_segment( + self: &Arc, + process_id: ChainSegmentProcessId, + blocks: Vec>>, + ) -> Result<(), Error> { + let is_backfill = matches!(&process_id, ChainSegmentProcessId::BackSyncBatchId { .. }); + let processor = self.clone(); + let process_fn = async move { + let notify_execution_layer = if processor + .network_globals + .sync_state + .read() + .is_syncing_finalized() + { + NotifyExecutionLayer::No + } else { + NotifyExecutionLayer::Yes + }; + processor + .process_chain_segment(process_id, blocks, notify_execution_layer) + .await; + }; + let process_fn = Box::pin(process_fn); + + // Back-sync batches are dispatched with a different `Work` variant so + // they can be rate-limited. + let work = if is_backfill { + Work::ChainSegmentBackfill(process_fn) + } else { + Work::ChainSegment(process_fn) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work, + }) + } + + /// Create a new work event to process `StatusMessage`s from the RPC network. + pub fn send_status_message( + self: &Arc, + peer_id: PeerId, + message: StatusMessage, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move || processor.process_status(peer_id, message); + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::Status(Box::new(process_fn)), + }) + } + + /// Create a new work event to process `BlocksByRangeRequest`s from the RPC network. + pub fn send_blocks_by_range_request( + self: &Arc, + peer_id: PeerId, + request_id: PeerRequestId, + request: BlocksByRangeRequest, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move |send_idle_on_drop| { + let executor = processor.executor.clone(); + processor.handle_blocks_by_range_request( + executor, + send_idle_on_drop, + peer_id, + request_id, + request, + ) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::BlocksByRangeRequest(Box::new(process_fn)), + }) + } + + /// Create a new work event to process `BlocksByRootRequest`s from the RPC network. + pub fn send_blocks_by_roots_request( + self: &Arc, + peer_id: PeerId, + request_id: PeerRequestId, + request: BlocksByRootRequest, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = move |send_idle_on_drop| { + let executor = processor.executor.clone(); + processor.handle_blocks_by_root_request( + executor, + send_idle_on_drop, + peer_id, + request_id, + request, + ) + }; + + self.try_send(BeaconWorkEvent { + drop_during_sync: false, + work: Work::BlocksByRootsRequest(Box::new(process_fn)), + }) + } + + /// Create a new work event to process `LightClientBootstrap`s from the RPC network. + pub fn send_lightclient_bootstrap_request( + self: &Arc, + peer_id: PeerId, + request_id: PeerRequestId, + request: LightClientBootstrapRequest, + ) -> Result<(), Error> { + let processor = self.clone(); + let process_fn = + move || processor.handle_light_client_bootstrap(peer_id, request_id, request); + + self.try_send(BeaconWorkEvent { + drop_during_sync: true, + work: Work::LightClientBootstrapRequest(Box::new(process_fn)), + }) + } + + /// Send a message to `sync_tx`. + /// + /// Creates a log if there is an internal error. + fn send_sync_message(&self, message: SyncMessage) { + self.sync_tx.send(message).unwrap_or_else(|e| { + debug!(self.log, "Could not send message to the sync service"; + "error" => %e) + }); + } + + /// Send a message to `network_tx`. + /// + /// Creates a log if there is an internal error. + fn send_network_message(&self, message: NetworkMessage) { + self.network_tx.send(message).unwrap_or_else(|e| { + debug!(self.log, "Could not send message to the network service. Likely shutdown"; + "error" => %e) + }); + } +} + +type TestBeaconChainType = + Witness, E, MemoryStore, MemoryStore>; + +impl NetworkBeaconProcessor> { + // Instantiates a mostly non-functional version of `Self` and returns the + // event receiver that would normally go to the beacon processor. This is + // useful for testing that messages are actually being sent to the beacon + // processor (but not much else). + pub fn null_for_testing( + network_globals: Arc>, + ) -> (Self, mpsc::Receiver>) { + let (beacon_processor_send, beacon_processor_receive) = + mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); + let (network_tx, _network_rx) = mpsc::unbounded_channel(); + let (sync_tx, _sync_rx) = mpsc::unbounded_channel(); + let (reprocess_tx, _reprocess_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); + let log = null_logger().unwrap(); + let harness: BeaconChainHarness> = + BeaconChainHarness::builder(E::default()) + .spec(E::default_spec()) + .deterministic_keypairs(8) + .logger(log.clone()) + .fresh_ephemeral_store() + .mock_execution_layer() + .build(); + let runtime = TestRuntime::default(); + + let network_beacon_processor = Self { + beacon_processor_send: BeaconProcessorSend(beacon_processor_send), + duplicate_cache: DuplicateCache::default(), + chain: harness.chain, + network_tx, + sync_tx, + reprocess_tx, + network_globals, + invalid_block_storage: InvalidBlockStorage::Disabled, + executor: runtime.task_executor.clone(), + log, + }; + + (network_beacon_processor, beacon_processor_receive) + } +} + +#[cfg(test)] +mod test { + #[test] + fn queued_block_delay_is_sane() { + assert!( + beacon_processor::work_reprocessing_queue::ADDITIONAL_QUEUED_BLOCK_DELAY + < beacon_chain::MAXIMUM_GOSSIP_CLOCK_DISPARITY + ); + } +} diff --git a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs b/beacon_node/network/src/network_beacon_processor/rpc_methods.rs similarity index 98% rename from beacon_node/network/src/beacon_processor/worker/rpc_methods.rs rename to beacon_node/network/src/network_beacon_processor/rpc_methods.rs index 83baa0417bc..19b0a60a43e 100644 --- a/beacon_node/network/src/beacon_processor/worker/rpc_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/rpc_methods.rs @@ -1,21 +1,21 @@ -use crate::beacon_processor::{worker::FUTURE_SLOT_TOLERANCE, SendOnDrop}; +use crate::network_beacon_processor::{NetworkBeaconProcessor, FUTURE_SLOT_TOLERANCE}; use crate::service::NetworkMessage; use crate::status::ToStatusMessage; use crate::sync::SyncMessage; use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped}; +use beacon_processor::SendOnDrop; use itertools::process_results; use lighthouse_network::rpc::StatusMessage; use lighthouse_network::rpc::*; use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo}; use slog::{debug, error, warn}; use slot_clock::SlotClock; +use std::sync::Arc; use task_executor::TaskExecutor; use tokio_stream::StreamExt; use types::{light_client_bootstrap::LightClientBootstrap, Epoch, EthSpec, Hash256, Slot}; -use super::Worker; - -impl Worker { +impl NetworkBeaconProcessor { /* Auxiliary functions */ /// Disconnects and ban's a peer, sending a Goodbye request with the associated reason. @@ -124,7 +124,7 @@ impl Worker { /// Handle a `BlocksByRoot` request from the peer. pub fn handle_blocks_by_root_request( - self, + self: Arc, executor: TaskExecutor, send_on_drop: SendOnDrop, peer_id: PeerId, @@ -210,7 +210,7 @@ impl Worker { /// Handle a `BlocksByRoot` request from the peer. pub fn handle_light_client_bootstrap( - self, + self: &Arc, peer_id: PeerId, request_id: PeerRequestId, request: LightClientBootstrapRequest, @@ -283,7 +283,7 @@ impl Worker { /// Handle a `BlocksByRange` request from the peer. pub fn handle_blocks_by_range_request( - self, + self: Arc, executor: TaskExecutor, send_on_drop: SendOnDrop, peer_id: PeerId, diff --git a/beacon_node/network/src/beacon_processor/worker/sync_methods.rs b/beacon_node/network/src/network_beacon_processor/sync_methods.rs similarity index 90% rename from beacon_node/network/src/beacon_processor/worker/sync_methods.rs rename to beacon_node/network/src/network_beacon_processor/sync_methods.rs index ac59b1daa93..c33e2acf542 100644 --- a/beacon_node/network/src/beacon_processor/worker/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -1,17 +1,21 @@ use std::time::Duration; -use super::{super::work_reprocessing_queue::ReprocessQueueMessage, Worker}; -use crate::beacon_processor::work_reprocessing_queue::QueuedRpcBlock; -use crate::beacon_processor::worker::FUTURE_SLOT_TOLERANCE; -use crate::beacon_processor::DuplicateCache; use crate::metrics; -use crate::sync::manager::{BlockProcessType, SyncMessage}; -use crate::sync::{BatchProcessResult, ChainId}; +use crate::network_beacon_processor::{NetworkBeaconProcessor, FUTURE_SLOT_TOLERANCE}; +use crate::sync::BatchProcessResult; +use crate::sync::{ + manager::{BlockProcessType, SyncMessage}, + ChainId, +}; use beacon_chain::{ observed_block_producers::Error as ObserveError, validator_monitor::get_block_delay_ms, BeaconChainError, BeaconChainTypes, BlockError, ChainSegmentResult, HistoricalBlockError, NotifyExecutionLayer, }; +use beacon_processor::{ + work_reprocessing_queue::{QueuedRpcBlock, ReprocessQueueMessage}, + AsyncFn, BlockingFn, DuplicateCache, +}; use lighthouse_network::PeerAction; use slog::{debug, error, info, warn}; use slot_clock::SlotClock; @@ -39,27 +43,71 @@ struct ChainSegmentFailed { peer_action: Option, } -impl Worker { - /// Attempt to process a block received from a direct RPC request. - #[allow(clippy::too_many_arguments)] - pub async fn process_rpc_block( - self, +impl NetworkBeaconProcessor { + /// Returns an async closure which processes a beacon block recieved via RPC. + /// + /// This separate function was required to prevent a cycle during compiler + /// type checking. + pub fn generate_rpc_beacon_block_process_fn( + self: Arc, block_root: Hash256, block: Arc>, seen_timestamp: Duration, process_type: BlockProcessType, - reprocess_tx: mpsc::Sender>, - duplicate_cache: DuplicateCache, - should_process: bool, - ) { - if !should_process { + ) -> AsyncFn { + let process_fn = async move { + let reprocess_tx = self.reprocess_tx.clone(); + let duplicate_cache = self.duplicate_cache.clone(); + self.process_rpc_block( + block_root, + block, + seen_timestamp, + process_type, + reprocess_tx, + duplicate_cache, + ) + .await; + }; + Box::pin(process_fn) + } + + /// Returns the `process_fn` and `ignore_fn` required when requeuing an RPC block. + pub fn generate_rpc_beacon_block_fns( + self: Arc, + block_root: Hash256, + block: Arc>, + seen_timestamp: Duration, + process_type: BlockProcessType, + ) -> (AsyncFn, BlockingFn) { + // An async closure which will import the block. + let process_fn = self.clone().generate_rpc_beacon_block_process_fn( + block_root, + block, + seen_timestamp, + process_type.clone(), + ); + // A closure which will ignore the block. + let ignore_fn = move || { // Sync handles these results self.send_sync_message(SyncMessage::BlockProcessed { process_type, result: crate::sync::manager::BlockProcessResult::Ignored, }); - return; - } + }; + (process_fn, Box::new(ignore_fn)) + } + + /// Attempt to process a block received from a direct RPC request. + #[allow(clippy::too_many_arguments)] + pub async fn process_rpc_block( + self: Arc>, + block_root: Hash256, + block: Arc>, + seen_timestamp: Duration, + process_type: BlockProcessType, + reprocess_tx: mpsc::Sender, + duplicate_cache: DuplicateCache, + ) { // Check if the block is already being imported through another source let handle = match duplicate_cache.check_and_insert(block_root) { Some(handle) => handle, @@ -70,13 +118,18 @@ impl Worker { "action" => "sending rpc block to reprocessing queue", "block_root" => %block_root, ); + // Send message to work reprocess queue to retry the block - let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock { + let (process_fn, ignore_fn) = self.clone().generate_rpc_beacon_block_fns( block_root, - block: block.clone(), - process_type, + block, seen_timestamp, - should_process: true, + process_type, + ); + let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock { + beacon_block_root: block_root, + process_fn, + ignore_fn, }); if reprocess_tx.try_send(reprocess_msg).is_err() { @@ -130,12 +183,16 @@ impl Worker { ); // Send message to work reprocess queue to retry the block - let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock { + let (process_fn, ignore_fn) = self.clone().generate_rpc_beacon_block_fns( block_root, - block: block.clone(), - process_type, + block, seen_timestamp, - should_process: true, + process_type, + ); + let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock { + beacon_block_root: block_root, + process_fn, + ignore_fn, }); if reprocess_tx.try_send(reprocess_msg).is_err() { diff --git a/beacon_node/network/src/beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs similarity index 89% rename from beacon_node/network/src/beacon_processor/tests.rs rename to beacon_node/network/src/network_beacon_processor/tests.rs index b93e83ad785..b8d5db568ec 100644 --- a/beacon_node/network/src/beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -1,20 +1,23 @@ #![cfg(not(debug_assertions))] // Tests are too slow in debug. #![cfg(test)] -use crate::beacon_processor::work_reprocessing_queue::{ - QUEUED_ATTESTATION_DELAY, QUEUED_RPC_BLOCK_DELAY, +use crate::{ + network_beacon_processor::{ + ChainSegmentProcessId, DuplicateCache, InvalidBlockStorage, NetworkBeaconProcessor, + }, + service::NetworkMessage, + sync::{manager::BlockProcessType, SyncMessage}, }; -use crate::beacon_processor::*; -use crate::{service::NetworkMessage, sync::SyncMessage}; use beacon_chain::test_utils::{ AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType, }; use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY}; +use beacon_processor::{work_reprocessing_queue::*, *}; use lighthouse_network::{ discv5::enr::{CombinedKey, EnrBuilder}, rpc::methods::{MetaData, MetaDataV2}, types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield}, - MessageId, NetworkGlobals, PeerId, + Client, MessageId, NetworkGlobals, PeerId, }; use slot_clock::SlotClock; use std::cmp; @@ -23,8 +26,8 @@ use std::sync::Arc; use std::time::Duration; use tokio::sync::mpsc; use types::{ - Attestation, AttesterSlashing, Epoch, EthSpec, MainnetEthSpec, ProposerSlashing, - SignedBeaconBlock, SignedVoluntaryExit, SubnetId, + Attestation, AttesterSlashing, Epoch, EthSpec, Hash256, MainnetEthSpec, ProposerSlashing, + SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, SubnetId, }; type E = MainnetEthSpec; @@ -51,11 +54,12 @@ struct TestRig { attester_slashing: AttesterSlashing, proposer_slashing: ProposerSlashing, voluntary_exit: SignedVoluntaryExit, - beacon_processor_tx: mpsc::Sender>, + beacon_processor_tx: BeaconProcessorSend, work_journal_rx: mpsc::Receiver<&'static str>, _network_rx: mpsc::UnboundedReceiver>, _sync_rx: mpsc::UnboundedReceiver>, duplicate_cache: DuplicateCache, + network_beacon_processor: Arc>, _harness: BeaconChainHarness, } @@ -64,7 +68,7 @@ struct TestRig { impl Drop for TestRig { fn drop(&mut self) { // Causes the beacon processor to shutdown. - self.beacon_processor_tx = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN).0; + self.beacon_processor_tx = BeaconProcessorSend(mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN).0); } } @@ -169,6 +173,7 @@ impl TestRig { let log = harness.logger().clone(); let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); + let beacon_processor_tx = BeaconProcessorSend(beacon_processor_tx); let (sync_tx, _sync_rx) = mpsc::unbounded_channel(); // Default metadata @@ -191,22 +196,40 @@ impl TestRig { let executor = harness.runtime.task_executor.clone(); + let (work_reprocessing_tx, work_reprocessing_rx) = + mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); let (work_journal_tx, work_journal_rx) = mpsc::channel(16_364); let duplicate_cache = DuplicateCache::default(); - BeaconProcessor { - beacon_chain: Arc::downgrade(&chain), + let network_beacon_processor = NetworkBeaconProcessor { + beacon_processor_send: beacon_processor_tx.clone(), + duplicate_cache: duplicate_cache.clone(), + chain: harness.chain.clone(), network_tx, sync_tx, + reprocess_tx: work_reprocessing_tx.clone(), + network_globals: network_globals.clone(), + invalid_block_storage: InvalidBlockStorage::Disabled, + executor: executor.clone(), + log: log.clone(), + }; + let network_beacon_processor = Arc::new(network_beacon_processor); + + BeaconProcessor { network_globals, executor, max_workers: cmp::max(1, num_cpus::get()), current_workers: 0, - importing_blocks: duplicate_cache.clone(), - invalid_block_storage: InvalidBlockStorage::Disabled, + enable_backfill_rate_limiting: harness.chain.config.enable_backfill_rate_limiting, log: log.clone(), } - .spawn_manager(beacon_processor_rx, Some(work_journal_tx)); + .spawn_manager( + beacon_processor_rx, + work_reprocessing_tx, + work_reprocessing_rx, + Some(work_journal_tx), + harness.chain.slot_clock.clone(), + ); Self { chain, @@ -222,6 +245,7 @@ impl TestRig { _network_rx, _sync_rx, duplicate_cache, + network_beacon_processor, _harness: harness, } } @@ -235,102 +259,105 @@ impl TestRig { } pub fn enqueue_gossip_block(&self) { - self.beacon_processor_tx - .try_send(WorkEvent::gossip_beacon_block( + self.network_beacon_processor + .send_gossip_beacon_block( junk_message_id(), junk_peer_id(), Client::default(), self.next_block.clone(), Duration::from_secs(0), - )) + ) .unwrap(); } pub fn enqueue_rpc_block(&self) { - let event = WorkEvent::rpc_beacon_block( - self.next_block.canonical_root(), - self.next_block.clone(), - std::time::Duration::default(), - BlockProcessType::ParentLookup { - chain_hash: Hash256::random(), - }, - ); - self.beacon_processor_tx.try_send(event).unwrap(); + self.network_beacon_processor + .send_rpc_beacon_block( + self.next_block.canonical_root(), + self.next_block.clone(), + std::time::Duration::default(), + BlockProcessType::ParentLookup { + chain_hash: Hash256::random(), + }, + ) + .unwrap(); } pub fn enqueue_single_lookup_rpc_block(&self) { - let event = WorkEvent::rpc_beacon_block( - self.next_block.canonical_root(), - self.next_block.clone(), - std::time::Duration::default(), - BlockProcessType::SingleBlock { id: 1 }, - ); - self.beacon_processor_tx.try_send(event).unwrap(); + self.network_beacon_processor + .send_rpc_beacon_block( + self.next_block.canonical_root(), + self.next_block.clone(), + std::time::Duration::default(), + BlockProcessType::SingleBlock { id: 1 }, + ) + .unwrap(); } pub fn enqueue_backfill_batch(&self) { - let event = WorkEvent::chain_segment( - ChainSegmentProcessId::BackSyncBatchId(Epoch::default()), - Vec::default(), - ); - self.beacon_processor_tx.try_send(event).unwrap(); + self.network_beacon_processor + .send_chain_segment( + ChainSegmentProcessId::BackSyncBatchId(Epoch::default()), + Vec::default(), + ) + .unwrap(); } pub fn enqueue_unaggregated_attestation(&self) { let (attestation, subnet_id) = self.attestations.first().unwrap().clone(); - self.beacon_processor_tx - .try_send(WorkEvent::unaggregated_attestation( + self.network_beacon_processor + .send_unaggregated_attestation( junk_message_id(), junk_peer_id(), attestation, subnet_id, true, Duration::from_secs(0), - )) + ) .unwrap(); } pub fn enqueue_gossip_attester_slashing(&self) { - self.beacon_processor_tx - .try_send(WorkEvent::gossip_attester_slashing( + self.network_beacon_processor + .send_gossip_attester_slashing( junk_message_id(), junk_peer_id(), Box::new(self.attester_slashing.clone()), - )) + ) .unwrap(); } pub fn enqueue_gossip_proposer_slashing(&self) { - self.beacon_processor_tx - .try_send(WorkEvent::gossip_proposer_slashing( + self.network_beacon_processor + .send_gossip_proposer_slashing( junk_message_id(), junk_peer_id(), Box::new(self.proposer_slashing.clone()), - )) + ) .unwrap(); } pub fn enqueue_gossip_voluntary_exit(&self) { - self.beacon_processor_tx - .try_send(WorkEvent::gossip_voluntary_exit( + self.network_beacon_processor + .send_gossip_voluntary_exit( junk_message_id(), junk_peer_id(), Box::new(self.voluntary_exit.clone()), - )) + ) .unwrap(); } pub fn enqueue_next_block_unaggregated_attestation(&self) { let (attestation, subnet_id) = self.next_block_attestations.first().unwrap().clone(); - self.beacon_processor_tx - .try_send(WorkEvent::unaggregated_attestation( + self.network_beacon_processor + .send_unaggregated_attestation( junk_message_id(), junk_peer_id(), attestation, subnet_id, true, Duration::from_secs(0), - )) + ) .unwrap(); } @@ -340,13 +367,13 @@ impl TestRig { .first() .unwrap() .clone(); - self.beacon_processor_tx - .try_send(WorkEvent::aggregated_attestation( + self.network_beacon_processor + .send_aggregated_attestation( junk_message_id(), junk_peer_id(), aggregate, Duration::from_secs(0), - )) + ) .unwrap(); } diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 7a91f2d0b1a..c8332705cfa 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -5,16 +5,16 @@ //! syncing-related responses to the Sync manager. #![allow(clippy::unit_arg)] -use crate::beacon_processor::{ - BeaconProcessor, BeaconProcessorSend, InvalidBlockStorage, WorkEvent as BeaconWorkEvent, - MAX_WORK_EVENT_QUEUE_LEN, -}; use crate::error; +use crate::network_beacon_processor::{InvalidBlockStorage, NetworkBeaconProcessor}; use crate::service::{NetworkMessage, RequestId}; use crate::status::status_message; use crate::sync::manager::RequestId as SyncId; use crate::sync::SyncMessage; use beacon_chain::{BeaconChain, BeaconChainTypes}; +use beacon_processor::{ + work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend, DuplicateCache, +}; use futures::prelude::*; use lighthouse_network::rpc::*; use lighthouse_network::{ @@ -23,7 +23,6 @@ use lighthouse_network::{ use logging::TimeLatch; use slog::{debug, o, trace}; use slog::{error, warn}; -use std::cmp; use std::sync::Arc; use std::time::{Duration, SystemTime, UNIX_EPOCH}; use tokio::sync::mpsc; @@ -41,7 +40,7 @@ pub struct Router { /// A network context to return and handle RPC requests. network: HandlerNetworkContext, /// A multi-threaded, non-blocking processor for applying messages to the beacon chain. - beacon_processor_send: BeaconProcessorSend, + network_beacon_processor: Arc>, /// The `Router` logger. log: slog::Logger, /// Provides de-bounce functionality for logging. @@ -80,12 +79,15 @@ pub enum RouterMessage { impl Router { /// Initializes and runs the Router. + #[allow(clippy::too_many_arguments)] pub fn spawn( beacon_chain: Arc>, network_globals: Arc>, network_send: mpsc::UnboundedSender>, executor: task_executor::TaskExecutor, invalid_block_storage: InvalidBlockStorage, + beacon_processor_send: BeaconProcessorSend, + beacon_processor_reprocess_tx: mpsc::Sender, log: slog::Logger, ) -> error::Result>> { let message_handler_log = log.new(o!("service"=> "router")); @@ -93,42 +95,41 @@ impl Router { let (handler_send, handler_recv) = mpsc::unbounded_channel(); - let (beacon_processor_send, beacon_processor_receive) = - mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); - let sync_logger = log.new(o!("service"=> "sync")); + // generate the message channel + let (sync_send, sync_recv) = mpsc::unbounded_channel::>(); + + let network_beacon_processor = NetworkBeaconProcessor { + beacon_processor_send, + duplicate_cache: DuplicateCache::default(), + chain: beacon_chain.clone(), + network_tx: network_send.clone(), + sync_tx: sync_send.clone(), + reprocess_tx: beacon_processor_reprocess_tx, + network_globals: network_globals.clone(), + invalid_block_storage, + executor: executor.clone(), + log: log.clone(), + }; + let network_beacon_processor = Arc::new(network_beacon_processor); // spawn the sync thread - let sync_send = crate::sync::manager::spawn( + crate::sync::manager::spawn( executor.clone(), beacon_chain.clone(), - network_globals.clone(), network_send.clone(), - BeaconProcessorSend(beacon_processor_send.clone()), + network_beacon_processor.clone(), + sync_recv, sync_logger, ); - BeaconProcessor { - beacon_chain: Arc::downgrade(&beacon_chain), - network_tx: network_send.clone(), - sync_tx: sync_send.clone(), - network_globals: network_globals.clone(), - executor: executor.clone(), - max_workers: cmp::max(1, num_cpus::get()), - current_workers: 0, - importing_blocks: Default::default(), - invalid_block_storage, - log: log.clone(), - } - .spawn_manager(beacon_processor_receive, None); - // generate the Message handler let mut handler = Router { network_globals, chain: beacon_chain, sync_send, network: HandlerNetworkContext::new(network_send, log.clone()), - beacon_processor_send: BeaconProcessorSend(beacon_processor_send), + network_beacon_processor, log: message_handler_log, logger_debounce: TimeLatch::default(), }; @@ -197,14 +198,17 @@ impl Router { Request::Status(status_message) => { self.on_status_request(peer_id, request_id, status_message) } - Request::BlocksByRange(request) => self.send_beacon_processor_work( - BeaconWorkEvent::blocks_by_range_request(peer_id, request_id, request), + Request::BlocksByRange(request) => self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_blocks_by_range_request(peer_id, request_id, request), ), - Request::BlocksByRoot(request) => self.send_beacon_processor_work( - BeaconWorkEvent::blocks_by_roots_request(peer_id, request_id, request), + Request::BlocksByRoot(request) => self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_blocks_by_roots_request(peer_id, request_id, request), ), - Request::LightClientBootstrap(request) => self.send_beacon_processor_work( - BeaconWorkEvent::lightclient_bootstrap_request(peer_id, request_id, request), + Request::LightClientBootstrap(request) => self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_lightclient_bootstrap_request(peer_id, request_id, request), ), } } @@ -219,10 +223,10 @@ impl Router { match response { Response::Status(status_message) => { debug!(self.log, "Received Status Response"; "peer_id" => %peer_id, &status_message); - self.send_beacon_processor_work(BeaconWorkEvent::status_message( - peer_id, - status_message, - )) + self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_status_message(peer_id, status_message), + ) } Response::BlocksByRange(beacon_block) => { self.on_blocks_by_range_response(peer_id, request_id, beacon_block); @@ -247,36 +251,40 @@ impl Router { ) { match gossip_message { PubsubMessage::AggregateAndProofAttestation(aggregate_and_proof) => self - .send_beacon_processor_work(BeaconWorkEvent::aggregated_attestation( - message_id, - peer_id, - *aggregate_and_proof, - timestamp_now(), - )), - PubsubMessage::Attestation(subnet_attestation) => { - self.send_beacon_processor_work(BeaconWorkEvent::unaggregated_attestation( - message_id, - peer_id, - subnet_attestation.1, - subnet_attestation.0, - should_process, - timestamp_now(), - )) - } - PubsubMessage::BeaconBlock(block) => { - self.send_beacon_processor_work(BeaconWorkEvent::gossip_beacon_block( + .handle_beacon_processor_send_result( + self.network_beacon_processor.send_aggregated_attestation( + message_id, + peer_id, + *aggregate_and_proof, + timestamp_now(), + ), + ), + PubsubMessage::Attestation(subnet_attestation) => self + .handle_beacon_processor_send_result( + self.network_beacon_processor.send_unaggregated_attestation( + message_id, + peer_id, + subnet_attestation.1, + subnet_attestation.0, + should_process, + timestamp_now(), + ), + ), + PubsubMessage::BeaconBlock(block) => self.handle_beacon_processor_send_result( + self.network_beacon_processor.send_gossip_beacon_block( message_id, peer_id, self.network_globals.client(&peer_id), block, timestamp_now(), - )) - } + ), + ), PubsubMessage::VoluntaryExit(exit) => { debug!(self.log, "Received a voluntary exit"; "peer_id" => %peer_id); - self.send_beacon_processor_work(BeaconWorkEvent::gossip_voluntary_exit( - message_id, peer_id, exit, - )) + self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_gossip_voluntary_exit(message_id, peer_id, exit), + ) } PubsubMessage::ProposerSlashing(proposer_slashing) => { debug!( @@ -284,11 +292,13 @@ impl Router { "Received a proposer slashing"; "peer_id" => %peer_id ); - self.send_beacon_processor_work(BeaconWorkEvent::gossip_proposer_slashing( - message_id, - peer_id, - proposer_slashing, - )) + self.handle_beacon_processor_send_result( + self.network_beacon_processor.send_gossip_proposer_slashing( + message_id, + peer_id, + proposer_slashing, + ), + ) } PubsubMessage::AttesterSlashing(attester_slashing) => { debug!( @@ -296,11 +306,13 @@ impl Router { "Received a attester slashing"; "peer_id" => %peer_id ); - self.send_beacon_processor_work(BeaconWorkEvent::gossip_attester_slashing( - message_id, - peer_id, - attester_slashing, - )) + self.handle_beacon_processor_send_result( + self.network_beacon_processor.send_gossip_attester_slashing( + message_id, + peer_id, + attester_slashing, + ), + ) } PubsubMessage::SignedContributionAndProof(contribution_and_proof) => { trace!( @@ -308,12 +320,14 @@ impl Router { "Received sync committee aggregate"; "peer_id" => %peer_id ); - self.send_beacon_processor_work(BeaconWorkEvent::gossip_sync_contribution( - message_id, - peer_id, - *contribution_and_proof, - timestamp_now(), - )) + self.handle_beacon_processor_send_result( + self.network_beacon_processor.send_gossip_sync_contribution( + message_id, + peer_id, + *contribution_and_proof, + timestamp_now(), + ), + ) } PubsubMessage::SyncCommitteeMessage(sync_committtee_msg) => { trace!( @@ -321,13 +335,15 @@ impl Router { "Received sync committee signature"; "peer_id" => %peer_id ); - self.send_beacon_processor_work(BeaconWorkEvent::gossip_sync_signature( - message_id, - peer_id, - sync_committtee_msg.1, - sync_committtee_msg.0, - timestamp_now(), - )) + self.handle_beacon_processor_send_result( + self.network_beacon_processor.send_gossip_sync_signature( + message_id, + peer_id, + sync_committtee_msg.1, + sync_committtee_msg.0, + timestamp_now(), + ), + ) } PubsubMessage::LightClientFinalityUpdate(light_client_finality_update) => { trace!( @@ -335,13 +351,14 @@ impl Router { "Received light client finality update"; "peer_id" => %peer_id ); - self.send_beacon_processor_work( - BeaconWorkEvent::gossip_light_client_finality_update( - message_id, - peer_id, - light_client_finality_update, - timestamp_now(), - ), + self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_gossip_light_client_finality_update( + message_id, + peer_id, + *light_client_finality_update, + timestamp_now(), + ), ) } PubsubMessage::LightClientOptimisticUpdate(light_client_optimistic_update) => { @@ -350,21 +367,25 @@ impl Router { "Received light client optimistic update"; "peer_id" => %peer_id ); - self.send_beacon_processor_work( - BeaconWorkEvent::gossip_light_client_optimistic_update( - message_id, - peer_id, - light_client_optimistic_update, - timestamp_now(), - ), + self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_gossip_light_client_optimistic_update( + message_id, + peer_id, + *light_client_optimistic_update, + timestamp_now(), + ), ) } PubsubMessage::BlsToExecutionChange(bls_to_execution_change) => self - .send_beacon_processor_work(BeaconWorkEvent::gossip_bls_to_execution_change( - message_id, - peer_id, - bls_to_execution_change, - )), + .handle_beacon_processor_send_result( + self.network_beacon_processor + .send_gossip_bls_to_execution_change( + message_id, + peer_id, + bls_to_execution_change, + ), + ), } } @@ -415,7 +436,10 @@ impl Router { request_id, ); - self.send_beacon_processor_work(BeaconWorkEvent::status_message(peer_id, status)) + self.handle_beacon_processor_send_result( + self.network_beacon_processor + .send_status_message(peer_id, status), + ) } /// Handle a `BlocksByRange` response from the peer. @@ -480,20 +504,22 @@ impl Router { }); } - fn send_beacon_processor_work(&mut self, work: BeaconWorkEvent) { - self.beacon_processor_send - .try_send(work) - .unwrap_or_else(|e| { - let work_type = match &*e { - mpsc::error::TrySendError::Closed(work) - | mpsc::error::TrySendError::Full(work) => work.work_type(), - }; - - if self.logger_debounce.elapsed() { - error!(&self.log, "Unable to send message to the beacon processor"; - "error" => %e, "type" => work_type) + fn handle_beacon_processor_send_result( + &mut self, + result: Result<(), crate::network_beacon_processor::Error>, + ) { + if let Err(e) = result { + let work_type = match &e { + mpsc::error::TrySendError::Closed(work) | mpsc::error::TrySendError::Full(work) => { + work.work_type() } - }) + }; + + if self.logger_debounce.elapsed() { + error!(&self.log, "Unable to send message to the beacon processor"; + "error" => %e, "type" => work_type) + } + } } } diff --git a/beacon_node/network/src/service.rs b/beacon_node/network/src/service.rs index 2c919233fca..c2719477f1f 100644 --- a/beacon_node/network/src/service.rs +++ b/beacon_node/network/src/service.rs @@ -1,5 +1,5 @@ use super::sync::manager::RequestId as SyncId; -use crate::beacon_processor::InvalidBlockStorage; +use crate::network_beacon_processor::InvalidBlockStorage; use crate::persisted_dht::{clear_dht, load_dht, persist_dht}; use crate::router::{Router, RouterMessage}; use crate::subnet_service::SyncCommitteeService; @@ -9,6 +9,7 @@ use crate::{ NetworkConfig, }; use beacon_chain::{BeaconChain, BeaconChainTypes}; +use beacon_processor::{work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend}; use futures::channel::mpsc::Sender; use futures::future::OptionFuture; use futures::prelude::*; @@ -224,6 +225,8 @@ impl NetworkService { config: &NetworkConfig, executor: task_executor::TaskExecutor, gossipsub_registry: Option<&'_ mut Registry>, + beacon_processor_send: BeaconProcessorSend, + beacon_processor_reprocess_tx: mpsc::Sender, ) -> error::Result<(Arc>, NetworkSenders)> { let network_log = executor.log().clone(); // build the channels for external comms @@ -311,6 +314,8 @@ impl NetworkService { network_senders.network_send(), executor.clone(), invalid_block_storage, + beacon_processor_send, + beacon_processor_reprocess_tx, network_log.clone(), )?; diff --git a/beacon_node/network/src/service/tests.rs b/beacon_node/network/src/service/tests.rs index 83fcc8c9ac8..9943da15a0d 100644 --- a/beacon_node/network/src/service/tests.rs +++ b/beacon_node/network/src/service/tests.rs @@ -4,12 +4,15 @@ mod tests { use crate::persisted_dht::load_dht; use crate::{NetworkConfig, NetworkService}; use beacon_chain::test_utils::BeaconChainHarness; + use beacon_processor::{ + BeaconProcessorSend, MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN, + }; use lighthouse_network::Enr; use slog::{o, Drain, Level, Logger}; use sloggers::{null::NullLoggerBuilder, Build}; use std::str::FromStr; use std::sync::Arc; - use tokio::runtime::Runtime; + use tokio::{runtime::Runtime, sync::mpsc}; use types::MinimalEthSpec; fn get_logger(actual_log: bool) -> Logger { @@ -67,10 +70,20 @@ mod tests { // Create a new network service which implicitly gets dropped at the // end of the block. - let _network_service = - NetworkService::start(beacon_chain.clone(), &config, executor, None) - .await - .unwrap(); + let (beacon_processor_send, _beacon_processor_receive) = + mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN); + let (beacon_processor_reprocess_tx, _beacon_processor_reprocess_rx) = + mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN); + let _network_service = NetworkService::start( + beacon_chain.clone(), + &config, + executor, + None, + BeaconProcessorSend(beacon_processor_send), + beacon_processor_reprocess_tx, + ) + .await + .unwrap(); drop(signal); }); diff --git a/beacon_node/network/src/sync/backfill_sync/mod.rs b/beacon_node/network/src/sync/backfill_sync/mod.rs index 9f676ba017d..a1c2404e5ed 100644 --- a/beacon_node/network/src/sync/backfill_sync/mod.rs +++ b/beacon_node/network/src/sync/backfill_sync/mod.rs @@ -8,7 +8,7 @@ //! If a batch fails, the backfill sync cannot progress. In this scenario, we mark the backfill //! sync as failed, log an error and attempt to retry once a new peer joins the node. -use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent as BeaconWorkEvent}; +use crate::network_beacon_processor::ChainSegmentProcessId; use crate::sync::manager::{BatchProcessResult, Id}; use crate::sync::network_context::SyncNetworkContext; use crate::sync::range_sync::{ @@ -537,8 +537,8 @@ impl BackFillSync { self.current_processing_batch = Some(batch_id); if let Err(e) = network - .processor_channel() - .try_send(BeaconWorkEvent::chain_segment(process_id, blocks)) + .beacon_processor() + .send_chain_segment(process_id, blocks) { crit!(self.log, "Failed to send backfill segment to processor."; "msg" => "process_batch", "error" => %e, "batch" => self.processing_target); diff --git a/beacon_node/network/src/sync/block_lookups/mod.rs b/beacon_node/network/src/sync/block_lookups/mod.rs index aa2694769c2..4340aa41d8b 100644 --- a/beacon_node/network/src/sync/block_lookups/mod.rs +++ b/beacon_node/network/src/sync/block_lookups/mod.rs @@ -2,6 +2,7 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; use std::time::Duration; +use crate::network_beacon_processor::ChainSegmentProcessId; use beacon_chain::{BeaconChainTypes, BlockError}; use fnv::FnvHashMap; use lighthouse_network::{PeerAction, PeerId}; @@ -11,7 +12,6 @@ use smallvec::SmallVec; use std::sync::Arc; use store::{Hash256, SignedBeaconBlock}; -use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent}; use crate::metrics; use self::parent_lookup::PARENT_FAIL_TOLERANCE; @@ -542,8 +542,8 @@ impl BlockLookups { BlockProcessResult::Ok | BlockProcessResult::Err(BlockError::BlockIsAlreadyKnown { .. }) => { // Check if the beacon processor is available - let beacon_processor_send = match cx.processor_channel_if_enabled() { - Some(channel) => channel, + let beacon_processor = match cx.beacon_processor_if_enabled() { + Some(beacon_processor) => beacon_processor, None => { return trace!( self.log, @@ -555,7 +555,7 @@ impl BlockLookups { let (chain_hash, blocks, hashes, request) = parent_lookup.parts_for_processing(); let process_id = ChainSegmentProcessId::ParentLookup(chain_hash); - match beacon_processor_send.try_send(WorkEvent::chain_segment(process_id, blocks)) { + match beacon_processor.send_chain_segment(process_id, blocks) { Ok(_) => { self.processing_parent_lookups .insert(chain_hash, (hashes, request)); @@ -664,11 +664,15 @@ impl BlockLookups { process_type: BlockProcessType, cx: &mut SyncNetworkContext, ) -> Result<(), ()> { - match cx.processor_channel_if_enabled() { - Some(beacon_processor_send) => { + match cx.beacon_processor_if_enabled() { + Some(beacon_processor) => { trace!(self.log, "Sending block for processing"; "block" => ?block_root, "process" => ?process_type); - let event = WorkEvent::rpc_beacon_block(block_root, block, duration, process_type); - if let Err(e) = beacon_processor_send.try_send(event) { + if let Err(e) = beacon_processor.send_rpc_beacon_block( + block_root, + block, + duration, + process_type, + ) { error!( self.log, "Failed to send sync block to processor"; diff --git a/beacon_node/network/src/sync/block_lookups/tests.rs b/beacon_node/network/src/sync/block_lookups/tests.rs index 82334db0f8e..c588f867bd9 100644 --- a/beacon_node/network/src/sync/block_lookups/tests.rs +++ b/beacon_node/network/src/sync/block_lookups/tests.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use crate::beacon_processor::BeaconProcessorSend; +use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::RequestId; use crate::sync::manager::RequestId as SyncId; use crate::NetworkMessage; @@ -9,18 +9,19 @@ use super::*; use beacon_chain::builder::Witness; use beacon_chain::eth1_chain::CachingEth1Backend; +use beacon_processor::WorkEvent; use lighthouse_network::{NetworkGlobals, Request}; use slog::{Drain, Level}; -use slot_clock::SystemTimeSlotClock; +use slot_clock::ManualSlotClock; use store::MemoryStore; use tokio::sync::mpsc; use types::test_utils::{SeedableRng, TestRandom, XorShiftRng}; use types::MinimalEthSpec as E; -type T = Witness, E, MemoryStore, MemoryStore>; +type T = Witness, E, MemoryStore, MemoryStore>; struct TestRig { - beacon_processor_rx: mpsc::Receiver>, + beacon_processor_rx: mpsc::Receiver>, network_rx: mpsc::UnboundedReceiver>, rng: XorShiftRng, } @@ -41,8 +42,10 @@ impl TestRig { } }; - let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(100); let (network_tx, network_rx) = mpsc::unbounded_channel(); + let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log)); + let (network_beacon_processor, beacon_processor_rx) = + NetworkBeaconProcessor::null_for_testing(globals); let rng = XorShiftRng::from_seed([42; 16]); let rig = TestRig { beacon_processor_rx, @@ -51,11 +54,9 @@ impl TestRig { }; let bl = BlockLookups::new(log.new(slog::o!("component" => "block_lookups"))); let cx = { - let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log)); SyncNetworkContext::new( network_tx, - globals, - BeaconProcessorSend(beacon_processor_tx), + Arc::new(network_beacon_processor), log.new(slog::o!("component" => "network_context")), ) }; @@ -102,7 +103,7 @@ impl TestRig { fn expect_block_process(&mut self) { match self.beacon_processor_rx.try_recv() { Ok(work) => { - assert_eq!(work.work_type(), crate::beacon_processor::RPC_BLOCK); + assert_eq!(work.work_type(), beacon_processor::RPC_BLOCK); } other => panic!("Expected block process, found {:?}", other), } @@ -112,7 +113,7 @@ impl TestRig { fn expect_parent_chain_process(&mut self) { match self.beacon_processor_rx.try_recv() { Ok(work) => { - assert_eq!(work.work_type(), crate::beacon_processor::CHAIN_SEGMENT); + assert_eq!(work.work_type(), beacon_processor::CHAIN_SEGMENT); } other => panic!("Expected chain segment process, found {:?}", other), } diff --git a/beacon_node/network/src/sync/manager.rs b/beacon_node/network/src/sync/manager.rs index c24d4c192b1..72542752c51 100644 --- a/beacon_node/network/src/sync/manager.rs +++ b/beacon_node/network/src/sync/manager.rs @@ -38,7 +38,7 @@ use super::block_lookups::BlockLookups; use super::network_context::SyncNetworkContext; use super::peer_sync_info::{remote_sync_type, PeerSyncType}; use super::range_sync::{RangeSync, RangeSyncType, EPOCHS_PER_BATCH}; -use crate::beacon_processor::{BeaconProcessorSend, ChainSegmentProcessId}; +use crate::network_beacon_processor::{ChainSegmentProcessId, NetworkBeaconProcessor}; use crate::service::NetworkMessage; use crate::status::ToStatusMessage; use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, EngineState}; @@ -159,9 +159,6 @@ pub struct SyncManager { /// A reference to the underlying beacon chain. chain: Arc>, - /// A reference to the network globals and peer-db. - network_globals: Arc>, - /// A receiving channel sent by the message processor thread. input_channel: mpsc::UnboundedReceiver>, @@ -186,29 +183,22 @@ pub struct SyncManager { pub fn spawn( executor: task_executor::TaskExecutor, beacon_chain: Arc>, - network_globals: Arc>, network_send: mpsc::UnboundedSender>, - beacon_processor_send: BeaconProcessorSend, + beacon_processor: Arc>, + sync_recv: mpsc::UnboundedReceiver>, log: slog::Logger, -) -> mpsc::UnboundedSender> { +) { assert!( MAX_REQUEST_BLOCKS >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH, "Max blocks that can be requested in a single batch greater than max allowed blocks in a single request" ); - // generate the message channel - let (sync_send, sync_recv) = mpsc::unbounded_channel::>(); // create an instance of the SyncManager + let network_globals = beacon_processor.network_globals.clone(); let mut sync_manager = SyncManager { chain: beacon_chain.clone(), - network_globals: network_globals.clone(), input_channel: sync_recv, - network: SyncNetworkContext::new( - network_send, - network_globals.clone(), - beacon_processor_send, - log.clone(), - ), + network: SyncNetworkContext::new(network_send, beacon_processor, log.clone()), range_sync: RangeSync::new(beacon_chain.clone(), log.clone()), backfill_sync: BackFillSync::new(beacon_chain, network_globals, log.clone()), block_lookups: BlockLookups::new(log.clone()), @@ -218,10 +208,13 @@ pub fn spawn( // spawn the sync manager thread debug!(log, "Sync Manager started"); executor.spawn(async move { Box::pin(sync_manager.main()).await }, "sync"); - sync_send } impl SyncManager { + fn network_globals(&self) -> &NetworkGlobals { + self.network.network_globals() + } + /* Input Handling Functions */ /// A peer has connected which has blocks that are unknown to us. @@ -322,12 +315,12 @@ impl SyncManager { let rpr = new_state.as_str(); // Drop the write lock let update_sync_status = self - .network_globals + .network_globals() .peers .write() .update_sync_status(peer_id, new_state.clone()); if let Some(was_updated) = update_sync_status { - let is_connected = self.network_globals.peers.read().is_connected(peer_id); + let is_connected = self.network_globals().peers.read().is_connected(peer_id); if was_updated { debug!( self.log, @@ -383,7 +376,7 @@ impl SyncManager { let head = self.chain.best_slot(); let current_slot = self.chain.slot().unwrap_or_else(|_| Slot::new(0)); - let peers = self.network_globals.peers.read(); + let peers = self.network_globals().peers.read(); if current_slot >= head && current_slot.sub(head) <= (SLOT_IMPORT_TOLERANCE as u64) && head > 0 @@ -445,8 +438,8 @@ impl SyncManager { }, }; - let old_state = self.network_globals.set_sync_state(new_state); - let new_state = self.network_globals.sync_state.read(); + let old_state = self.network_globals().set_sync_state(new_state); + let new_state = self.network_globals().sync_state.read().clone(); if !new_state.eq(&old_state) { info!(self.log, "Sync state updated"; "old_state" => %old_state, "new_state" => %new_state); // If we have become synced - Subscribe to all the core subnet topics @@ -505,7 +498,7 @@ impl SyncManager { } SyncMessage::UnknownBlock(peer_id, block, block_root) => { // If we are not synced or within SLOT_IMPORT_TOLERANCE of the block, ignore - if !self.network_globals.sync_state.read().is_synced() { + if !self.network_globals().sync_state.read().is_synced() { let head_slot = self.chain.canonical_head.cached_head().head_slot(); let unknown_block_slot = block.slot(); @@ -519,7 +512,7 @@ impl SyncManager { return; } } - if self.network_globals.peers.read().is_connected(&peer_id) + if self.network_globals().peers.read().is_connected(&peer_id) && self.network.is_execution_engine_online() { self.block_lookups @@ -528,8 +521,8 @@ impl SyncManager { } SyncMessage::UnknownBlockHash(peer_id, block_hash) => { // If we are not synced, ignore this block. - if self.network_globals.sync_state.read().is_synced() - && self.network_globals.peers.read().is_connected(&peer_id) + if self.network_globals().sync_state.read().is_synced() + && self.network_globals().peers.read().is_connected(&peer_id) && self.network.is_execution_engine_online() { self.block_lookups diff --git a/beacon_node/network/src/sync/network_context.rs b/beacon_node/network/src/sync/network_context.rs index 03c466eecea..adc235130b0 100644 --- a/beacon_node/network/src/sync/network_context.rs +++ b/beacon_node/network/src/sync/network_context.rs @@ -3,7 +3,7 @@ use super::manager::{Id, RequestId as SyncRequestId}; use super::range_sync::{BatchId, ChainId}; -use crate::beacon_processor::BeaconProcessorSend; +use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::{NetworkMessage, RequestId}; use crate::status::ToStatusMessage; use beacon_chain::{BeaconChainTypes, EngineState}; @@ -20,9 +20,6 @@ pub struct SyncNetworkContext { /// The network channel to relay messages to the Network service. network_send: mpsc::UnboundedSender>, - /// Access to the network global vars. - network_globals: Arc>, - /// A sequential ID for all RPC requests. request_id: Id, @@ -36,8 +33,8 @@ pub struct SyncNetworkContext { /// `beacon_processor_send`. execution_engine_state: EngineState, - /// Channel to send work to the beacon processor. - beacon_processor_send: BeaconProcessorSend, + /// Sends work to the beacon processor via a channel. + network_beacon_processor: Arc>, /// Logger for the `SyncNetworkContext`. log: slog::Logger, @@ -46,25 +43,27 @@ pub struct SyncNetworkContext { impl SyncNetworkContext { pub fn new( network_send: mpsc::UnboundedSender>, - network_globals: Arc>, - beacon_processor_send: BeaconProcessorSend, + network_beacon_processor: Arc>, log: slog::Logger, ) -> Self { Self { network_send, execution_engine_state: EngineState::Online, // always assume `Online` at the start - network_globals, request_id: 1, range_requests: FnvHashMap::default(), backfill_requests: FnvHashMap::default(), - beacon_processor_send, + network_beacon_processor, log, } } + pub fn network_globals(&self) -> &NetworkGlobals { + &self.network_beacon_processor.network_globals + } + /// Returns the Client type of the peer if known pub fn client_type(&self, peer_id: &PeerId) -> Client { - self.network_globals + self.network_globals() .peers .read() .peer_info(peer_id) @@ -278,13 +277,13 @@ impl SyncNetworkContext { }) } - pub fn processor_channel_if_enabled(&self) -> Option<&BeaconProcessorSend> { + pub fn beacon_processor_if_enabled(&self) -> Option<&Arc>> { self.is_execution_engine_online() - .then_some(&self.beacon_processor_send) + .then_some(&self.network_beacon_processor) } - pub fn processor_channel(&self) -> &BeaconProcessorSend { - &self.beacon_processor_send + pub fn beacon_processor(&self) -> &Arc> { + &self.network_beacon_processor } fn next_id(&mut self) -> Id { diff --git a/beacon_node/network/src/sync/range_sync/chain.rs b/beacon_node/network/src/sync/range_sync/chain.rs index 51ca9e2b071..af547885dca 100644 --- a/beacon_node/network/src/sync/range_sync/chain.rs +++ b/beacon_node/network/src/sync/range_sync/chain.rs @@ -1,5 +1,5 @@ use super::batch::{BatchInfo, BatchProcessingResult, BatchState}; -use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent as BeaconWorkEvent}; +use crate::network_beacon_processor::ChainSegmentProcessId; use crate::sync::{ manager::Id, network_context::SyncNetworkContext, BatchOperationOutcome, BatchProcessResult, }; @@ -294,8 +294,8 @@ impl SyncingChain { return Ok(KeepChain); } - let beacon_processor_send = match network.processor_channel_if_enabled() { - Some(channel) => channel, + let beacon_processor = match network.beacon_processor_if_enabled() { + Some(beacon_processor) => beacon_processor, None => return Ok(KeepChain), }; @@ -317,9 +317,7 @@ impl SyncingChain { let process_id = ChainSegmentProcessId::RangeBatchId(self.id, batch_id); self.current_processing_batch = Some(batch_id); - if let Err(e) = - beacon_processor_send.try_send(BeaconWorkEvent::chain_segment(process_id, blocks)) - { + if let Err(e) = beacon_processor.send_chain_segment(process_id, blocks) { crit!(self.log, "Failed to send chain segment to processor."; "msg" => "process_batch", "error" => %e, "batch" => self.processing_target); // This is unlikely to happen but it would stall syncing since the batch now has no diff --git a/beacon_node/network/src/sync/range_sync/range.rs b/beacon_node/network/src/sync/range_sync/range.rs index 2c35c57d9e4..05ad5204b9e 100644 --- a/beacon_node/network/src/sync/range_sync/range.rs +++ b/beacon_node/network/src/sync/range_sync/range.rs @@ -371,22 +371,23 @@ where #[cfg(test)] mod tests { + use crate::network_beacon_processor::NetworkBeaconProcessor; use crate::service::RequestId; use crate::NetworkMessage; use super::*; - use crate::beacon_processor::{BeaconProcessorSend, WorkEvent as BeaconWorkEvent}; use beacon_chain::builder::Witness; use beacon_chain::eth1_chain::CachingEth1Backend; use beacon_chain::parking_lot::RwLock; use beacon_chain::EngineState; + use beacon_processor::WorkEvent as BeaconWorkEvent; use lighthouse_network::rpc::BlocksByRangeRequest; use lighthouse_network::Request; use lighthouse_network::{rpc::StatusMessage, NetworkGlobals}; use slog::{o, Drain}; use tokio::sync::mpsc; - use slot_clock::SystemTimeSlotClock; + use slot_clock::ManualSlotClock; use std::collections::HashSet; use std::sync::Arc; use store::MemoryStore; @@ -437,7 +438,7 @@ mod tests { } type TestBeaconChainType = - Witness, E, MemoryStore, MemoryStore>; + Witness, E, MemoryStore, MemoryStore>; fn build_log(level: slog::Level, enabled: bool) -> slog::Logger { let decorator = slog_term::TermDecorator::new().build(); @@ -455,7 +456,7 @@ mod tests { struct TestRig { log: slog::Logger, /// To check what does sync send to the beacon processor. - beacon_processor_rx: mpsc::Receiver>, + beacon_processor_rx: mpsc::Receiver>, /// To set up different scenarios where sync is told about known/unkown blocks. chain: Arc, /// Needed by range to handle communication with the network. @@ -583,7 +584,7 @@ mod tests { fn expect_chain_segment(&mut self) { match self.beacon_processor_rx.try_recv() { Ok(work) => { - assert_eq!(work.work_type(), crate::beacon_processor::CHAIN_SEGMENT); + assert_eq!(work.work_type(), beacon_processor::CHAIN_SEGMENT); } other => panic!("Expected chain segment process, found {:?}", other), } @@ -593,17 +594,17 @@ mod tests { fn range(log_enabled: bool) -> (TestRig, RangeSync) { let chain = Arc::new(FakeStorage::default()); let log = build_log(slog::Level::Trace, log_enabled); - let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(10); let range_sync = RangeSync::::new( chain.clone(), log.new(o!("component" => "range")), ); let (network_tx, network_rx) = mpsc::unbounded_channel(); let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log)); + let (network_beacon_processor, beacon_processor_rx) = + NetworkBeaconProcessor::null_for_testing(globals.clone()); let cx = SyncNetworkContext::new( network_tx, - globals.clone(), - BeaconProcessorSend(beacon_processor_tx), + Arc::new(network_beacon_processor), log.new(o!("component" => "network_context")), ); let test_rig = TestRig {