diff --git a/beacon_node/beacon_processor/src/lib.rs b/beacon_node/beacon_processor/src/lib.rs index e864cb1fd91..4eac2bded39 100644 --- a/beacon_node/beacon_processor/src/lib.rs +++ b/beacon_node/beacon_processor/src/lib.rs @@ -37,43 +37,32 @@ //! Whenever the manager receives a notification that a worker has finished a parcel of work, it //! checks the queues to see if there are more parcels of work that can be spawned in a new worker //! task. - -use crate::work_reprocessing_queue::{ - QueuedBackfillBatch, QueuedGossipBlock, ReprocessQueueMessage, -}; -use futures::stream::{Stream, StreamExt}; -use futures::task::Poll; +mod scheduler; +use crate::scheduler::interface::SchedulerType; use lighthouse_network::{MessageId, NetworkGlobals, PeerId}; -use logging::crit; -use logging::TimeLatch; use parking_lot::Mutex; +use scheduler::interface::Scheduler; use serde::{Deserialize, Serialize}; use slot_clock::SlotClock; use std::cmp; -use std::collections::{HashSet, VecDeque}; +use std::collections::HashSet; use std::fmt; use std::future::Future; use std::pin::Pin; use std::sync::Arc; -use std::task::Context; use std::time::Duration; +use strum::AsRefStr; use strum::IntoStaticStr; use task_executor::TaskExecutor; use tokio::sync::mpsc; use tokio::sync::mpsc::error::TrySendError; -use tracing::{debug, error, trace, warn}; + +use tracing::warn; use types::{ - Attestation, BeaconState, ChainSpec, EthSpec, Hash256, RelativeEpoch, SignedAggregateAndProof, + Attestation, BeaconState, ChainSpec, EthSpec, Hash256, SignedAggregateAndProof, SingleAttestation, Slot, SubnetId, }; -use work_reprocessing_queue::{ - spawn_reprocess_scheduler, QueuedAggregate, QueuedLightClientUpdate, QueuedRpcBlock, - QueuedUnaggregate, ReadyWork, -}; -use work_reprocessing_queue::{IgnoredRpcBlock, QueuedSamplingRequest}; - mod metrics; -pub mod work_reprocessing_queue; /// The maximum size of the channel for work events to the `BeaconProcessor`. /// @@ -89,131 +78,6 @@ const MAX_IDLE_QUEUE_LEN: usize = 16_384; /// The maximum size of the channel for re-processing work events. const DEFAULT_MAX_SCHEDULED_WORK_QUEUE_LEN: usize = 3 * DEFAULT_MAX_WORK_EVENT_QUEUE_LEN / 4; -/// Over-provision queues based on active validator count by some factor. The beacon chain has -/// strict churns that prevent the validator set size from changing rapidly. By over-provisioning -/// slightly, we don't need to adjust the queues during the lifetime of a process. -const ACTIVE_VALIDATOR_COUNT_OVERPROVISION_PERCENT: usize = 110; - -/// Minimum size of dynamically sized queues. Due to integer division we don't want 0 length queues -/// as the processor won't process that message type. 128 is an arbitrary value value >= 1 that -/// seems reasonable. -const MIN_QUEUE_LEN: usize = 128; - -/// Maximum number of queued items that will be stored before dropping them -pub struct BeaconProcessorQueueLengths { - aggregate_queue: usize, - attestation_queue: usize, - unknown_block_aggregate_queue: usize, - unknown_block_attestation_queue: usize, - sync_message_queue: usize, - sync_contribution_queue: usize, - gossip_voluntary_exit_queue: usize, - gossip_proposer_slashing_queue: usize, - gossip_attester_slashing_queue: usize, - unknown_light_client_update_queue: usize, - unknown_block_sampling_request_queue: usize, - rpc_block_queue: usize, - rpc_blob_queue: usize, - rpc_custody_column_queue: usize, - rpc_verify_data_column_queue: usize, - sampling_result_queue: usize, - chain_segment_queue: usize, - backfill_chain_segment: usize, - gossip_block_queue: usize, - gossip_blob_queue: usize, - gossip_data_column_queue: usize, - delayed_block_queue: usize, - status_queue: usize, - bbrange_queue: usize, - bbroots_queue: usize, - blbroots_queue: usize, - blbrange_queue: usize, - dcbroots_queue: usize, - dcbrange_queue: usize, - gossip_bls_to_execution_change_queue: usize, - lc_gossip_finality_update_queue: usize, - lc_gossip_optimistic_update_queue: usize, - lc_bootstrap_queue: usize, - lc_rpc_optimistic_update_queue: usize, - lc_rpc_finality_update_queue: usize, - lc_update_range_queue: usize, - api_request_p0_queue: usize, - api_request_p1_queue: usize, -} - -impl BeaconProcessorQueueLengths { - pub fn from_state( - state: &BeaconState, - spec: &ChainSpec, - ) -> Result { - let active_validator_count = - match state.get_cached_active_validator_indices(RelativeEpoch::Current) { - Ok(indices) => indices.len(), - Err(_) => state - .get_active_validator_indices(state.current_epoch(), spec) - .map_err(|e| format!("Error computing active indices: {:?}", e))? - .len(), - }; - let active_validator_count = - (ACTIVE_VALIDATOR_COUNT_OVERPROVISION_PERCENT * active_validator_count) / 100; - let slots_per_epoch = E::slots_per_epoch() as usize; - - Ok(Self { - aggregate_queue: 4096, - unknown_block_aggregate_queue: 1024, - // Capacity for a full slot's worth of attestations if subscribed to all subnets - attestation_queue: std::cmp::max( - active_validator_count / slots_per_epoch, - MIN_QUEUE_LEN, - ), - // Capacity for a full slot's worth of attestations if subscribed to all subnets - unknown_block_attestation_queue: std::cmp::max( - active_validator_count / slots_per_epoch, - MIN_QUEUE_LEN, - ), - sync_message_queue: 2048, - sync_contribution_queue: 1024, - gossip_voluntary_exit_queue: 4096, - gossip_proposer_slashing_queue: 4096, - gossip_attester_slashing_queue: 4096, - unknown_light_client_update_queue: 128, - rpc_block_queue: 1024, - rpc_blob_queue: 1024, - // TODO(das): Placeholder values - rpc_custody_column_queue: 1000, - rpc_verify_data_column_queue: 1000, - unknown_block_sampling_request_queue: 16384, - sampling_result_queue: 1000, - chain_segment_queue: 64, - backfill_chain_segment: 64, - gossip_block_queue: 1024, - gossip_blob_queue: 1024, - gossip_data_column_queue: 1024, - delayed_block_queue: 1024, - status_queue: 1024, - bbrange_queue: 1024, - bbroots_queue: 1024, - blbroots_queue: 1024, - blbrange_queue: 1024, - // TODO(das): pick proper values - dcbroots_queue: 1024, - dcbrange_queue: 1024, - gossip_bls_to_execution_change_queue: 16384, - lc_gossip_finality_update_queue: 1024, - lc_gossip_optimistic_update_queue: 1024, - lc_bootstrap_queue: 1024, - lc_rpc_optimistic_update_queue: 512, - lc_rpc_finality_update_queue: 512, - lc_update_range_queue: 512, - api_request_p0_queue: 1024, - api_request_p1_queue: 1024, - }) - } -} - -/// The name of the manager tokio task. -const MANAGER_TASK_NAME: &str = "beacon_processor_manager"; - /// The name of the worker tokio tasks. const WORKER_TASK_NAME: &str = "beacon_processor_worker"; @@ -243,6 +107,7 @@ pub struct BeaconProcessorConfig { pub max_gossip_attestation_batch_size: usize, pub max_gossip_aggregate_batch_size: usize, pub enable_backfill_rate_limiting: bool, + pub beacon_processor_type: BeaconProcessorType, } impl Default for BeaconProcessorConfig { @@ -254,6 +119,7 @@ impl Default for BeaconProcessorConfig { max_gossip_attestation_batch_size: DEFAULT_MAX_GOSSIP_ATTESTATION_BATCH_SIZE, max_gossip_aggregate_batch_size: DEFAULT_MAX_GOSSIP_AGGREGATE_BATCH_SIZE, enable_backfill_rate_limiting: true, + beacon_processor_type: BeaconProcessorType::EarliestDeadline, } } } @@ -262,22 +128,16 @@ impl Default for BeaconProcessorConfig { pub struct BeaconProcessorChannels { pub beacon_processor_tx: BeaconProcessorSend, pub beacon_processor_rx: mpsc::Receiver>, - pub work_reprocessing_tx: mpsc::Sender, - pub work_reprocessing_rx: mpsc::Receiver, } impl BeaconProcessorChannels { pub fn new(config: &BeaconProcessorConfig) -> Self { let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(config.max_work_event_queue_len); - let (work_reprocessing_tx, work_reprocessing_rx) = - mpsc::channel(config.max_scheduled_work_queue_len); Self { beacon_processor_tx: BeaconProcessorSend(beacon_processor_tx), beacon_processor_rx, - work_reprocessing_rx, - work_reprocessing_tx, } } } @@ -288,87 +148,10 @@ impl Default for BeaconProcessorChannels { } } -/// A simple first-in-first-out queue with a maximum length. -struct FifoQueue { - queue: VecDeque, - max_length: usize, -} - -impl FifoQueue { - /// Create a new, empty queue with the given length. - pub fn new(max_length: usize) -> Self { - Self { - queue: VecDeque::default(), - max_length, - } - } - - /// Add a new item to the queue. - /// - /// Drops `item` if the queue is full. - pub fn push(&mut self, item: T, item_desc: &str) { - if self.queue.len() == self.max_length { - error!( - msg = "the system has insufficient resources for load", - queue_len = self.max_length, - queue = item_desc, - "Work queue is full" - ) - } else { - self.queue.push_back(item); - } - } - - /// Remove the next item from the queue. - pub fn pop(&mut self) -> Option { - self.queue.pop_front() - } - - /// Returns the current length of the queue. - pub fn len(&self) -> usize { - self.queue.len() - } -} - -/// A simple last-in-first-out queue with a maximum length. -struct LifoQueue { - queue: VecDeque, - max_length: usize, -} - -impl LifoQueue { - /// Create a new, empty queue with the given length. - pub fn new(max_length: usize) -> Self { - Self { - queue: VecDeque::default(), - max_length, - } - } - - /// Add a new item to the front of the queue. - /// - /// If the queue is full, the item at the back of the queue is dropped. - pub fn push(&mut self, item: T) { - if self.queue.len() == self.max_length { - self.queue.pop_back(); - } - self.queue.push_front(item); - } - - /// Remove the next item from the queue. - pub fn pop(&mut self) -> Option { - self.queue.pop_front() - } - - /// Returns `true` if the queue is full. - pub fn is_full(&self) -> bool { - self.queue.len() >= self.max_length - } - - /// Returns the current length of the queue. - pub fn len(&self) -> usize { - self.queue.len() - } +#[derive(Clone, PartialEq, Debug, Serialize, Deserialize)] +pub enum BeaconProcessorType { + Priority, + EarliestDeadline, } /// A handle that sends a message on the provided channel to a receiver when it gets dropped. @@ -420,6 +203,121 @@ impl DuplicateCache { } } +#[derive(IntoStaticStr, PartialEq, Eq, Debug)] +#[strum(serialize_all = "snake_case")] +pub enum WorkType { + GossipAttestation, + GossipAttestationToConvert, + UnknownBlockAttestation, + GossipAttestationBatch, + GossipAggregate, + UnknownBlockAggregate, + UnknownLightClientOptimisticUpdate, + UnknownBlockSamplingRequest, + GossipAggregateBatch, + GossipBlock, + GossipBlobSidecar, + GossipDataColumnSidecar, + DelayedImportBlock, + GossipVoluntaryExit, + GossipProposerSlashing, + GossipAttesterSlashing, + GossipSyncSignature, + GossipSyncContribution, + GossipLightClientFinalityUpdate, + GossipLightClientOptimisticUpdate, + RpcBlock, + RpcBlobs, + RpcCustodyColumn, + RpcVerifyDataColumn, + SamplingResult, + IgnoredRpcBlock, + ChainSegment, + ChainSegmentBackfill, + Status, + BlocksByRangeRequest, + BlocksByRootsRequest, + BlobsByRangeRequest, + BlobsByRootsRequest, + DataColumnsByRootsRequest, + DataColumnsByRangeRequest, + GossipBlsToExecutionChange, + LightClientBootstrapRequest, + LightClientOptimisticUpdateRequest, + LightClientFinalityUpdateRequest, + LightClientUpdatesByRangeRequest, + ApiRequestP0, + ApiRequestP1, + Reprocess, + GossipCanonicalBlock, + RpcCanonicalBlock, +} + +impl Work { + pub fn str_id(&self) -> &'static str { + self.to_type().into() + } + + /// Provides a `&str` that uniquely identifies each enum variant. + pub fn to_type(&self) -> WorkType { + match self { + Work::GossipAttestation { .. } => WorkType::GossipAttestation, + Work::GossipAttestationToConvert { .. } => WorkType::GossipAttestationToConvert, + Work::GossipAttestationBatch { .. } => WorkType::GossipAttestationBatch, + Work::GossipAggregate { .. } => WorkType::GossipAggregate, + Work::GossipAggregateBatch { .. } => WorkType::GossipAggregateBatch, + Work::GossipBlock(_) => WorkType::GossipBlock, + Work::GossipBlobSidecar(_) => WorkType::GossipBlobSidecar, + Work::GossipDataColumnSidecar(_) => WorkType::GossipDataColumnSidecar, + Work::DelayedImportBlock { .. } => WorkType::DelayedImportBlock, + Work::GossipVoluntaryExit(_) => WorkType::GossipVoluntaryExit, + Work::GossipProposerSlashing(_) => WorkType::GossipProposerSlashing, + Work::GossipAttesterSlashing(_) => WorkType::GossipAttesterSlashing, + Work::GossipSyncSignature(_) => WorkType::GossipSyncSignature, + Work::GossipSyncContribution(_) => WorkType::GossipSyncContribution, + Work::GossipLightClientFinalityUpdate(_) => WorkType::GossipLightClientFinalityUpdate, + Work::GossipLightClientOptimisticUpdate(_) => { + WorkType::GossipLightClientOptimisticUpdate + } + Work::GossipBlsToExecutionChange(_) => WorkType::GossipBlsToExecutionChange, + Work::RpcBlock { .. } => WorkType::RpcBlock, + Work::RpcBlobs { .. } => WorkType::RpcBlobs, + Work::RpcCustodyColumn { .. } => WorkType::RpcCustodyColumn, + Work::RpcVerifyDataColumn { .. } => WorkType::RpcVerifyDataColumn, + Work::SamplingResult { .. } => WorkType::SamplingResult, + Work::IgnoredRpcBlock { .. } => WorkType::IgnoredRpcBlock, + Work::ChainSegment { .. } => WorkType::ChainSegment, + Work::ChainSegmentBackfill(_) => WorkType::ChainSegmentBackfill, + Work::Status(_) => WorkType::Status, + Work::BlocksByRangeRequest(_) => WorkType::BlocksByRangeRequest, + Work::BlocksByRootsRequest(_) => WorkType::BlocksByRootsRequest, + Work::BlobsByRangeRequest(_) => WorkType::BlobsByRangeRequest, + Work::BlobsByRootsRequest(_) => WorkType::BlobsByRootsRequest, + Work::DataColumnsByRootsRequest(_) => WorkType::DataColumnsByRootsRequest, + Work::DataColumnsByRangeRequest(_) => WorkType::DataColumnsByRangeRequest, + Work::LightClientBootstrapRequest(_) => WorkType::LightClientBootstrapRequest, + Work::LightClientOptimisticUpdateRequest(_) => { + WorkType::LightClientOptimisticUpdateRequest + } + Work::LightClientFinalityUpdateRequest(_) => WorkType::LightClientFinalityUpdateRequest, + Work::UnknownBlockAttestation { .. } => WorkType::UnknownBlockAttestation, + Work::UnknownBlockAggregate { .. } => WorkType::UnknownBlockAggregate, + Work::UnknownBlockSamplingRequest { .. } => WorkType::UnknownBlockSamplingRequest, + Work::UnknownLightClientOptimisticUpdate { .. } => { + WorkType::UnknownLightClientOptimisticUpdate + } + Work::LightClientUpdatesByRangeRequest { .. } => { + WorkType::LightClientUpdatesByRangeRequest + } + Work::ApiRequestP0 { .. } => WorkType::ApiRequestP0, + Work::ApiRequestP1 { .. } => WorkType::ApiRequestP1, + Work::Reprocess { .. } => WorkType::Reprocess, + Work::RpcCanonicalBlock { .. } => WorkType::RpcCanonicalBlock, + Work::GossipCanonicalBlock(_) => WorkType::GossipCanonicalBlock, + } + } +} + /// An event to be processed by the manager task. #[derive(Debug)] pub struct WorkEvent { @@ -439,69 +337,6 @@ impl WorkEvent { } } -impl From for WorkEvent { - fn from(ready_work: ReadyWork) -> Self { - match ready_work { - ReadyWork::Block(QueuedGossipBlock { - beacon_block_slot, - beacon_block_root, - process_fn, - }) => Self { - drop_during_sync: false, - work: Work::DelayedImportBlock { - beacon_block_slot, - beacon_block_root, - process_fn, - }, - }, - ReadyWork::RpcBlock(QueuedRpcBlock { - beacon_block_root: _, - process_fn, - ignore_fn: _, - }) => Self { - drop_during_sync: false, - work: Work::RpcBlock { process_fn }, - }, - ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock { process_fn }) => Self { - drop_during_sync: false, - work: Work::IgnoredRpcBlock { process_fn }, - }, - ReadyWork::Unaggregate(QueuedUnaggregate { - beacon_block_root: _, - process_fn, - }) => Self { - drop_during_sync: true, - work: Work::UnknownBlockAttestation { process_fn }, - }, - ReadyWork::Aggregate(QueuedAggregate { - process_fn, - beacon_block_root: _, - }) => Self { - drop_during_sync: true, - work: Work::UnknownBlockAggregate { process_fn }, - }, - ReadyWork::LightClientUpdate(QueuedLightClientUpdate { - parent_root, - process_fn, - }) => Self { - drop_during_sync: true, - work: Work::UnknownLightClientOptimisticUpdate { - parent_root, - process_fn, - }, - }, - ReadyWork::SamplingRequest(QueuedSamplingRequest { process_fn, .. }) => Self { - drop_during_sync: true, - work: Work::UnknownBlockSamplingRequest { process_fn }, - }, - ReadyWork::BackfillSync(QueuedBackfillBatch(process_fn)) => Self { - drop_during_sync: false, - work: Work::ChainSegmentBackfill(process_fn), - }, - } - } -} - /// Items required to verify a batch of unaggregated gossip attestations. #[derive(Debug)] pub struct GossipAttestationPackage { @@ -532,6 +367,7 @@ impl BeaconProcessorSend { match self.0.try_send(message) { Ok(res) => Ok(res), Err(e) => { + println!("{e}"); metrics::inc_counter_vec( &metrics::BEACON_PROCESSOR_SEND_ERROR_PER_WORK_TYPE, &[work_type.into()], @@ -551,6 +387,111 @@ pub enum BlockingOrAsync { } pub type GossipAttestationBatch = Vec>>; +/// Messages that the scheduler can receive. +#[derive(AsRefStr)] +pub enum ReprocessQueueMessage { + /// A block that has been received early and we should queue for later processing. + 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), + /// A block that was successfully processed. We use this to handle attestations updates + /// for unknown blocks. + BlockImported { + block_root: Hash256, + parent_root: Hash256, + }, + /// A new `LightClientOptimisticUpdate` has been produced. We use this to handle light client + /// updates for unknown parent blocks. + NewLightClientOptimisticUpdate { parent_root: Hash256 }, + /// An unaggregated attestation that references an unknown block. + UnknownBlockUnaggregate(QueuedUnaggregate), + /// An aggregated attestation that references an unknown block. + UnknownBlockAggregate(QueuedAggregate), + /// A light client optimistic update that references a parent root that has not been seen as a parent. + UnknownLightClientOptimisticUpdate(QueuedLightClientUpdate), + /// A sampling request that references an unknown block. + UnknownBlockSamplingRequest(QueuedSamplingRequest), + /// A new backfill batch that needs to be scheduled for processing. + BackfillSync(QueuedBackfillBatch), +} + +/// An Attestation for which the corresponding block was not seen while processing, queued for +/// later. +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 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 parent_root: Hash256, + pub process_fn: BlockingFn, +} + +/// A sampling request for which the corresponding block is not known while processing. +pub struct QueuedSamplingRequest { + pub beacon_block_root: Hash256, + pub process_fn: BlockingFn, +} + +/// A block that arrived early and has been queued for later import. +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 beacon_block_root: Hash256, + /// Processes/imports the block. + pub process_fn: AsyncFn, + /// Ignores the block. + pub ignore_fn: BlockingFn, +} + +/// 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, +} + +/// 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::ChainSegmentBackfill(process_fn), + .. + } => Ok(QueuedBackfillBatch(process_fn)), + _ => Err(event), + } + } +} + +impl From for WorkEvent { + fn from(queued_backfill_batch: QueuedBackfillBatch) -> WorkEvent { + WorkEvent { + drop_during_sync: false, + work: Work::ChainSegmentBackfill(queued_backfill_batch.0), + } + } +} + /// Indicates the type of work to be performed and therefore its priority and /// queuing specifics. pub enum Work { @@ -638,6 +579,11 @@ pub enum Work { LightClientUpdatesByRangeRequest(BlockingFn), ApiRequestP0(BlockingOrAsync), ApiRequestP1(BlockingOrAsync), + Reprocess(ReprocessQueueMessage), + GossipCanonicalBlock(AsyncFn), + RpcCanonicalBlock { + process_fn: AsyncFn, + }, } impl fmt::Debug for Work { @@ -646,175 +592,12 @@ impl fmt::Debug for Work { } } -#[derive(IntoStaticStr, PartialEq, Eq, Debug)] -#[strum(serialize_all = "snake_case")] -pub enum WorkType { - GossipAttestation, - GossipAttestationToConvert, - UnknownBlockAttestation, - GossipAttestationBatch, - GossipAggregate, - UnknownBlockAggregate, - UnknownLightClientOptimisticUpdate, - UnknownBlockSamplingRequest, - GossipAggregateBatch, - GossipBlock, - GossipBlobSidecar, - GossipDataColumnSidecar, - DelayedImportBlock, - GossipVoluntaryExit, - GossipProposerSlashing, - GossipAttesterSlashing, - GossipSyncSignature, - GossipSyncContribution, - GossipLightClientFinalityUpdate, - GossipLightClientOptimisticUpdate, - RpcBlock, - RpcBlobs, - RpcCustodyColumn, - RpcVerifyDataColumn, - SamplingResult, - IgnoredRpcBlock, - ChainSegment, - ChainSegmentBackfill, - Status, - BlocksByRangeRequest, - BlocksByRootsRequest, - BlobsByRangeRequest, - BlobsByRootsRequest, - DataColumnsByRootsRequest, - DataColumnsByRangeRequest, - GossipBlsToExecutionChange, - LightClientBootstrapRequest, - LightClientOptimisticUpdateRequest, - LightClientFinalityUpdateRequest, - LightClientUpdatesByRangeRequest, - ApiRequestP0, - ApiRequestP1, -} - impl Work { - fn str_id(&self) -> &'static str { - self.to_type().into() - } - - /// Provides a `&str` that uniquely identifies each enum variant. - fn to_type(&self) -> WorkType { - match self { - Work::GossipAttestation { .. } => WorkType::GossipAttestation, - Work::GossipAttestationToConvert { .. } => WorkType::GossipAttestationToConvert, - Work::GossipAttestationBatch { .. } => WorkType::GossipAttestationBatch, - Work::GossipAggregate { .. } => WorkType::GossipAggregate, - Work::GossipAggregateBatch { .. } => WorkType::GossipAggregateBatch, - Work::GossipBlock(_) => WorkType::GossipBlock, - Work::GossipBlobSidecar(_) => WorkType::GossipBlobSidecar, - Work::GossipDataColumnSidecar(_) => WorkType::GossipDataColumnSidecar, - Work::DelayedImportBlock { .. } => WorkType::DelayedImportBlock, - Work::GossipVoluntaryExit(_) => WorkType::GossipVoluntaryExit, - Work::GossipProposerSlashing(_) => WorkType::GossipProposerSlashing, - Work::GossipAttesterSlashing(_) => WorkType::GossipAttesterSlashing, - Work::GossipSyncSignature(_) => WorkType::GossipSyncSignature, - Work::GossipSyncContribution(_) => WorkType::GossipSyncContribution, - Work::GossipLightClientFinalityUpdate(_) => WorkType::GossipLightClientFinalityUpdate, - Work::GossipLightClientOptimisticUpdate(_) => { - WorkType::GossipLightClientOptimisticUpdate - } - Work::GossipBlsToExecutionChange(_) => WorkType::GossipBlsToExecutionChange, - Work::RpcBlock { .. } => WorkType::RpcBlock, - Work::RpcBlobs { .. } => WorkType::RpcBlobs, - Work::RpcCustodyColumn { .. } => WorkType::RpcCustodyColumn, - Work::RpcVerifyDataColumn { .. } => WorkType::RpcVerifyDataColumn, - Work::SamplingResult { .. } => WorkType::SamplingResult, - Work::IgnoredRpcBlock { .. } => WorkType::IgnoredRpcBlock, - Work::ChainSegment { .. } => WorkType::ChainSegment, - Work::ChainSegmentBackfill(_) => WorkType::ChainSegmentBackfill, - Work::Status(_) => WorkType::Status, - Work::BlocksByRangeRequest(_) => WorkType::BlocksByRangeRequest, - Work::BlocksByRootsRequest(_) => WorkType::BlocksByRootsRequest, - Work::BlobsByRangeRequest(_) => WorkType::BlobsByRangeRequest, - Work::BlobsByRootsRequest(_) => WorkType::BlobsByRootsRequest, - Work::DataColumnsByRootsRequest(_) => WorkType::DataColumnsByRootsRequest, - Work::DataColumnsByRangeRequest(_) => WorkType::DataColumnsByRangeRequest, - Work::LightClientBootstrapRequest(_) => WorkType::LightClientBootstrapRequest, - Work::LightClientOptimisticUpdateRequest(_) => { - WorkType::LightClientOptimisticUpdateRequest - } - Work::LightClientFinalityUpdateRequest(_) => WorkType::LightClientFinalityUpdateRequest, - Work::LightClientUpdatesByRangeRequest(_) => WorkType::LightClientUpdatesByRangeRequest, - Work::UnknownBlockAttestation { .. } => WorkType::UnknownBlockAttestation, - Work::UnknownBlockAggregate { .. } => WorkType::UnknownBlockAggregate, - Work::UnknownBlockSamplingRequest { .. } => WorkType::UnknownBlockSamplingRequest, - Work::UnknownLightClientOptimisticUpdate { .. } => { - WorkType::UnknownLightClientOptimisticUpdate - } - Work::ApiRequestP0 { .. } => WorkType::ApiRequestP0, - Work::ApiRequestP1 { .. } => WorkType::ApiRequestP1, - } - } -} - -/// Unifies all the messages processed by the `BeaconProcessor`. -enum InboundEvent { - /// A worker has completed a task and is free. - WorkerIdle, - /// There is new work to be done. - WorkEvent(WorkEvent), - /// A work event that was queued for re-processing has become ready. - 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 { - /// 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>, - /// Used internally for queuing work ready to be re-processed. - reprocess_work_rx: mpsc::Receiver, -} - -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 - // stream of new events doesn't suppress the processing of existing events. - match self.idle_rx.poll_recv(cx) { - Poll::Ready(Some(())) => { - return Poll::Ready(Some(InboundEvent::WorkerIdle)); - } - Poll::Ready(None) => { - return Poll::Ready(None); - } - Poll::Pending => {} - } - - // Poll for delayed blocks before polling for new work. It might be the case that a delayed - // block is required to successfully process some new work. - match self.reprocess_work_rx.poll_recv(cx) { - Poll::Ready(Some(ready_work)) => { - return Poll::Ready(Some(InboundEvent::ReprocessingWork(ready_work.into()))); - } - Poll::Ready(None) => { - return Poll::Ready(None); - } - Poll::Pending => {} - } - - match self.event_rx.poll_recv(cx) { - Poll::Ready(Some(event)) => { - return Poll::Ready(Some(InboundEvent::WorkEvent(event))); - } - Poll::Ready(None) => { - return Poll::Ready(None); - } - Poll::Pending => {} - } - - Poll::Pending + pub fn is_priority_work(&self) -> bool { + matches!( + self, + Work::GossipCanonicalBlock(..) | Work::RpcCanonicalBlock { .. } + ) } } @@ -843,802 +626,20 @@ impl BeaconProcessor { /// events processed by `self`. This should only be used during testing. #[allow(clippy::too_many_arguments)] pub fn spawn_manager( - mut self, + self, + beacon_state: &BeaconState, event_rx: mpsc::Receiver>, - work_reprocessing_tx: mpsc::Sender, - work_reprocessing_rx: mpsc::Receiver, work_journal_tx: Option>, slot_clock: S, - maximum_gossip_clock_disparity: Duration, - queue_lengths: BeaconProcessorQueueLengths, + spec: &ChainSpec, ) -> Result<(), String> { - // Used by workers to communicate that they are finished a task. - let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); - - // Using LIFO queues for attestations since validator profits rely upon getting fresh - // attestations into blocks. Additionally, later attestations contain more information than - // earlier ones, so we consider them more valuable. - let mut aggregate_queue = LifoQueue::new(queue_lengths.aggregate_queue); - let mut aggregate_debounce = TimeLatch::default(); - let mut attestation_queue = LifoQueue::new(queue_lengths.attestation_queue); - let mut attestation_to_convert_queue = LifoQueue::new(queue_lengths.attestation_queue); - let mut attestation_debounce = TimeLatch::default(); - let mut unknown_block_aggregate_queue = - LifoQueue::new(queue_lengths.unknown_block_aggregate_queue); - let mut unknown_block_attestation_queue = - LifoQueue::new(queue_lengths.unknown_block_attestation_queue); - - let mut sync_message_queue = LifoQueue::new(queue_lengths.sync_message_queue); - let mut sync_contribution_queue = LifoQueue::new(queue_lengths.sync_contribution_queue); - - // Using a FIFO queue for voluntary exits since it prevents exit censoring. I don't have - // a strong feeling about queue type for exits. - let mut gossip_voluntary_exit_queue = - FifoQueue::new(queue_lengths.gossip_voluntary_exit_queue); - - // Using a FIFO queue for slashing to prevent people from flushing their slashings from the - // queues with lots of junk messages. - let mut gossip_proposer_slashing_queue = - FifoQueue::new(queue_lengths.gossip_proposer_slashing_queue); - let mut gossip_attester_slashing_queue = - FifoQueue::new(queue_lengths.gossip_attester_slashing_queue); - - // Using a FIFO queue since blocks need to be imported sequentially. - let mut rpc_block_queue = FifoQueue::new(queue_lengths.rpc_block_queue); - let mut rpc_blob_queue = FifoQueue::new(queue_lengths.rpc_blob_queue); - let mut rpc_custody_column_queue = FifoQueue::new(queue_lengths.rpc_custody_column_queue); - let mut rpc_verify_data_column_queue = - FifoQueue::new(queue_lengths.rpc_verify_data_column_queue); - // TODO(das): the sampling_request_queue is never read - let mut sampling_result_queue = FifoQueue::new(queue_lengths.sampling_result_queue); - let mut unknown_block_sampling_request_queue = - FifoQueue::new(queue_lengths.unknown_block_sampling_request_queue); - let mut chain_segment_queue = FifoQueue::new(queue_lengths.chain_segment_queue); - let mut backfill_chain_segment = FifoQueue::new(queue_lengths.backfill_chain_segment); - let mut gossip_block_queue = FifoQueue::new(queue_lengths.gossip_block_queue); - let mut gossip_blob_queue = FifoQueue::new(queue_lengths.gossip_blob_queue); - let mut gossip_data_column_queue = FifoQueue::new(queue_lengths.gossip_data_column_queue); - let mut delayed_block_queue = FifoQueue::new(queue_lengths.delayed_block_queue); - - let mut status_queue = FifoQueue::new(queue_lengths.status_queue); - let mut bbrange_queue = FifoQueue::new(queue_lengths.bbrange_queue); - let mut bbroots_queue = FifoQueue::new(queue_lengths.bbroots_queue); - let mut blbroots_queue = FifoQueue::new(queue_lengths.blbroots_queue); - let mut blbrange_queue = FifoQueue::new(queue_lengths.blbrange_queue); - let mut dcbroots_queue = FifoQueue::new(queue_lengths.dcbroots_queue); - let mut dcbrange_queue = FifoQueue::new(queue_lengths.dcbrange_queue); - - let mut gossip_bls_to_execution_change_queue = - FifoQueue::new(queue_lengths.gossip_bls_to_execution_change_queue); - - // Using FIFO queues for light client updates to maintain sequence order. - let mut lc_gossip_finality_update_queue = - FifoQueue::new(queue_lengths.lc_gossip_finality_update_queue); - let mut lc_gossip_optimistic_update_queue = - FifoQueue::new(queue_lengths.lc_gossip_optimistic_update_queue); - let mut unknown_light_client_update_queue = - FifoQueue::new(queue_lengths.unknown_light_client_update_queue); - let mut lc_bootstrap_queue = FifoQueue::new(queue_lengths.lc_bootstrap_queue); - let mut lc_rpc_optimistic_update_queue = - FifoQueue::new(queue_lengths.lc_rpc_optimistic_update_queue); - let mut lc_rpc_finality_update_queue = - FifoQueue::new(queue_lengths.lc_rpc_finality_update_queue); - let mut lc_update_range_queue = FifoQueue::new(queue_lengths.lc_update_range_queue); - - let mut api_request_p0_queue = FifoQueue::new(queue_lengths.api_request_p0_queue); - let mut api_request_p1_queue = FifoQueue::new(queue_lengths.api_request_p1_queue); - - // 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::(self.config.max_scheduled_work_queue_len); - spawn_reprocess_scheduler( - ready_work_tx, - work_reprocessing_rx, - &self.executor, - Arc::new(slot_clock), - maximum_gossip_clock_disparity, - )?; - - let executor = self.executor.clone(); - - // The manager future will run on the core executor and delegate tasks to worker - // threads on the blocking executor. - let manager_future = async move { - let mut inbound_events = InboundEvents { - idle_rx, - event_rx, - reprocess_work_rx: ready_work_rx, - }; - - let enable_backfill_rate_limiting = self.config.enable_backfill_rate_limiting; - - loop { - let work_event = match inbound_events.next().await { - Some(InboundEvent::WorkerIdle) => { - self.current_workers = self.current_workers.saturating_sub(1); - None - } - Some(InboundEvent::WorkEvent(event)) if enable_backfill_rate_limiting => { - match QueuedBackfillBatch::try_from(event) { - Ok(backfill_batch) => { - match work_reprocessing_tx - .try_send(ReprocessQueueMessage::BackfillSync(backfill_batch)) - { - Err(e) => { - warn!( - error = %e, - "Unable to queue backfill work event. Will try to process now." - ); - match e { - TrySendError::Full(reprocess_queue_message) - | TrySendError::Closed(reprocess_queue_message) => { - match reprocess_queue_message { - ReprocessQueueMessage::BackfillSync( - backfill_batch, - ) => Some(backfill_batch.into()), - other => { - crit!( - message_type = other.as_ref(), - "Unexpected queue message type" - ); - // This is an unhandled exception, drop the message. - continue; - } - } - } - } - } - Ok(..) => { - // backfill work sent to "reprocessing" queue. Process the next event. - continue; - } - } - } - Err(event) => Some(event), - } - } - Some(InboundEvent::WorkEvent(event)) - | Some(InboundEvent::ReprocessingWork(event)) => Some(event), - None => { - debug!(msg = "stream ended", "Gossip processor stopped"); - break; - } - }; - - let _event_timer = - metrics::start_timer(&metrics::BEACON_PROCESSOR_EVENT_HANDLING_SECONDS); - if let Some(event) = &work_event { - metrics::inc_counter_vec( - &metrics::BEACON_PROCESSOR_WORK_EVENTS_RX_COUNT, - &[event.work.str_id()], - ); - } else { - metrics::inc_counter(&metrics::BEACON_PROCESSOR_IDLE_EVENTS_TOTAL); - } - - if let Some(work_journal_tx) = &work_journal_tx { - let id = work_event - .as_ref() - .map(|event| event.work.str_id()) - .unwrap_or(WORKER_FREED); - - // We don't care if this message was successfully sent, we only use the journal - // during testing. - let _ = work_journal_tx.try_send(id); - } - - let can_spawn = self.current_workers < self.config.max_workers; - let drop_during_sync = work_event - .as_ref() - .is_some_and(|event| event.drop_during_sync); - - let idle_tx = idle_tx.clone(); - let modified_queue_id = match work_event { - // There is no new work event, but we are able to spawn a new worker. - // - // We don't check the `work.drop_during_sync` here. We assume that if it made - // it into the queue at any point then we should process it. - None if can_spawn => { - // Check for chain segments first, they're the most efficient way to get - // blocks into the system. - let work_event: Option> = - if let Some(item) = chain_segment_queue.pop() { - Some(item) - // Check sync blocks before gossip blocks, since we've already explicitly - // requested these blocks. - } else if let Some(item) = rpc_block_queue.pop() { - Some(item) - } else if let Some(item) = rpc_blob_queue.pop() { - Some(item) - } else if let Some(item) = rpc_custody_column_queue.pop() { - Some(item) - // TODO(das): decide proper prioritization for sampling columns - } else if let Some(item) = rpc_custody_column_queue.pop() { - Some(item) - } else if let Some(item) = rpc_verify_data_column_queue.pop() { - Some(item) - } else if let Some(item) = sampling_result_queue.pop() { - Some(item) - // Check delayed blocks before gossip blocks, the gossip blocks might rely - // on the delayed ones. - } else if let Some(item) = delayed_block_queue.pop() { - Some(item) - // Check gossip blocks before gossip attestations, since a block might be - // required to verify some attestations. - } else if let Some(item) = gossip_block_queue.pop() { - Some(item) - } else if let Some(item) = gossip_blob_queue.pop() { - Some(item) - } else if let Some(item) = gossip_data_column_queue.pop() { - Some(item) - // Check the priority 0 API requests after blocks and blobs, but before attestations. - } else if let Some(item) = api_request_p0_queue.pop() { - Some(item) - // Check the aggregates, *then* the unaggregates since we assume that - // aggregates are more valuable to local validators and effectively give us - // more information with less signature verification time. - } else if aggregate_queue.len() > 0 { - let batch_size = cmp::min( - aggregate_queue.len(), - self.config.max_gossip_aggregate_batch_size, - ); - - if batch_size < 2 { - // One single aggregate is in the queue, process it individually. - aggregate_queue.pop() - } else { - // Collect two or more aggregates into a batch, so they can take - // advantage of batch signature verification. - // - // Note: this will convert the `Work::GossipAggregate` item into a - // `Work::GossipAggregateBatch` item. - 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 { - aggregate, - process_individual: _, - process_batch, - } => { - aggregates.push(*aggregate); - if process_batch_opt.is_none() { - process_batch_opt = Some(process_batch); - } - } - _ => { - error!("Invalid item in aggregate queue"); - } - } - } - } - - if let Some(process_batch) = process_batch_opt { - // Process all aggregates with a single worker. - Some(Work::GossipAggregateBatch { - aggregates, - process_batch, - }) - } 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!("Missing aggregate work"); - None - } - } - // Check the unaggregated attestation queue. - // - // Potentially use batching. - } else if attestation_queue.len() > 0 { - let batch_size = cmp::min( - attestation_queue.len(), - self.config.max_gossip_attestation_batch_size, - ); - - if batch_size < 2 { - // One single attestation is in the queue, process it individually. - attestation_queue.pop() - } else { - // Collect two or more attestations into a batch, so they can take - // advantage of batch signature verification. - // - // Note: this will convert the `Work::GossipAttestation` item into a - // `Work::GossipAttestationBatch` item. - 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 { - attestation, - process_individual: _, - process_batch, - } => { - attestations.push(*attestation); - if process_batch_opt.is_none() { - process_batch_opt = Some(process_batch); - } - } - _ => error!("Invalid item in attestation queue"), - } - } - } - - if let Some(process_batch) = process_batch_opt { - // Process all attestations with a single worker. - Some(Work::GossipAttestationBatch { - attestations, - process_batch, - }) - } 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!("Missing attestations work"); - None - } - } - // Convert any gossip attestations that need to be converted. - } else if let Some(item) = attestation_to_convert_queue.pop() { - Some(item) - // 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() { - Some(item) - } else if let Some(item) = sync_message_queue.pop() { - Some(item) - // 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() { - Some(item) - } else if let Some(item) = unknown_block_attestation_queue.pop() { - Some(item) - // 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() { - Some(item) - } else if let Some(item) = bbrange_queue.pop() { - Some(item) - } else if let Some(item) = bbroots_queue.pop() { - Some(item) - } else if let Some(item) = blbrange_queue.pop() { - Some(item) - } else if let Some(item) = blbroots_queue.pop() { - Some(item) - } else if let Some(item) = dcbroots_queue.pop() { - Some(item) - } else if let Some(item) = dcbrange_queue.pop() { - Some(item) - // Prioritize sampling requests after block syncing requests - } else if let Some(item) = unknown_block_sampling_request_queue.pop() { - Some(item) - // 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() { - Some(item) - } else if let Some(item) = gossip_proposer_slashing_queue.pop() { - Some(item) - // 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() { - Some(item) - } else if let Some(item) = gossip_bls_to_execution_change_queue.pop() { - Some(item) - // Check the priority 1 API requests after we've - // processed all the interesting things from the network - // and things required for us to stay in good repute - // with our P2P peers. - } else if let Some(item) = api_request_p1_queue.pop() { - Some(item) - // Handle backfill sync chain segments. - } else if let Some(item) = backfill_chain_segment.pop() { - Some(item) - // Handle light client requests. - } else if let Some(item) = lc_gossip_finality_update_queue.pop() { - Some(item) - } else if let Some(item) = lc_gossip_optimistic_update_queue.pop() { - Some(item) - } else if let Some(item) = unknown_light_client_update_queue.pop() { - Some(item) - } else if let Some(item) = lc_bootstrap_queue.pop() { - Some(item) - } else if let Some(item) = lc_rpc_optimistic_update_queue.pop() { - Some(item) - } else if let Some(item) = lc_rpc_finality_update_queue.pop() { - Some(item) - } else if let Some(item) = lc_update_range_queue.pop() { - Some(item) - // This statement should always be the final else statement. - } else { - // Let the journal know that a worker is freed and there's nothing else - // for it to do. - if let Some(work_journal_tx) = &work_journal_tx { - // We don't care if this message was successfully sent, we only use the journal - // during testing. - let _ = work_journal_tx.try_send(NOTHING_TO_DO); - } - None - }; - - if let Some(work_event) = work_event { - let work_type = work_event.to_type(); - self.spawn_worker(work_event, idle_tx); - Some(work_type) - } else { - None - } - } - // There is no new work event and we are unable to spawn a new worker. - // - // I cannot see any good reason why this would happen. - None => { - warn!( - msg = "no new work and cannot spawn worker", - "Unexpected gossip processor condition" - ); - None - } - // The chain is syncing and this event should be dropped during sync. - Some(work_event) - if self.network_globals.sync_state.read().is_syncing() - && drop_during_sync => - { - let work_id = work_event.work.str_id(); - metrics::inc_counter_vec( - &metrics::BEACON_PROCESSOR_WORK_EVENTS_IGNORED_COUNT, - &[work_id], - ); - trace!( - msg = "chain is syncing", - work_id = work_id, - "Gossip processor skipping work" - ); - None - } - // There is a new work event and the chain is not syncing. Process it or queue - // it. - Some(WorkEvent { work, .. }) => { - let work_id = work.str_id(); - let work_type = work.to_type(); - - match work { - _ if can_spawn => self.spawn_worker(work, idle_tx), - Work::GossipAttestation { .. } => attestation_queue.push(work), - Work::GossipAttestationToConvert { .. } => { - attestation_to_convert_queue.push(work) - } - // Attestation batches are formed internally within the - // `BeaconProcessor`, they are not sent from external services. - Work::GossipAttestationBatch { .. } => crit!( - work_type = "GossipAttestationBatch", - "Unsupported inbound event" - ), - Work::GossipAggregate { .. } => aggregate_queue.push(work), - // Aggregate batches are formed internally within the `BeaconProcessor`, - // they are not sent from external services. - Work::GossipAggregateBatch { .. } => { - crit!( - work_type = "GossipAggregateBatch", - "Unsupported inbound event" - ) - } - Work::GossipBlock { .. } => gossip_block_queue.push(work, work_id), - Work::GossipBlobSidecar { .. } => gossip_blob_queue.push(work, work_id), - Work::GossipDataColumnSidecar { .. } => { - gossip_data_column_queue.push(work, work_id) - } - Work::DelayedImportBlock { .. } => { - delayed_block_queue.push(work, work_id) - } - Work::GossipVoluntaryExit { .. } => { - gossip_voluntary_exit_queue.push(work, work_id) - } - Work::GossipProposerSlashing { .. } => { - gossip_proposer_slashing_queue.push(work, work_id) - } - Work::GossipAttesterSlashing { .. } => { - gossip_attester_slashing_queue.push(work, work_id) - } - Work::GossipSyncSignature { .. } => sync_message_queue.push(work), - Work::GossipSyncContribution { .. } => { - sync_contribution_queue.push(work) - } - Work::GossipLightClientFinalityUpdate { .. } => { - lc_gossip_finality_update_queue.push(work, work_id) - } - Work::GossipLightClientOptimisticUpdate { .. } => { - lc_gossip_optimistic_update_queue.push(work, work_id) - } - Work::RpcBlock { .. } | Work::IgnoredRpcBlock { .. } => { - rpc_block_queue.push(work, work_id) - } - Work::RpcBlobs { .. } => rpc_blob_queue.push(work, work_id), - Work::RpcCustodyColumn { .. } => { - rpc_custody_column_queue.push(work, work_id) - } - Work::RpcVerifyDataColumn(_) => { - rpc_verify_data_column_queue.push(work, work_id) - } - Work::SamplingResult(_) => sampling_result_queue.push(work, work_id), - Work::ChainSegment { .. } => chain_segment_queue.push(work, work_id), - Work::ChainSegmentBackfill { .. } => { - backfill_chain_segment.push(work, work_id) - } - Work::Status { .. } => status_queue.push(work, work_id), - Work::BlocksByRangeRequest { .. } => bbrange_queue.push(work, work_id), - Work::BlocksByRootsRequest { .. } => bbroots_queue.push(work, work_id), - Work::BlobsByRangeRequest { .. } => blbrange_queue.push(work, work_id), - Work::LightClientBootstrapRequest { .. } => { - lc_bootstrap_queue.push(work, work_id) - } - Work::LightClientOptimisticUpdateRequest { .. } => { - lc_rpc_optimistic_update_queue.push(work, work_id) - } - Work::LightClientFinalityUpdateRequest { .. } => { - lc_rpc_finality_update_queue.push(work, work_id) - } - Work::LightClientUpdatesByRangeRequest { .. } => { - lc_update_range_queue.push(work, work_id) - } - Work::UnknownBlockAttestation { .. } => { - unknown_block_attestation_queue.push(work) - } - Work::UnknownBlockAggregate { .. } => { - unknown_block_aggregate_queue.push(work) - } - Work::GossipBlsToExecutionChange { .. } => { - gossip_bls_to_execution_change_queue.push(work, work_id) - } - Work::BlobsByRootsRequest { .. } => blbroots_queue.push(work, work_id), - Work::DataColumnsByRootsRequest { .. } => { - dcbroots_queue.push(work, work_id) - } - Work::DataColumnsByRangeRequest { .. } => { - dcbrange_queue.push(work, work_id) - } - Work::UnknownLightClientOptimisticUpdate { .. } => { - unknown_light_client_update_queue.push(work, work_id) - } - Work::UnknownBlockSamplingRequest { .. } => { - unknown_block_sampling_request_queue.push(work, work_id) - } - Work::ApiRequestP0 { .. } => api_request_p0_queue.push(work, work_id), - Work::ApiRequestP1 { .. } => api_request_p1_queue.push(work, work_id), - }; - Some(work_type) - } - }; - - metrics::set_gauge( - &metrics::BEACON_PROCESSOR_WORKERS_ACTIVE_TOTAL, - self.current_workers as i64, - ); - - if let Some(modified_queue_id) = modified_queue_id { - let queue_len = match modified_queue_id { - WorkType::GossipAttestation => attestation_queue.len(), - WorkType::GossipAttestationToConvert => attestation_to_convert_queue.len(), - WorkType::UnknownBlockAttestation => unknown_block_attestation_queue.len(), - WorkType::GossipAttestationBatch => 0, // No queue - WorkType::GossipAggregate => aggregate_queue.len(), - WorkType::UnknownBlockAggregate => unknown_block_aggregate_queue.len(), - WorkType::UnknownLightClientOptimisticUpdate => { - unknown_light_client_update_queue.len() - } - WorkType::UnknownBlockSamplingRequest => { - unknown_block_sampling_request_queue.len() - } - WorkType::GossipAggregateBatch => 0, // No queue - WorkType::GossipBlock => gossip_block_queue.len(), - WorkType::GossipBlobSidecar => gossip_blob_queue.len(), - WorkType::GossipDataColumnSidecar => gossip_data_column_queue.len(), - WorkType::DelayedImportBlock => delayed_block_queue.len(), - WorkType::GossipVoluntaryExit => gossip_voluntary_exit_queue.len(), - WorkType::GossipProposerSlashing => gossip_proposer_slashing_queue.len(), - WorkType::GossipAttesterSlashing => gossip_attester_slashing_queue.len(), - WorkType::GossipSyncSignature => sync_message_queue.len(), - WorkType::GossipSyncContribution => sync_contribution_queue.len(), - WorkType::GossipLightClientFinalityUpdate => { - lc_gossip_finality_update_queue.len() - } - WorkType::GossipLightClientOptimisticUpdate => { - lc_gossip_optimistic_update_queue.len() - } - WorkType::RpcBlock => rpc_block_queue.len(), - WorkType::RpcBlobs | WorkType::IgnoredRpcBlock => rpc_blob_queue.len(), - WorkType::RpcCustodyColumn => rpc_custody_column_queue.len(), - WorkType::RpcVerifyDataColumn => rpc_verify_data_column_queue.len(), - WorkType::SamplingResult => sampling_result_queue.len(), - WorkType::ChainSegment => chain_segment_queue.len(), - WorkType::ChainSegmentBackfill => backfill_chain_segment.len(), - WorkType::Status => status_queue.len(), - WorkType::BlocksByRangeRequest => blbrange_queue.len(), - WorkType::BlocksByRootsRequest => blbroots_queue.len(), - WorkType::BlobsByRangeRequest => bbrange_queue.len(), - WorkType::BlobsByRootsRequest => bbroots_queue.len(), - WorkType::DataColumnsByRootsRequest => dcbroots_queue.len(), - WorkType::DataColumnsByRangeRequest => dcbrange_queue.len(), - WorkType::GossipBlsToExecutionChange => { - gossip_bls_to_execution_change_queue.len() - } - WorkType::LightClientBootstrapRequest => lc_bootstrap_queue.len(), - WorkType::LightClientOptimisticUpdateRequest => { - lc_rpc_optimistic_update_queue.len() - } - WorkType::LightClientFinalityUpdateRequest => { - lc_rpc_finality_update_queue.len() - } - WorkType::LightClientUpdatesByRangeRequest => lc_update_range_queue.len(), - WorkType::ApiRequestP0 => api_request_p0_queue.len(), - WorkType::ApiRequestP1 => api_request_p1_queue.len(), - }; - metrics::observe_vec( - &metrics::BEACON_PROCESSOR_QUEUE_LENGTH, - &[modified_queue_id.into()], - queue_len as f64, - ); - } - - if aggregate_queue.is_full() && aggregate_debounce.elapsed() { - error!( - msg = "the system has insufficient resources for load", - queue_len = aggregate_queue.max_length, - "Aggregate attestation queue full" - ) - } - - if attestation_queue.is_full() && attestation_debounce.elapsed() { - error!( - msg = "the system has insufficient resources for load", - queue_len = attestation_queue.max_length, - "Attestation queue full" - ) - } - } - }; - - // Spawn on the core executor. - executor.spawn(manager_future, MANAGER_TASK_NAME); - Ok(()) - } - - /// 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, idle_tx: mpsc::Sender<()>) { - let work_id = work.str_id(); - let worker_timer = - metrics::start_timer_vec(&metrics::BEACON_PROCESSOR_WORKER_TIME, &[work_id]); - metrics::inc_counter(&metrics::BEACON_PROCESSOR_WORKERS_SPAWNED_TOTAL); - metrics::inc_counter_vec( - &metrics::BEACON_PROCESSOR_WORK_EVENTS_STARTED_COUNT, - &[work.str_id()], - ); - - // Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped. - // - // This helps ensure that the worker is always freed in the case of an early exit or panic. - // As such, this instantiation should happen as early in the function as possible. - let send_idle_on_drop = SendOnDrop { - tx: idle_tx, - _worker_timer: worker_timer, - }; - - let worker_id = self.current_workers; - self.current_workers = self.current_workers.saturating_add(1); - - let executor = self.executor.clone(); - - trace!( - work = work_id, - worker = worker_id, - "Spawning beacon processor worker" - ); - - let task_spawner = TaskSpawner { - executor, - send_idle_on_drop, - }; - - match work { - Work::GossipAttestation { - attestation, - process_individual, - process_batch: _, - } => task_spawner.spawn_blocking(move || { - process_individual(*attestation); - }), - Work::GossipAttestationToConvert { - attestation, - process_individual, - } => task_spawner.spawn_blocking(move || { - process_individual(*attestation); - }), - Work::GossipAttestationBatch { - attestations, - process_batch, - } => task_spawner.spawn_blocking(move || { - process_batch(attestations); - }), - Work::GossipAggregate { - aggregate, - process_individual, - process_batch: _, - } => task_spawner.spawn_blocking(move || { - process_individual(*aggregate); - }), - Work::GossipAggregateBatch { - aggregates, - process_batch, - } => task_spawner.spawn_blocking(move || { - process_batch(aggregates); - }), - Work::ChainSegment(process_fn) => task_spawner.spawn_async(async move { - process_fn.await; - }), - Work::UnknownBlockAttestation { process_fn } - | Work::UnknownBlockAggregate { process_fn } - | Work::UnknownLightClientOptimisticUpdate { process_fn, .. } - | Work::UnknownBlockSamplingRequest { 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 } - | Work::RpcBlobs { process_fn } - | Work::RpcCustodyColumn(process_fn) - | Work::RpcVerifyDataColumn(process_fn) - | Work::SamplingResult(process_fn) => task_spawner.spawn_async(process_fn), - Work::IgnoredRpcBlock { process_fn } => task_spawner.spawn_blocking(process_fn), - Work::GossipBlock(work) - | Work::GossipBlobSidecar(work) - | Work::GossipDataColumnSidecar(work) => task_spawner.spawn_async(async move { - work.await; - }), - Work::BlobsByRangeRequest(process_fn) - | Work::BlobsByRootsRequest(process_fn) - | Work::DataColumnsByRootsRequest(process_fn) - | Work::DataColumnsByRangeRequest(process_fn) => { - task_spawner.spawn_blocking(process_fn) - } - Work::BlocksByRangeRequest(work) | Work::BlocksByRootsRequest(work) => { - task_spawner.spawn_async(work) - } - Work::ChainSegmentBackfill(process_fn) => task_spawner.spawn_async(process_fn), - Work::ApiRequestP0(process_fn) | Work::ApiRequestP1(process_fn) => match process_fn { - BlockingOrAsync::Blocking(process_fn) => task_spawner.spawn_blocking(process_fn), - BlockingOrAsync::Async(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) - | Work::LightClientOptimisticUpdateRequest(process_fn) - | Work::LightClientFinalityUpdateRequest(process_fn) - | Work::LightClientUpdatesByRangeRequest(process_fn) => { - task_spawner.spawn_blocking(process_fn) - } - }; + let scheduler = SchedulerType::::new(self, beacon_state, spec)?; + scheduler.run( + event_rx, + work_journal_tx, + slot_clock, + spec.maximum_gossip_clock_disparity(), + ) } } @@ -1708,21 +709,3 @@ impl Drop for SendOnDrop { } } } - -#[cfg(test)] -mod tests { - use super::*; - use types::{BeaconState, ChainSpec, Eth1Data, ForkName, MainnetEthSpec}; - - #[test] - fn min_queue_len() { - // State with no validators. - let spec = ForkName::latest_stable().make_genesis_spec(ChainSpec::mainnet()); - let genesis_time = 0; - let state = BeaconState::::new(genesis_time, Eth1Data::default(), &spec); - assert_eq!(state.validators().len(), 0); - let queue_lengths = BeaconProcessorQueueLengths::from_state(&state, &spec).unwrap(); - assert_eq!(queue_lengths.attestation_queue, MIN_QUEUE_LEN); - assert_eq!(queue_lengths.unknown_block_attestation_queue, MIN_QUEUE_LEN); - } -} diff --git a/beacon_node/beacon_processor/src/scheduler/earliest_deadline_scheduler/earliest_deadline_queue.rs b/beacon_node/beacon_processor/src/scheduler/earliest_deadline_scheduler/earliest_deadline_queue.rs new file mode 100644 index 00000000000..0090fec558b --- /dev/null +++ b/beacon_node/beacon_processor/src/scheduler/earliest_deadline_scheduler/earliest_deadline_queue.rs @@ -0,0 +1,239 @@ +use std::{ + cmp::{max, Reverse}, + collections::BinaryHeap, + marker::PhantomData, + time::Duration, +}; + +use slot_clock::SlotClock; +use types::{EthSpec, Slot}; + +use crate::{ReprocessQueueMessage, Work, WorkEvent}; + +pub struct WorkQueue { + min_heap: BinaryHeap>, +} + +pub struct QueueItem { + deadline: Duration, + pub work_event: WorkEvent, + phantom_data: PhantomData, +} + +impl QueueItem { + pub fn new(work_event: WorkEvent, slot_clock: &S) -> Option { + let Some(deadline) = QueueItem::calculate_deadline(&work_event.work, slot_clock) else { + return None; + }; + + Some(Self { + work_event, + deadline, + phantom_data: PhantomData, + }) + } + + fn calculate_deadline(work: &Work, slot_clock: &S) -> Option { + let Some(current_time) = slot_clock.now_duration() else { + return None; + }; + println!("work: {:?}", work); + let deadline = match work { + Work::GossipAttestation { attestation, .. } => { + let attestation_slot = attestation.attestation.data().slot; + Self::calculate_unaggregated_attestation_deadline(attestation_slot, slot_clock) + } + Work::GossipAttestationToConvert { attestation, .. } => { + let attestation_slot = attestation.attestation.data.slot; + Self::calculate_unaggregated_attestation_deadline(attestation_slot, slot_clock) + } + Work::GossipAttestationBatch { attestations, .. } => { + let Some(attestation) = attestations.first() else { + return None; + }; + let attestation_slot = attestation.attestation.data().slot; + Self::calculate_unaggregated_attestation_deadline(attestation_slot, slot_clock) + } + Work::GossipAggregate { aggregate, .. } => { + let attestation_slot = aggregate.aggregate.message().aggregate().data().slot; + Self::calculate_aggregate_attestation_deadline(attestation_slot, slot_clock) + } + Work::GossipAggregateBatch { aggregates, .. } => { + let Some(aggregate) = aggregates.first() else { + return None; + }; + + let attestation_slot = aggregate.aggregate.message().aggregate().data().slot; + Self::calculate_aggregate_attestation_deadline(attestation_slot, slot_clock) + } + Work::UnknownBlockAttestation { .. } + | Work::UnknownBlockSamplingRequest { .. } + | Work::GossipBlobSidecar(_) + | Work::GossipDataColumnSidecar(_) + | Work::GossipProposerSlashing(_) + | Work::GossipAttesterSlashing(_) + | Work::GossipSyncSignature(_) + | Work::GossipSyncContribution(_) + | Work::RpcBlobs { .. } + | Work::RpcCustodyColumn { .. } + | Work::RpcVerifyDataColumn { .. } + | Work::SamplingResult(_) + | Work::BlocksByRangeRequest(_) + | Work::BlocksByRootsRequest(_) + | Work::BlobsByRangeRequest(_) + | Work::BlobsByRootsRequest(_) + | Work::DataColumnsByRootsRequest(_) + | Work::DataColumnsByRangeRequest(_) + | Work::GossipBlsToExecutionChange(_) => { + Some(current_time.saturating_add(Duration::from_secs(1))) + } + Work::UnknownLightClientOptimisticUpdate { .. } + | Work::GossipVoluntaryExit(_) + | Work::GossipLightClientFinalityUpdate(_) + | Work::GossipLightClientOptimisticUpdate(_) + | Work::Status(_) + | Work::LightClientBootstrapRequest(_) + | Work::LightClientOptimisticUpdateRequest(_) + | Work::LightClientFinalityUpdateRequest(_) + | Work::LightClientUpdatesByRangeRequest(_) + | Work::ApiRequestP0(_) + | Work::ApiRequestP1(_) => Some(current_time.saturating_add(Duration::from_secs(4))), + Work::RpcBlock { .. } + | Work::IgnoredRpcBlock { .. } + | Work::ChainSegment(_) + | Work::ChainSegmentBackfill(_) + | Work::UnknownBlockAggregate { .. } + | Work::GossipBlock(_) + | Work::DelayedImportBlock { .. } => Some(current_time), + Work::Reprocess(reprocess_queue_message) => { + Self::calculate_reprocess_deadline(reprocess_queue_message, slot_clock) + } + Work::GossipCanonicalBlock(_) | Work::RpcCanonicalBlock { .. } => { + Some(Duration::from_secs(0)) + } + }; + + println!("deadline {:?}", deadline); + + deadline + } + + /// An unaggregated attestation should be scheduled to be processed no later than within four seconds of the start of the current slot + /// or within a second of its arrival time if received later than the four second deadline. + fn calculate_unaggregated_attestation_deadline( + attestation_slot: Slot, + slot_clock: &S, + ) -> Option { + let Some(current_time) = slot_clock.now_duration() else { + return None; + }; + + let Some(start_of_attestation_slot) = slot_clock.start_of(attestation_slot) else { + return None; + }; + + let four_seconds_into_slot = + start_of_attestation_slot.saturating_add(Duration::from_secs(4)); + + let arrival_time_with_buffer = current_time.saturating_add(Duration::from_secs(1)); + Some(max(four_seconds_into_slot, arrival_time_with_buffer)) + } + + /// An aggregation attestation should be scheduled to be processed no later than the start of the next slot + /// or within a second of its arrival time if received later than the start of the next slot. + fn calculate_aggregate_attestation_deadline( + attestation_slot: Slot, + slot_clock: &S, + ) -> Option { + let Some(current_time) = slot_clock.now_duration() else { + return None; + }; + + let Some(start_of_next_slot) = slot_clock.start_of(attestation_slot + Slot::new(1)) else { + return None; + }; + let arrival_time_with_buffer = current_time.saturating_add(Duration::from_secs(1)); + + Some(max(start_of_next_slot, arrival_time_with_buffer)) + } + + fn calculate_reprocess_deadline( + reprocess_queue_message: &ReprocessQueueMessage, + slot_clock: &S, + ) -> Option { + let Some(current_time) = slot_clock.now_duration() else { + return None; + }; + + println!("reprocessing"); + + match reprocess_queue_message { + ReprocessQueueMessage::EarlyBlock(_) + | ReprocessQueueMessage::RpcBlock(_) + | ReprocessQueueMessage::BlockImported { .. } + | ReprocessQueueMessage::UnknownBlockUnaggregate(_) + | ReprocessQueueMessage::UnknownBlockAggregate(_) => Some(current_time), + ReprocessQueueMessage::NewLightClientOptimisticUpdate { .. } + | ReprocessQueueMessage::UnknownLightClientOptimisticUpdate(_) + | ReprocessQueueMessage::UnknownBlockSamplingRequest(_) => { + Some(current_time.saturating_add(Duration::from_secs(1))) + } + ReprocessQueueMessage::BackfillSync(_) => { + Some(current_time.saturating_add(Duration::from_secs(4))) + } + } + } +} + +impl std::cmp::Eq for QueueItem {} + +impl PartialEq for QueueItem { + fn eq(&self, other: &Self) -> bool { + self.deadline == other.deadline + } +} + +impl PartialOrd for QueueItem { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for QueueItem { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.deadline.cmp(&other.deadline) + } +} + +impl WorkQueue { + pub fn new() -> Self { + WorkQueue { + min_heap: BinaryHeap::new(), + } + } + + pub fn insert(&mut self, queue_item: Q) { + self.min_heap.push(Reverse(queue_item)) + } + + pub fn pop(&mut self) -> Option { + if let Some(queue_item) = self.min_heap.pop() { + Some(queue_item.0) + } else { + None + } + } + + fn _peek(&self) -> Option<&Reverse> { + self.min_heap.peek() + } + + pub fn len(&self) -> usize { + self.min_heap.len() + } + + // TODO do we want an is_full method? should there be a concept of full? + pub fn _is_full(&self) -> bool { + todo!() + } +} diff --git a/beacon_node/beacon_processor/src/scheduler/earliest_deadline_scheduler/mod.rs b/beacon_node/beacon_processor/src/scheduler/earliest_deadline_scheduler/mod.rs new file mode 100644 index 00000000000..d69636d6295 --- /dev/null +++ b/beacon_node/beacon_processor/src/scheduler/earliest_deadline_scheduler/mod.rs @@ -0,0 +1,225 @@ +use std::{sync::Arc, time::Duration}; + +use crate::{scheduler::InboundEvents, ReprocessQueueMessage}; +use earliest_deadline_queue::{QueueItem, WorkQueue}; +use slot_clock::SlotClock; +use tokio::sync::mpsc::{self, Sender}; +use tracing::{error, trace, warn}; +use types::EthSpec; + +use crate::{ + metrics, BeaconProcessor, Work, WorkEvent, WorkType, MAX_IDLE_QUEUE_LEN, NOTHING_TO_DO, +}; + +use super::{ + spawn_worker, + work_reprocessing_queue::{spawn_reprocess_scheduler, ReadyWork}, + worker_journal, NextWorkEvent, +}; + +mod earliest_deadline_queue; + +/// The name of the manager tokio task. +const MANAGER_TASK_NAME: &str = "earliest_deadline_first_scheduler"; + +pub struct Scheduler { + beacon_processor: BeaconProcessor, + work_queue: WorkQueue>, +} + +impl Scheduler { + pub fn new(beacon_processor: BeaconProcessor) -> Self { + let work_queue = WorkQueue::new(); + Scheduler { + beacon_processor, + work_queue, + } + } + + pub fn run( + mut self, + event_rx: mpsc::Receiver>, + work_journal_tx: Option>, + slot_clock: S, + maximum_gossip_clock_disparity: Duration, + ) -> Result<(), String> { + let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); + + let (ready_work_tx, ready_work_rx) = + mpsc::channel::(self.beacon_processor.config.max_scheduled_work_queue_len); + + let (reprocess_work_tx, reprocess_work_rx) = mpsc::channel::( + self.beacon_processor.config.max_scheduled_work_queue_len, + ); + + let executor = self.beacon_processor.executor.clone(); + + let mut inbound_events = InboundEvents { + idle_rx, + event_rx, + ready_work_rx, + }; + + spawn_reprocess_scheduler( + ready_work_tx, + reprocess_work_rx, + &self.beacon_processor.executor, + Arc::new(slot_clock.clone()), + maximum_gossip_clock_disparity, + )?; + + let manager_future = async move { + loop { + let work_event = match inbound_events + .next_work_event(&reprocess_work_tx, &mut self.beacon_processor) + .await + { + NextWorkEvent::WorkEvent(work_event) => work_event, + NextWorkEvent::Continue => continue, + NextWorkEvent::Break => break, + }; + + let can_spawn = self.beacon_processor.current_workers + < self.beacon_processor.config.max_workers; + + worker_journal(&work_event, &work_journal_tx); + let drop_during_sync = work_event + .as_ref() + .is_some_and(|event| event.drop_during_sync); + + let modified_queue_id = 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 => { + if let Some(queue_item) = self.work_queue.pop() { + self.process_or_queue_item( + &reprocess_work_tx, + &idle_tx, + queue_item, + can_spawn, + ) + } else { + // Let the journal know that a worker is freed and there's nothing else + // for it to do. + if let Some(work_journal_tx) = &work_journal_tx { + // We don't care if this message was successfully sent, we only use the journal + // during testing. + let _ = work_journal_tx.try_send(NOTHING_TO_DO); + } + None + } + } + // There is no new work event and we are unable to spawn a new worker. + // + // I cannot see any good reason why this would happen. + None => { + warn!( + msg = "no new work and cannot spawn worker", + "Unexpected gossip processor condition" + ); + None + } + // The chain is syncing and this event should be dropped during sync. + Some(work_event) + if self + .beacon_processor + .network_globals + .sync_state + .read() + .is_syncing() + && drop_during_sync => + { + let work_id = work_event.work.str_id(); + metrics::inc_counter_vec( + &metrics::BEACON_PROCESSOR_WORK_EVENTS_IGNORED_COUNT, + &[work_id], + ); + trace!( + msg = "chain is syncing", + work_id, + "Gossip processor skipping work", + ); + None + } + // There is a new work event and the chain is not syncing. Process it or queue + // it. + Some(work_event) => { + if let Some(queue_item) = QueueItem::new(work_event, &slot_clock) { + self.process_or_queue_item( + &reprocess_work_tx, + &idle_tx, + queue_item, + can_spawn, + ) + } else { + None + } + } + }; + + self.update_metrics(modified_queue_id); + } + }; + + // Spawn on the core executor. + executor.spawn(manager_future, MANAGER_TASK_NAME); + + Ok(()) + } + + fn update_metrics(&mut self, modified_queue_id: Option) { + metrics::set_gauge( + &metrics::BEACON_PROCESSOR_WORKERS_ACTIVE_TOTAL, + self.beacon_processor.current_workers as i64, + ); + + if let Some(modified_queue_id) = modified_queue_id { + metrics::observe_vec( + &metrics::BEACON_PROCESSOR_QUEUE_LENGTH, + &[modified_queue_id.into()], + self.work_queue.len() as f64, + ); + } + } + + pub fn process_or_queue_item( + &mut self, + reprocess_work_tx: &Sender, + idle_tx: &Sender<()>, + queue_item: QueueItem, + can_spawn: bool, + ) -> Option { + let work_type = queue_item.work_event.work_type(); + + let workers_available = + self.beacon_processor.config.max_workers - self.beacon_processor.current_workers; + + match queue_item.work_event.work { + Work::Reprocess(work_event) => { + if let Err(e) = reprocess_work_tx.try_send(work_event) { + error!( + error = %e, + "Failed to reprocess work event", + ) + } + } + _ if can_spawn => { + if queue_item.work_event.work.is_priority_work() || workers_available > 1 { + spawn_worker( + &mut self.beacon_processor, + idle_tx.clone(), + queue_item.work_event.work, + ) + } else { + self.work_queue.insert(queue_item); + } + } + _ => { + self.work_queue.insert(queue_item); + } + } + + Some(work_type) + } +} diff --git a/beacon_node/beacon_processor/src/scheduler/interface.rs b/beacon_node/beacon_processor/src/scheduler/interface.rs new file mode 100644 index 00000000000..8c6911de3d2 --- /dev/null +++ b/beacon_node/beacon_processor/src/scheduler/interface.rs @@ -0,0 +1,73 @@ +use std::time::Duration; + +use slot_clock::SlotClock; +use tokio::sync::mpsc; +use types::{BeaconState, ChainSpec, EthSpec}; + +use crate::{BeaconProcessor, WorkEvent}; + +use super::{earliest_deadline_scheduler, priority_scheduler}; + +pub trait Scheduler { + fn new( + beacon_processor: BeaconProcessor, + beacon_state: &BeaconState, + spec: &ChainSpec, + ) -> Result, String>; + + fn run( + self, + event_rx: mpsc::Receiver>, + work_journal_tx: Option>, + slot_clock: S, + maximum_gossip_clock_disparity: Duration, + ) -> Result<(), String>; +} + +pub enum SchedulerType { + PriorityScheduler(priority_scheduler::Scheduler), + EarliestDeadlineScheduler(earliest_deadline_scheduler::Scheduler), +} + +impl Scheduler for SchedulerType { + // TODO(beacon-processor) make this config driven + fn new( + beacon_processor: BeaconProcessor, + beacon_state: &BeaconState, + spec: &ChainSpec, + ) -> Result, String> { + match beacon_processor.config.beacon_processor_type { + crate::BeaconProcessorType::Priority => Ok(Box::new(SchedulerType::PriorityScheduler( + priority_scheduler::Scheduler::new(beacon_processor, beacon_state, spec)?, + ))), + crate::BeaconProcessorType::EarliestDeadline => { + Ok(Box::new(SchedulerType::EarliestDeadlineScheduler( + earliest_deadline_scheduler::Scheduler::new(beacon_processor), + ))) + } + } + } + + fn run( + self, + event_rx: mpsc::Receiver>, + work_journal_tx: Option>, + slot_clock: S, + maximum_gossip_clock_disparity: Duration, + ) -> Result<(), String> { + match self { + SchedulerType::PriorityScheduler(scheduler) => scheduler.run( + event_rx, + work_journal_tx, + slot_clock, + maximum_gossip_clock_disparity, + ), + SchedulerType::EarliestDeadlineScheduler(scheduler) => scheduler.run( + event_rx, + work_journal_tx, + slot_clock, + maximum_gossip_clock_disparity, + ), + } + } +} diff --git a/beacon_node/beacon_processor/src/scheduler/mod.rs b/beacon_node/beacon_processor/src/scheduler/mod.rs new file mode 100644 index 00000000000..f03c0a49c5e --- /dev/null +++ b/beacon_node/beacon_processor/src/scheduler/mod.rs @@ -0,0 +1,307 @@ +use std::task::Poll; + +use futures::stream::Stream; +use futures::StreamExt; +use logging::crit; +use std::pin::Pin; +use std::task::Context; +use tokio::sync::mpsc; +use tokio::sync::mpsc::error::TrySendError; +use tokio::sync::mpsc::Sender; +use tracing::{debug, trace, warn}; +use types::EthSpec; +use work_reprocessing_queue::ReadyWork; + +use crate::{metrics, QueuedBackfillBatch, ReprocessQueueMessage, WorkEvent, WORKER_FREED}; +use crate::{BeaconProcessor, BlockingOrAsync, SendOnDrop, TaskSpawner, Work}; + +mod earliest_deadline_scheduler; +pub mod interface; +mod priority_scheduler; +pub mod work_reprocessing_queue; + +/// Unifies all the messages processed by the `BeaconProcessor`. +enum InboundEvent { + /// A worker has completed a task and is free. + WorkerIdle, + /// There is new work to be done. + WorkEvent(WorkEvent), + /// A work event that was queued for re-processing has become ready. + 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 { + /// 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>, + ready_work_rx: mpsc::Receiver, +} + +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 + // stream of new events doesn't suppress the processing of existing events. + match self.idle_rx.poll_recv(cx) { + Poll::Ready(Some(())) => { + return Poll::Ready(Some(InboundEvent::WorkerIdle)); + } + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => {} + } + + // Poll for delayed blocks before polling for new work. It might be the case that a delayed + // block is required to successfully process some new work. + match self.ready_work_rx.poll_recv(cx) { + Poll::Ready(Some(ready_work)) => { + return Poll::Ready(Some(InboundEvent::ReprocessingWork(ready_work.into()))); + } + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => {} + } + + match self.event_rx.poll_recv(cx) { + Poll::Ready(Some(event)) => { + return Poll::Ready(Some(InboundEvent::WorkEvent(event))); + } + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => {} + } + + Poll::Pending + } +} + +pub enum NextWorkEvent { + WorkEvent(Option>), + Continue, + Break, +} + +impl InboundEvents { + pub async fn next_work_event( + &mut self, + reprocess_work_tx: &Sender, + beacon_processor: &mut BeaconProcessor, + ) -> NextWorkEvent { + match self.next().await { + Some(InboundEvent::WorkerIdle) => { + beacon_processor.current_workers = + beacon_processor.current_workers.saturating_sub(1); + NextWorkEvent::WorkEvent(None) + } + Some(InboundEvent::WorkEvent(event)) + if beacon_processor.config.enable_backfill_rate_limiting => + { + match QueuedBackfillBatch::try_from(event) { + Ok(backfill_batch) => { + match reprocess_work_tx + .try_send(ReprocessQueueMessage::BackfillSync(backfill_batch)) + { + Err(e) => { + warn!( + error = ?e, + "Unable to queue backfill work event. Will try to process now." + ); + match e { + TrySendError::Full(reprocess_queue_message) + | TrySendError::Closed(reprocess_queue_message) => { + match reprocess_queue_message { + ReprocessQueueMessage::BackfillSync(backfill_batch) => { + NextWorkEvent::WorkEvent(Some( + backfill_batch.into(), + )) + } + other => { + crit!( + message_type = other.as_ref(), + "Unexpected queue message type" + ); + // This is an unhandled exception, drop the message. + NextWorkEvent::Continue + } + } + } + } + } + Ok(..) => { + // backfill work sent to "reprocessing" queue. Process the next event. + NextWorkEvent::Continue + } + } + } + Err(event) => NextWorkEvent::WorkEvent(Some(event)), + } + } + Some(InboundEvent::WorkEvent(event)) | Some(InboundEvent::ReprocessingWork(event)) => { + NextWorkEvent::WorkEvent(Some(event)) + } + None => { + debug!(msg = "stream ended", "Gossip processor stopped",); + NextWorkEvent::Break + } + } + } +} + +/// 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. +pub fn spawn_worker( + beacon_processor: &mut BeaconProcessor, + idle_tx: Sender<()>, + work: Work, +) { + let work_id = work.str_id(); + let worker_timer = metrics::start_timer_vec(&metrics::BEACON_PROCESSOR_WORKER_TIME, &[work_id]); + metrics::inc_counter(&metrics::BEACON_PROCESSOR_WORKERS_SPAWNED_TOTAL); + metrics::inc_counter_vec( + &metrics::BEACON_PROCESSOR_WORK_EVENTS_STARTED_COUNT, + &[work.str_id()], + ); + + // Wrap the `idle_tx` in a struct that will fire the idle message whenever it is dropped. + // + // This helps ensure that the worker is always freed in the case of an early exit or panic. + // As such, this instantiation should happen as early in the function as possible. + let send_idle_on_drop = SendOnDrop { + tx: idle_tx, + _worker_timer: worker_timer, + }; + + let worker_id = beacon_processor.current_workers; + beacon_processor.current_workers = beacon_processor.current_workers.saturating_add(1); + + let executor = beacon_processor.executor.clone(); + + trace!( + work = work_id, + worker = worker_id, + "Spawning beacon processor worker" + ); + + let task_spawner = TaskSpawner { + executor, + send_idle_on_drop, + }; + + match work { + Work::GossipAttestation { + attestation, + process_individual, + process_batch: _, + } => task_spawner.spawn_blocking(move || { + process_individual(*attestation); + }), + Work::GossipAttestationToConvert { + attestation, + process_individual, + } => task_spawner.spawn_blocking(move || { + process_individual(*attestation); + }), + Work::GossipAttestationBatch { + attestations, + process_batch, + } => task_spawner.spawn_blocking(move || { + process_batch(attestations); + }), + Work::GossipAggregate { + aggregate, + process_individual, + process_batch: _, + } => task_spawner.spawn_blocking(move || { + process_individual(*aggregate); + }), + Work::GossipAggregateBatch { + aggregates, + process_batch, + } => task_spawner.spawn_blocking(move || { + process_batch(aggregates); + }), + Work::ChainSegment(process_fn) => task_spawner.spawn_async(async move { + process_fn.await; + }), + Work::UnknownBlockAttestation { process_fn } + | Work::UnknownBlockAggregate { process_fn } + | Work::UnknownLightClientOptimisticUpdate { process_fn, .. } + | Work::UnknownBlockSamplingRequest { 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 } + | Work::RpcCanonicalBlock { process_fn } + | Work::RpcBlobs { process_fn } + | Work::RpcCustodyColumn(process_fn) + | Work::RpcVerifyDataColumn(process_fn) + | Work::SamplingResult(process_fn) => task_spawner.spawn_async(process_fn), + Work::IgnoredRpcBlock { process_fn } => task_spawner.spawn_blocking(process_fn), + Work::GossipBlock(work) + | Work::GossipCanonicalBlock(work) + | Work::GossipBlobSidecar(work) + | Work::GossipDataColumnSidecar(work) => task_spawner.spawn_async(async move { + work.await; + }), + Work::BlobsByRangeRequest(process_fn) + | Work::BlobsByRootsRequest(process_fn) + | Work::DataColumnsByRootsRequest(process_fn) + | Work::DataColumnsByRangeRequest(process_fn) => task_spawner.spawn_blocking(process_fn), + Work::BlocksByRangeRequest(work) | Work::BlocksByRootsRequest(work) => { + task_spawner.spawn_async(work) + } + Work::ChainSegmentBackfill(process_fn) => task_spawner.spawn_async(process_fn), + Work::ApiRequestP0(process_fn) | Work::ApiRequestP1(process_fn) => match process_fn { + BlockingOrAsync::Blocking(process_fn) => task_spawner.spawn_blocking(process_fn), + BlockingOrAsync::Async(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) + | Work::LightClientOptimisticUpdateRequest(process_fn) + | Work::LightClientFinalityUpdateRequest(process_fn) + | Work::LightClientUpdatesByRangeRequest(process_fn) => { + task_spawner.spawn_blocking(process_fn) + } + Work::Reprocess(_) => {} + }; +} + +pub fn worker_journal( + work_event: &Option>, + work_journal_tx: &Option>, +) { + if let Some(work_journal_tx) = work_journal_tx { + let id = work_event + .as_ref() + .map(|event| event.work.str_id()) + .unwrap_or(WORKER_FREED); + + // We don't care if this message was successfully sent, we only use the journal + // during testing. We also ignore reprocess messages to ensure our test cases can pass. + if id != "reprocess" { + let _ = work_journal_tx.try_send(id); + } + } +} diff --git a/beacon_node/beacon_processor/src/scheduler/priority_scheduler/mod.rs b/beacon_node/beacon_processor/src/scheduler/priority_scheduler/mod.rs new file mode 100644 index 00000000000..a5925c47803 --- /dev/null +++ b/beacon_node/beacon_processor/src/scheduler/priority_scheduler/mod.rs @@ -0,0 +1,671 @@ +// The priority scheduler has three major facets +// 1. A priority ordering system +// 2. A backfill rate limiting feature +// 3. A retry queue + +mod work_queue; + +use crate::scheduler::work_reprocessing_queue::{spawn_reprocess_scheduler, ReadyWork}; +use crate::scheduler::InboundEvents; +use logging::crit; +use slot_clock::SlotClock; +use std::{cmp, marker::PhantomData, sync::Arc, time::Duration}; +use tokio::sync::mpsc::{self, Sender}; +use tracing::{error, trace, warn}; +use types::{BeaconState, ChainSpec, EthSpec}; +use work_queue::{BeaconProcessorQueueLengths, WorkQueues}; + +use crate::{ + metrics, BeaconProcessor, ReprocessQueueMessage, Work, WorkEvent, WorkType, MAX_IDLE_QUEUE_LEN, + NOTHING_TO_DO, +}; + +use super::{spawn_worker, worker_journal, NextWorkEvent}; + +/// The name of the manager tokio task. +const MANAGER_TASK_NAME: &str = "priority_scheduler"; + +// Backend trait inits a channel, a run function +// A channel trait has send_work, reprocess_work etc. +pub struct Scheduler { + beacon_processor: BeaconProcessor, + work_queues: WorkQueues, + phantom_data: PhantomData, +} + +impl Scheduler { + pub fn new( + beacon_processor: BeaconProcessor, + beacon_state: &BeaconState, + spec: &ChainSpec, + ) -> Result { + // Used by workers to communicate that they are finished a task. + + let queue_lengths = BeaconProcessorQueueLengths::from_state(beacon_state, spec)?; + + // Initialize the worker queues. + let work_queues: WorkQueues = WorkQueues::new(queue_lengths); + + Ok(Self { + beacon_processor, + work_queues, + phantom_data: PhantomData, + }) + } + + pub fn run( + mut self, + event_rx: mpsc::Receiver>, + work_journal_tx: Option>, + slot_clock: S, + maximum_gossip_clock_disparity: Duration, + ) -> Result<(), String> { + let (idle_tx, idle_rx) = mpsc::channel::<()>(MAX_IDLE_QUEUE_LEN); + + let (ready_work_tx, ready_work_rx) = + mpsc::channel::(self.beacon_processor.config.max_scheduled_work_queue_len); + + let (reprocess_work_tx, reprocess_work_rx) = mpsc::channel::( + self.beacon_processor.config.max_scheduled_work_queue_len, + ); + + let mut inbound_events = InboundEvents { + idle_rx, + event_rx, + ready_work_rx, + }; + + spawn_reprocess_scheduler( + ready_work_tx, + reprocess_work_rx, + &self.beacon_processor.executor, + Arc::new(slot_clock), + maximum_gossip_clock_disparity, + )?; + + let executor = self.beacon_processor.executor.clone(); + let manager_future = async move { + loop { + let work_event = match inbound_events + .next_work_event(&reprocess_work_tx, &mut self.beacon_processor) + .await + { + NextWorkEvent::WorkEvent(work_event) => work_event, + NextWorkEvent::Continue => continue, + NextWorkEvent::Break => break, + }; + + let _event_timer = self.increment_metrics(&work_event); + worker_journal(&work_event, &work_journal_tx); + + let can_spawn = self.beacon_processor.current_workers + < self.beacon_processor.config.max_workers; + let drop_during_sync = work_event + .as_ref() + .is_some_and(|event| event.drop_during_sync); + + let modified_queue_id = 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 work_event = self.priority_scheduler(&work_journal_tx); + if let Some(work_event) = work_event { + let work_type = work_event.to_type(); + spawn_worker(&mut self.beacon_processor, idle_tx.clone(), work_event); + Some(work_type) + } else { + None + } + } + // There is no new work event and we are unable to spawn a new worker. + // + // I cannot see any good reason why this would happen. + None => { + warn!( + msg = "no new work and cannot spawn worker", + "Unexpected gossip processor condition", + ); + None + } + // The chain is syncing and this event should be dropped during sync. + Some(work_event) + if self + .beacon_processor + .network_globals + .sync_state + .read() + .is_syncing() + && drop_during_sync => + { + let work_id = work_event.work.str_id(); + metrics::inc_counter_vec( + &metrics::BEACON_PROCESSOR_WORK_EVENTS_IGNORED_COUNT, + &[work_id], + ); + trace!( + work_id, + msg = "chain is syncing", + "Gossip processor skipping work", + ); + None + } + + // There is a new work event and the chain is not syncing. Process it or queue + // it. + Some(WorkEvent { work, .. }) => self.process_or_queue_work_event( + &reprocess_work_tx, + idle_tx.clone(), + work, + can_spawn, + ), + }; + + self.update_queue_metrics(modified_queue_id); + } + }; + + // Spawn on the core executor. + executor.spawn(manager_future, MANAGER_TASK_NAME); + + Ok(()) + } + + fn priority_scheduler( + &mut self, + work_journal_tx: &Option>, + ) -> Option> { + // Check for chain segments first, they're the most efficient way to get + // blocks into the system. + let work_event: Option> = + if let Some(item) = self.work_queues.chain_segment_queue.pop() { + Some(item) + // Check sync blocks before gossip blocks, since we've already explicitly + // requested these blocks. + } else if let Some(item) = self.work_queues.rpc_block_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.rpc_blob_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.rpc_custody_column_queue.pop() { + Some(item) + // TODO(das): decide proper prioritization for sampling columns + } else if let Some(item) = self.work_queues.rpc_custody_column_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.rpc_verify_data_column_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.sampling_result_queue.pop() { + Some(item) + // Check delayed blocks before gossip blocks, the gossip blocks might rely + // on the delayed ones. + } else if let Some(item) = self.work_queues.delayed_block_queue.pop() { + Some(item) + // Check gossip blocks before gossip attestations, since a block might be + // required to verify some attestations. + } else if let Some(item) = self.work_queues.gossip_block_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.gossip_blob_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.gossip_data_column_queue.pop() { + Some(item) + // Check the priority 0 API requests after blocks and blobs, but before attestations. + } else if let Some(item) = self.work_queues.api_request_p0_queue.pop() { + Some(item) + // Check the aggregates, *then* the unaggregates since we assume that + // aggregates are more valuable to local validators and effectively give us + // more information with less signature verification time. + } else if self.work_queues.aggregate_queue.len() > 0 { + let batch_size = cmp::min( + self.work_queues.aggregate_queue.len(), + self.beacon_processor.config.max_gossip_aggregate_batch_size, + ); + + if batch_size < 2 { + // One single aggregate is in the queue, process it individually. + self.work_queues.aggregate_queue.pop() + } else { + // Collect two or more aggregates into a batch, so they can take + // advantage of batch signature verification. + // + // Note: this will convert the `Work::GossipAggregate` item into a + // `Work::GossipAggregateBatch` item. + let mut aggregates = Vec::with_capacity(batch_size); + let mut process_batch_opt = None; + for _ in 0..batch_size { + if let Some(item) = self.work_queues.aggregate_queue.pop() { + match item { + Work::GossipAggregate { + aggregate, + process_individual: _, + process_batch, + } => { + aggregates.push(*aggregate); + if process_batch_opt.is_none() { + process_batch_opt = Some(process_batch); + } + } + _ => { + error!("Invalid item in aggregate queue"); + } + } + } + } + + if let Some(process_batch) = process_batch_opt { + // Process all aggregates with a single worker. + Some(Work::GossipAggregateBatch { + aggregates, + process_batch, + }) + } 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!("Missing aggregate work"); + None + } + } + // Check the unaggregated attestation queue. + // + // Potentially use batching. + } else if self.work_queues.attestation_queue.len() > 0 { + let batch_size = cmp::min( + self.work_queues.attestation_queue.len(), + self.beacon_processor + .config + .max_gossip_attestation_batch_size, + ); + + if batch_size < 2 { + // One single attestation is in the queue, process it individually. + self.work_queues.attestation_queue.pop() + } else { + // Collect two or more attestations into a batch, so they can take + // advantage of batch signature verification. + // + // Note: this will convert the `Work::GossipAttestation` item into a + // `Work::GossipAttestationBatch` item. + let mut attestations = Vec::with_capacity(batch_size); + let mut process_batch_opt = None; + for _ in 0..batch_size { + if let Some(item) = self.work_queues.attestation_queue.pop() { + match item { + Work::GossipAttestation { + attestation, + process_individual: _, + process_batch, + } => { + attestations.push(*attestation); + if process_batch_opt.is_none() { + process_batch_opt = Some(process_batch); + } + } + _ => error!("Invalid item in attestation queue"), + } + } + } + + if let Some(process_batch) = process_batch_opt { + // Process all attestations with a single worker. + Some(Work::GossipAttestationBatch { + attestations, + process_batch, + }) + } 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!("Missing attestations work"); + None + } + } + // Check sync committee messages after attestations as their rewards are lesser + // and they don't influence fork choice. + } else if let Some(item) = self.work_queues.sync_contribution_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.sync_message_queue.pop() { + Some(item) + // Aggregates and unaggregates queued for re-processing are older and we + // care about fresher ones, so check those first. + } else if let Some(item) = self.work_queues.unknown_block_aggregate_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.unknown_block_attestation_queue.pop() { + Some(item) + // 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) = self.work_queues.status_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.bbrange_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.bbroots_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.blbrange_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.blbroots_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.dcbroots_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.dcbrange_queue.pop() { + Some(item) + // Prioritize sampling requests after block syncing requests + } else if let Some(item) = self.work_queues.unknown_block_sampling_request_queue.pop() { + Some(item) + // 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) = self.work_queues.gossip_attester_slashing_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.gossip_proposer_slashing_queue.pop() { + Some(item) + // Check exits and address changes late since our validators don't get + // rewards from them. + } else if let Some(item) = self.work_queues.gossip_voluntary_exit_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.gossip_bls_to_execution_change_queue.pop() { + Some(item) + // Check the priority 1 API requests after we've + // processed all the interesting things from the network + // and things required for us to stay in good repute + // with our P2P peers. + } else if let Some(item) = self.work_queues.api_request_p1_queue.pop() { + Some(item) + // Handle backfill sync chain segments. + } else if let Some(item) = self.work_queues.backfill_chain_segment.pop() { + Some(item) + // Handle light client requests. + } else if let Some(item) = self.work_queues.lc_bootstrap_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.lc_optimistic_update_queue.pop() { + Some(item) + } else if let Some(item) = self.work_queues.lc_finality_update_queue.pop() { + Some(item) + // This statement should always be the final else statement. + } else { + // Let the journal know that a worker is freed and there's nothing else + // for it to do. + if let Some(work_journal_tx) = &work_journal_tx { + // We don't care if this message was successfully sent, we only use the journal + // during testing. + let _ = work_journal_tx.try_send(NOTHING_TO_DO); + } + None + }; + + work_event + } + + pub fn process_or_queue_work_event( + &mut self, + reprocess_work_tx: &Sender, + idle_tx: Sender<()>, + work: Work, + can_spawn: bool, + ) -> Option { + let work_id = work.str_id(); + + let work_type = work.to_type(); + + match work { + Work::Reprocess(work_event) => { + if let Err(e) = reprocess_work_tx.try_send(work_event) { + error!( + error = ?e, + "Failed to reprocess work event" + ) + } + } + _ if can_spawn => spawn_worker(&mut self.beacon_processor, idle_tx.clone(), work), + Work::GossipAttestation { .. } => self.work_queues.attestation_queue.push(work), + Work::GossipAttestationToConvert { .. } => { + self.work_queues.attestation_to_convert_queue.push(work) + } + // Attestation batches are formed internally within the + // `BeaconProcessor`, they are not sent from external services. + work_type @ Work::GossipAttestationBatch { .. } => { + crit!(?work_type, "Unsupported inbound event") + } + Work::GossipAggregate { .. } => self.work_queues.aggregate_queue.push(work), + // Aggregate batches are formed internally within the `BeaconProcessor`, + // they are not sent from external services. + work_type @ Work::GossipAggregateBatch { .. } => { + crit!(?work_type, "Unsupported inbound event") + } + Work::GossipBlock { .. } | Work::GossipCanonicalBlock { .. } => { + self.work_queues.gossip_block_queue.push(work, work_id) + } + Work::GossipBlobSidecar { .. } => { + self.work_queues.gossip_blob_queue.push(work, work_id) + } + Work::GossipDataColumnSidecar { .. } => self + .work_queues + .gossip_data_column_queue + .push(work, work_id), + Work::DelayedImportBlock { .. } => { + self.work_queues.delayed_block_queue.push(work, work_id) + } + Work::GossipVoluntaryExit { .. } => self + .work_queues + .gossip_voluntary_exit_queue + .push(work, work_id), + Work::GossipProposerSlashing { .. } => self + .work_queues + .gossip_proposer_slashing_queue + .push(work, work_id), + Work::GossipAttesterSlashing { .. } => self + .work_queues + .gossip_attester_slashing_queue + .push(work, work_id), + Work::GossipSyncSignature { .. } => self.work_queues.sync_message_queue.push(work), + Work::GossipSyncContribution { .. } => { + self.work_queues.sync_contribution_queue.push(work) + } + Work::GossipLightClientFinalityUpdate { .. } => { + self.work_queues.finality_update_queue.push(work, work_id) + } + Work::GossipLightClientOptimisticUpdate { .. } => { + self.work_queues.optimistic_update_queue.push(work, work_id) + } + Work::RpcBlock { .. } + | Work::IgnoredRpcBlock { .. } + | Work::RpcCanonicalBlock { .. } => { + self.work_queues.rpc_block_queue.push(work, work_id) + } + Work::RpcBlobs { .. } => self.work_queues.rpc_blob_queue.push(work, work_id), + Work::RpcCustodyColumn { .. } => self + .work_queues + .rpc_custody_column_queue + .push(work, work_id), + Work::RpcVerifyDataColumn(_) => self + .work_queues + .rpc_verify_data_column_queue + .push(work, work_id), + Work::SamplingResult(_) => self.work_queues.sampling_result_queue.push(work, work_id), + Work::ChainSegment { .. } => self.work_queues.chain_segment_queue.push(work, work_id), + Work::ChainSegmentBackfill { .. } => { + self.work_queues.backfill_chain_segment.push(work, work_id) + } + Work::Status { .. } => self.work_queues.status_queue.push(work, work_id), + Work::BlocksByRangeRequest { .. } => self.work_queues.bbrange_queue.push(work, work_id), + Work::BlocksByRootsRequest { .. } => self.work_queues.bbroots_queue.push(work, work_id), + Work::BlobsByRangeRequest { .. } => self.work_queues.blbrange_queue.push(work, work_id), + Work::LightClientBootstrapRequest { .. } => { + self.work_queues.lc_bootstrap_queue.push(work, work_id) + } + Work::LightClientOptimisticUpdateRequest { .. } => self + .work_queues + .lc_optimistic_update_queue + .push(work, work_id), + Work::LightClientFinalityUpdateRequest { .. } => self + .work_queues + .lc_finality_update_queue + .push(work, work_id), + Work::LightClientUpdatesByRangeRequest { .. } => { + self.work_queues.lc_update_range_queue.push(work, work_id) + } + Work::UnknownBlockAttestation { .. } => { + self.work_queues.unknown_block_attestation_queue.push(work) + } + Work::UnknownBlockAggregate { .. } => { + self.work_queues.unknown_block_aggregate_queue.push(work) + } + Work::GossipBlsToExecutionChange { .. } => self + .work_queues + .gossip_bls_to_execution_change_queue + .push(work, work_id), + Work::BlobsByRootsRequest { .. } => self.work_queues.blbroots_queue.push(work, work_id), + Work::DataColumnsByRootsRequest { .. } => { + self.work_queues.dcbroots_queue.push(work, work_id) + } + Work::DataColumnsByRangeRequest { .. } => { + self.work_queues.dcbrange_queue.push(work, work_id) + } + Work::UnknownLightClientOptimisticUpdate { .. } => self + .work_queues + .unknown_light_client_update_queue + .push(work, work_id), + Work::UnknownBlockSamplingRequest { .. } => self + .work_queues + .unknown_block_sampling_request_queue + .push(work, work_id), + Work::ApiRequestP0 { .. } => self.work_queues.api_request_p0_queue.push(work, work_id), + Work::ApiRequestP1 { .. } => self.work_queues.api_request_p1_queue.push(work, work_id), + } + Some(work_type) + } + + fn update_queue_metrics(&mut self, modified_queue_id: Option) { + metrics::set_gauge( + &metrics::BEACON_PROCESSOR_WORKERS_ACTIVE_TOTAL, + self.beacon_processor.current_workers as i64, + ); + + if let Some(modified_queue_id) = modified_queue_id { + let queue_len = match modified_queue_id { + WorkType::GossipAttestation => self.work_queues.aggregate_queue.len(), + WorkType::GossipAttestationToConvert => self.work_queues.aggregate_queue.len(), + WorkType::UnknownBlockAttestation => { + self.work_queues.unknown_block_attestation_queue.len() + } + WorkType::GossipAttestationBatch => 0, // No queue + WorkType::GossipAggregate => self.work_queues.aggregate_queue.len(), + WorkType::UnknownBlockAggregate => { + self.work_queues.unknown_block_aggregate_queue.len() + } + WorkType::UnknownLightClientOptimisticUpdate => { + self.work_queues.unknown_light_client_update_queue.len() + } + WorkType::UnknownBlockSamplingRequest => { + self.work_queues.unknown_block_sampling_request_queue.len() + } + WorkType::GossipAggregateBatch => 0, // No queue + WorkType::GossipBlock | WorkType::GossipCanonicalBlock => { + self.work_queues.gossip_block_queue.len() + } + WorkType::GossipBlobSidecar => self.work_queues.gossip_blob_queue.len(), + WorkType::GossipDataColumnSidecar => { + self.work_queues.gossip_data_column_queue.len() + } + WorkType::DelayedImportBlock => self.work_queues.delayed_block_queue.len(), + WorkType::GossipVoluntaryExit => self.work_queues.gossip_voluntary_exit_queue.len(), + WorkType::GossipProposerSlashing => { + self.work_queues.gossip_proposer_slashing_queue.len() + } + WorkType::GossipAttesterSlashing => { + self.work_queues.gossip_attester_slashing_queue.len() + } + WorkType::GossipSyncSignature => self.work_queues.sync_message_queue.len(), + WorkType::GossipSyncContribution => self.work_queues.sync_contribution_queue.len(), + WorkType::GossipLightClientFinalityUpdate => { + self.work_queues.finality_update_queue.len() + } + WorkType::GossipLightClientOptimisticUpdate => { + self.work_queues.optimistic_update_queue.len() + } + WorkType::RpcBlock | WorkType::RpcCanonicalBlock => { + self.work_queues.rpc_block_queue.len() + } + WorkType::RpcBlobs | WorkType::IgnoredRpcBlock => { + self.work_queues.rpc_blob_queue.len() + } + WorkType::RpcCustodyColumn => self.work_queues.rpc_custody_column_queue.len(), + WorkType::RpcVerifyDataColumn => { + self.work_queues.rpc_verify_data_column_queue.len() + } + WorkType::SamplingResult => self.work_queues.sampling_result_queue.len(), + WorkType::ChainSegment => self.work_queues.chain_segment_queue.len(), + WorkType::ChainSegmentBackfill => self.work_queues.backfill_chain_segment.len(), + WorkType::Status => self.work_queues.status_queue.len(), + WorkType::BlocksByRangeRequest => self.work_queues.blbrange_queue.len(), + WorkType::BlocksByRootsRequest => self.work_queues.blbroots_queue.len(), + WorkType::BlobsByRangeRequest => self.work_queues.bbrange_queue.len(), + WorkType::BlobsByRootsRequest => self.work_queues.bbroots_queue.len(), + WorkType::DataColumnsByRootsRequest => self.work_queues.dcbroots_queue.len(), + WorkType::DataColumnsByRangeRequest => self.work_queues.dcbrange_queue.len(), + WorkType::GossipBlsToExecutionChange => { + self.work_queues.gossip_bls_to_execution_change_queue.len() + } + WorkType::LightClientBootstrapRequest => self.work_queues.lc_bootstrap_queue.len(), + WorkType::LightClientOptimisticUpdateRequest => { + self.work_queues.lc_optimistic_update_queue.len() + } + WorkType::LightClientUpdatesByRangeRequest => { + self.work_queues.lc_update_range_queue.len() + } + WorkType::LightClientFinalityUpdateRequest => { + self.work_queues.lc_finality_update_queue.len() + } + WorkType::ApiRequestP0 => self.work_queues.api_request_p0_queue.len(), + WorkType::ApiRequestP1 => self.work_queues.api_request_p1_queue.len(), + WorkType::Reprocess => 0, + }; + metrics::observe_vec( + &metrics::BEACON_PROCESSOR_QUEUE_LENGTH, + &[modified_queue_id.into()], + queue_len as f64, + ); + } + + if self.work_queues.aggregate_queue.is_full() + && self.work_queues.aggregate_debounce.elapsed() + { + error!( + msg = "the system has insufficient resources for load", + queue_len = self.work_queues.aggregate_queue.max_length, + "Aggregate attestation queue full", + ) + } + + if self.work_queues.attestation_queue.is_full() + && self.work_queues.attestation_debounce.elapsed() + { + error!( + msg = "the system has insufficient resources for load", + queue_len = self.work_queues.attestation_queue.max_length, + "Attestation queue full", + ) + } + } + + // TODO(beacon-processor) this can live outside of this struct in a more general location + fn increment_metrics( + &self, + work_event: &Option>, + ) -> Option { + let _event_timer = metrics::start_timer(&metrics::BEACON_PROCESSOR_EVENT_HANDLING_SECONDS); + if let Some(event) = work_event { + metrics::inc_counter_vec( + &metrics::BEACON_PROCESSOR_WORK_EVENTS_RX_COUNT, + &[event.work.str_id()], + ); + } else { + metrics::inc_counter(&metrics::BEACON_PROCESSOR_IDLE_EVENTS_TOTAL); + } + _event_timer + } +} diff --git a/beacon_node/beacon_processor/src/scheduler/priority_scheduler/work_queue.rs b/beacon_node/beacon_processor/src/scheduler/priority_scheduler/work_queue.rs new file mode 100644 index 00000000000..c601dbbdfcf --- /dev/null +++ b/beacon_node/beacon_processor/src/scheduler/priority_scheduler/work_queue.rs @@ -0,0 +1,386 @@ +use crate::Work; +use logging::TimeLatch; +use std::collections::VecDeque; +use tracing::error; +use types::{BeaconState, ChainSpec, EthSpec, RelativeEpoch}; + +/// Over-provision queues based on active validator count by some factor. The beacon chain has +/// strict churns that prevent the validator set size from changing rapidly. By over-provisioning +/// slightly, we don't need to adjust the queues during the lifetime of a process. +const ACTIVE_VALIDATOR_COUNT_OVERPROVISION_PERCENT: usize = 110; + +/// Minimum size of dynamically sized queues. Due to integer division we don't want 0 length queues +/// as the processor won't process that message type. 128 is an arbitrary value value >= 1 that +/// seems reasonable. +const MIN_QUEUE_LEN: usize = 128; + +/// A simple first-in-first-out queue with a maximum length. +pub struct FifoQueue { + queue: VecDeque, + max_length: usize, +} + +impl FifoQueue { + /// Create a new, empty queue with the given length. + pub fn new(max_length: usize) -> Self { + Self { + queue: VecDeque::default(), + max_length, + } + } + + /// Add a new item to the queue. + /// + /// Drops `item` if the queue is full. + pub fn push(&mut self, item: T, item_desc: &str) { + if self.queue.len() == self.max_length { + error!( + queue = item_desc, + queue_len = self.max_length, + msg = "the system has insufficient resources for load", + "Work queue is full", + ) + } else { + self.queue.push_back(item); + } + } + + /// Remove the next item from the queue. + pub fn pop(&mut self) -> Option { + self.queue.pop_front() + } + + /// Returns the current length of the queue. + pub fn len(&self) -> usize { + self.queue.len() + } +} + +/// A simple last-in-first-out queue with a maximum length. +pub struct LifoQueue { + queue: VecDeque, + pub max_length: usize, +} + +impl LifoQueue { + /// Create a new, empty queue with the given length. + pub fn new(max_length: usize) -> Self { + Self { + queue: VecDeque::default(), + max_length, + } + } + + /// Add a new item to the front of the queue. + /// + /// If the queue is full, the item at the back of the queue is dropped. + pub fn push(&mut self, item: T) { + if self.queue.len() == self.max_length { + self.queue.pop_back(); + } + self.queue.push_front(item); + } + + /// Remove the next item from the queue. + pub fn pop(&mut self) -> Option { + self.queue.pop_front() + } + + /// Returns `true` if the queue is full. + pub fn is_full(&self) -> bool { + self.queue.len() >= self.max_length + } + + /// Returns the current length of the queue. + pub fn len(&self) -> usize { + self.queue.len() + } +} + +/// Maximum number of queued items that will be stored before dropping them +pub struct BeaconProcessorQueueLengths { + aggregate_queue: usize, + attestation_queue: usize, + unknown_block_aggregate_queue: usize, + unknown_block_attestation_queue: usize, + sync_message_queue: usize, + sync_contribution_queue: usize, + gossip_voluntary_exit_queue: usize, + gossip_proposer_slashing_queue: usize, + gossip_attester_slashing_queue: usize, + finality_update_queue: usize, + optimistic_update_queue: usize, + unknown_light_client_update_queue: usize, + unknown_block_sampling_request_queue: usize, + rpc_block_queue: usize, + rpc_blob_queue: usize, + rpc_custody_column_queue: usize, + rpc_verify_data_column_queue: usize, + sampling_result_queue: usize, + chain_segment_queue: usize, + backfill_chain_segment: usize, + gossip_block_queue: usize, + gossip_blob_queue: usize, + gossip_data_column_queue: usize, + delayed_block_queue: usize, + status_queue: usize, + bbrange_queue: usize, + bbroots_queue: usize, + blbroots_queue: usize, + blbrange_queue: usize, + dcbroots_queue: usize, + dcbrange_queue: usize, + gossip_bls_to_execution_change_queue: usize, + lc_bootstrap_queue: usize, + lc_optimistic_update_queue: usize, + lc_finality_update_queue: usize, + lc_update_range_queue: usize, + api_request_p0_queue: usize, + api_request_p1_queue: usize, +} + +impl BeaconProcessorQueueLengths { + pub fn from_state( + state: &BeaconState, + spec: &ChainSpec, + ) -> Result { + let active_validator_count = + match state.get_cached_active_validator_indices(RelativeEpoch::Current) { + Ok(indices) => indices.len(), + Err(_) => state + .get_active_validator_indices(state.current_epoch(), spec) + .map_err(|e| format!("Error computing active indices: {:?}", e))? + .len(), + }; + let active_validator_count = + (ACTIVE_VALIDATOR_COUNT_OVERPROVISION_PERCENT * active_validator_count) / 100; + let slots_per_epoch = E::slots_per_epoch() as usize; + + Ok(Self { + aggregate_queue: 4096, + unknown_block_aggregate_queue: 1024, + // Capacity for a full slot's worth of attestations if subscribed to all subnets + attestation_queue: std::cmp::max( + active_validator_count / slots_per_epoch, + MIN_QUEUE_LEN, + ), + // Capacity for a full slot's worth of attestations if subscribed to all subnets + unknown_block_attestation_queue: std::cmp::max( + active_validator_count / slots_per_epoch, + MIN_QUEUE_LEN, + ), + sync_message_queue: 2048, + sync_contribution_queue: 1024, + gossip_voluntary_exit_queue: 4096, + gossip_proposer_slashing_queue: 4096, + gossip_attester_slashing_queue: 4096, + finality_update_queue: 1024, + optimistic_update_queue: 1024, + unknown_block_sampling_request_queue: 16384, + unknown_light_client_update_queue: 128, + rpc_block_queue: 1024, + rpc_blob_queue: 1024, + // TODO(das): Placeholder values + rpc_custody_column_queue: 1000, + rpc_verify_data_column_queue: 1000, + sampling_result_queue: 1000, + chain_segment_queue: 64, + backfill_chain_segment: 64, + gossip_block_queue: 1024, + gossip_blob_queue: 1024, + gossip_data_column_queue: 1024, + delayed_block_queue: 1024, + status_queue: 1024, + bbrange_queue: 1024, + bbroots_queue: 1024, + blbroots_queue: 1024, + blbrange_queue: 1024, + // TODO(das): pick proper values + dcbroots_queue: 1024, + dcbrange_queue: 1024, + gossip_bls_to_execution_change_queue: 16384, + lc_bootstrap_queue: 1024, + lc_optimistic_update_queue: 512, + lc_finality_update_queue: 512, + lc_update_range_queue: 512, + api_request_p0_queue: 1024, + api_request_p1_queue: 1024, + }) + } +} + +pub struct WorkQueues { + pub aggregate_queue: LifoQueue>, + pub aggregate_debounce: TimeLatch, + pub attestation_queue: LifoQueue>, + pub attestation_to_convert_queue: LifoQueue>, + pub attestation_debounce: TimeLatch, + pub unknown_block_aggregate_queue: LifoQueue>, + pub unknown_block_attestation_queue: LifoQueue>, + pub sync_message_queue: LifoQueue>, + pub sync_contribution_queue: LifoQueue>, + pub gossip_voluntary_exit_queue: FifoQueue>, + pub gossip_proposer_slashing_queue: FifoQueue>, + pub gossip_attester_slashing_queue: FifoQueue>, + pub finality_update_queue: FifoQueue>, + pub optimistic_update_queue: FifoQueue>, + pub unknown_light_client_update_queue: FifoQueue>, + pub unknown_block_sampling_request_queue: FifoQueue>, + pub rpc_block_queue: FifoQueue>, + pub rpc_blob_queue: FifoQueue>, + pub rpc_custody_column_queue: FifoQueue>, + pub rpc_verify_data_column_queue: FifoQueue>, + pub sampling_result_queue: FifoQueue>, + pub chain_segment_queue: FifoQueue>, + pub backfill_chain_segment: FifoQueue>, + pub gossip_block_queue: FifoQueue>, + pub gossip_blob_queue: FifoQueue>, + pub gossip_data_column_queue: FifoQueue>, + pub delayed_block_queue: FifoQueue>, + pub status_queue: FifoQueue>, + pub bbrange_queue: FifoQueue>, + pub bbroots_queue: FifoQueue>, + pub blbroots_queue: FifoQueue>, + pub blbrange_queue: FifoQueue>, + pub dcbroots_queue: FifoQueue>, + pub dcbrange_queue: FifoQueue>, + pub gossip_bls_to_execution_change_queue: FifoQueue>, + pub lc_bootstrap_queue: FifoQueue>, + pub lc_optimistic_update_queue: FifoQueue>, + pub lc_finality_update_queue: FifoQueue>, + pub lc_update_range_queue: FifoQueue>, + pub api_request_p0_queue: FifoQueue>, + pub api_request_p1_queue: FifoQueue>, +} + +impl WorkQueues { + pub fn new(queue_lengths: BeaconProcessorQueueLengths) -> Self { + let aggregate_queue = LifoQueue::new(queue_lengths.aggregate_queue); + let aggregate_debounce = TimeLatch::default(); + let attestation_queue = LifoQueue::new(queue_lengths.attestation_queue); + let attestation_to_convert_queue = LifoQueue::new(queue_lengths.attestation_queue); + let attestation_debounce = TimeLatch::default(); + let unknown_block_aggregate_queue = + LifoQueue::new(queue_lengths.unknown_block_aggregate_queue); + let unknown_block_attestation_queue = + LifoQueue::new(queue_lengths.unknown_block_attestation_queue); + + let sync_message_queue = LifoQueue::new(queue_lengths.sync_message_queue); + let sync_contribution_queue = LifoQueue::new(queue_lengths.sync_contribution_queue); + + // Using a FIFO queue for voluntary exits since it prevents exit censoring. I don't have + // a strong feeling about queue type for exits. + let gossip_voluntary_exit_queue = FifoQueue::new(queue_lengths.gossip_voluntary_exit_queue); + + // Using a FIFO queue for slashing to prevent people from flushing their slashings from the + // queues with lots of junk messages. + let gossip_proposer_slashing_queue = + FifoQueue::new(queue_lengths.gossip_proposer_slashing_queue); + let gossip_attester_slashing_queue = + FifoQueue::new(queue_lengths.gossip_attester_slashing_queue); + + // Using a FIFO queue for light client updates to maintain sequence order. + let finality_update_queue = FifoQueue::new(queue_lengths.finality_update_queue); + let optimistic_update_queue = FifoQueue::new(queue_lengths.optimistic_update_queue); + let unknown_light_client_update_queue = + FifoQueue::new(queue_lengths.unknown_light_client_update_queue); + let unknown_block_sampling_request_queue = + FifoQueue::new(queue_lengths.unknown_block_sampling_request_queue); + + // Using a FIFO queue since blocks need to be imported sequentially. + let rpc_block_queue = FifoQueue::new(queue_lengths.rpc_block_queue); + let rpc_blob_queue = FifoQueue::new(queue_lengths.rpc_blob_queue); + let rpc_custody_column_queue = FifoQueue::new(queue_lengths.rpc_custody_column_queue); + let rpc_verify_data_column_queue = + FifoQueue::new(queue_lengths.rpc_verify_data_column_queue); + let sampling_result_queue = FifoQueue::new(queue_lengths.sampling_result_queue); + let chain_segment_queue = FifoQueue::new(queue_lengths.chain_segment_queue); + let backfill_chain_segment = FifoQueue::new(queue_lengths.backfill_chain_segment); + let gossip_block_queue = FifoQueue::new(queue_lengths.gossip_block_queue); + let gossip_blob_queue = FifoQueue::new(queue_lengths.gossip_blob_queue); + let gossip_data_column_queue = FifoQueue::new(queue_lengths.gossip_data_column_queue); + let delayed_block_queue = FifoQueue::new(queue_lengths.delayed_block_queue); + + let status_queue = FifoQueue::new(queue_lengths.status_queue); + let bbrange_queue = FifoQueue::new(queue_lengths.bbrange_queue); + let bbroots_queue = FifoQueue::new(queue_lengths.bbroots_queue); + let blbroots_queue = FifoQueue::new(queue_lengths.blbroots_queue); + let blbrange_queue = FifoQueue::new(queue_lengths.blbrange_queue); + let dcbroots_queue = FifoQueue::new(queue_lengths.dcbroots_queue); + let dcbrange_queue = FifoQueue::new(queue_lengths.dcbrange_queue); + + let gossip_bls_to_execution_change_queue = + FifoQueue::new(queue_lengths.gossip_bls_to_execution_change_queue); + + let lc_bootstrap_queue = FifoQueue::new(queue_lengths.lc_bootstrap_queue); + let lc_optimistic_update_queue = FifoQueue::new(queue_lengths.lc_optimistic_update_queue); + let lc_finality_update_queue = FifoQueue::new(queue_lengths.lc_finality_update_queue); + let lc_update_range_queue: FifoQueue> = + FifoQueue::new(queue_lengths.lc_update_range_queue); + + let api_request_p0_queue = FifoQueue::new(queue_lengths.api_request_p0_queue); + let api_request_p1_queue = FifoQueue::new(queue_lengths.api_request_p1_queue); + + WorkQueues { + aggregate_queue, + aggregate_debounce, + attestation_queue, + attestation_to_convert_queue, + attestation_debounce, + unknown_block_aggregate_queue, + unknown_block_attestation_queue, + sync_message_queue, + sync_contribution_queue, + gossip_voluntary_exit_queue, + gossip_proposer_slashing_queue, + gossip_attester_slashing_queue, + finality_update_queue, + optimistic_update_queue, + unknown_light_client_update_queue, + unknown_block_sampling_request_queue, + rpc_block_queue, + rpc_blob_queue, + rpc_custody_column_queue, + rpc_verify_data_column_queue, + sampling_result_queue, + chain_segment_queue, + backfill_chain_segment, + gossip_block_queue, + gossip_blob_queue, + gossip_data_column_queue, + delayed_block_queue, + status_queue, + bbrange_queue, + bbroots_queue, + blbroots_queue, + blbrange_queue, + dcbroots_queue, + dcbrange_queue, + gossip_bls_to_execution_change_queue, + lc_bootstrap_queue, + lc_optimistic_update_queue, + lc_finality_update_queue, + lc_update_range_queue, + api_request_p0_queue, + api_request_p1_queue, + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use types::{BeaconState, ChainSpec, Eth1Data, ForkName, MainnetEthSpec}; + + #[test] + fn min_queue_len() { + // State with no validators. + let spec = ForkName::latest().make_genesis_spec(ChainSpec::mainnet()); + let genesis_time = 0; + let state = BeaconState::::new(genesis_time, Eth1Data::default(), &spec); + assert_eq!(state.validators().len(), 0); + let queue_lengths = BeaconProcessorQueueLengths::from_state(&state, &spec).unwrap(); + assert_eq!(queue_lengths.attestation_queue, MIN_QUEUE_LEN); + assert_eq!(queue_lengths.unknown_block_attestation_queue, MIN_QUEUE_LEN); + } +} diff --git a/beacon_node/beacon_processor/src/scheduler/work_reprocessing_queue.rs b/beacon_node/beacon_processor/src/scheduler/work_reprocessing_queue.rs new file mode 100644 index 00000000000..9392b48cbf3 --- /dev/null +++ b/beacon_node/beacon_processor/src/scheduler/work_reprocessing_queue.rs @@ -0,0 +1,1092 @@ +//! Provides a mechanism which queues work for later processing. +//! +//! When the `beacon_processor::Worker` imports a block that is acceptably early (i.e., within the +//! gossip propagation tolerance) it will send it to this queue where it will be placed in a +//! `DelayQueue` until the slot arrives. Once the block has been determined to be ready, it will be +//! sent back out on a channel to be processed by the `BeaconProcessor` again. +//! +//! There is the edge-case where the slot arrives before this queue manages to process it. In that +//! case, the block will be sent off for immediate processing (skipping the `DelayQueue`). +//! +//! 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 crate::ReprocessQueueMessage::*; +use crate::{ + metrics, IgnoredRpcBlock, QueuedAggregate, QueuedBackfillBatch, QueuedGossipBlock, + QueuedLightClientUpdate, QueuedRpcBlock, QueuedSamplingRequest, QueuedUnaggregate, + ReprocessQueueMessage, +}; +use crate::{Work, WorkEvent}; +use fnv::FnvHashMap; +use futures::task::Poll; +use futures::{Stream, StreamExt}; +use itertools::Itertools; +use logging::crit; +use logging::TimeLatch; +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 task_executor::TaskExecutor; +use tokio::sync::mpsc::{self, Receiver, Sender}; +use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey}; +use tracing::{debug, error, trace, warn}; +use types::{EthSpec, Hash256}; + +const TASK_NAME: &str = "beacon_processor_reprocess_queue"; +const GOSSIP_BLOCKS: &str = "gossip_blocks"; +const RPC_BLOCKS: &str = "rpc_blocks"; +const ATTESTATIONS: &str = "attestations"; +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. +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); + +/// For how long to queue light client updates for re-processing. +pub const QUEUED_LIGHT_CLIENT_UPDATE_DELAY: Duration = Duration::from_secs(12); + +/// For how long to queue rpc blocks before sending them back for reprocessing. +pub const QUEUED_RPC_BLOCK_DELAY: Duration = Duration::from_secs(4); + +/// For how long to queue sampling requests for reprocessing. +pub const QUEUED_SAMPLING_REQUESTS_DELAY: Duration = Duration::from_secs(12); + +/// Set an arbitrary upper-bound on the number of queued blocks to avoid DoS attacks. The fact that +/// we signature-verify blocks before putting them in the queue *should* protect against this, but +/// it's nice to have extra protection. +const MAXIMUM_QUEUED_BLOCKS: usize = 16; + +/// How many attestations we keep before new ones get dropped. +const MAXIMUM_QUEUED_ATTESTATIONS: usize = 16_384; + +/// How many light client updates we keep before new ones get dropped. +const MAXIMUM_QUEUED_LIGHT_CLIENT_UPDATES: usize = 128; + +/// How many sampling requests we queue before new ones get dropped. +/// TODO(das): choose a sensible value +const MAXIMUM_QUEUED_SAMPLING_REQUESTS: usize = 16_384; + +// Process backfill batch 50%, 60%, 80% through each slot. +// +// Note: use caution to set these fractions in a way that won't cause panic-y +// arithmetic. +pub const BACKFILL_SCHEDULE_IN_SLOT: [(u32, u32); 3] = [ + // One half: 6s on mainnet, 2.5s on Gnosis. + (1, 2), + // Three fifths: 7.2s on mainnet, 3s on Gnosis. + (3, 5), + // Four fifths: 9.6s on mainnet, 4s on Gnosis. + (4, 5), +]; + +/// Events sent by the scheduler once they are ready for re-processing. +pub enum ReadyWork { + Block(QueuedGossipBlock), + RpcBlock(QueuedRpcBlock), + IgnoredRpcBlock(IgnoredRpcBlock), + Unaggregate(QueuedUnaggregate), + Aggregate(QueuedAggregate), + LightClientUpdate(QueuedLightClientUpdate), + SamplingRequest(QueuedSamplingRequest), + BackfillSync(QueuedBackfillBatch), +} + +impl From for WorkEvent { + fn from(ready_work: ReadyWork) -> Self { + match ready_work { + ReadyWork::Block(QueuedGossipBlock { + beacon_block_slot, + beacon_block_root, + process_fn, + }) => Self { + drop_during_sync: false, + work: Work::DelayedImportBlock { + beacon_block_slot, + beacon_block_root, + process_fn, + }, + }, + ReadyWork::RpcBlock(QueuedRpcBlock { + beacon_block_root: _, + process_fn, + ignore_fn: _, + }) => Self { + drop_during_sync: false, + work: Work::RpcBlock { process_fn }, + }, + ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock { process_fn }) => Self { + drop_during_sync: false, + work: Work::IgnoredRpcBlock { process_fn }, + }, + ReadyWork::Unaggregate(QueuedUnaggregate { + beacon_block_root: _, + process_fn, + }) => Self { + drop_during_sync: true, + work: Work::UnknownBlockAttestation { process_fn }, + }, + ReadyWork::Aggregate(QueuedAggregate { + process_fn, + beacon_block_root: _, + }) => Self { + drop_during_sync: true, + work: Work::UnknownBlockAggregate { process_fn }, + }, + ReadyWork::LightClientUpdate(QueuedLightClientUpdate { + parent_root, + process_fn, + }) => Self { + drop_during_sync: true, + work: Work::UnknownLightClientOptimisticUpdate { + parent_root, + process_fn, + }, + }, + ReadyWork::SamplingRequest(QueuedSamplingRequest { process_fn, .. }) => Self { + drop_during_sync: true, + work: Work::UnknownBlockSamplingRequest { process_fn }, + }, + ReadyWork::BackfillSync(QueuedBackfillBatch(process_fn)) => Self { + drop_during_sync: false, + work: Work::ChainSegmentBackfill(process_fn), + }, + } + } +} + +/// Unifies the different messages processed by the block delay queue. +enum InboundEvent { + /// A gossip block that was queued for later processing and is ready for import. + ReadyGossipBlock(QueuedGossipBlock), + /// A rpc block that was queued because the same gossip block was being imported + /// will now be retried for import. + 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), + /// A message sent to the `ReprocessQueue` + Msg(ReprocessQueueMessage), +} + +/// Manages scheduling works that need to be later re-processed. +struct ReprocessQueue { + /// Receiver of messages relevant to schedule works for reprocessing. + work_reprocessing_rx: Receiver, + /// Sender of works once they become ready + ready_work_tx: Sender, + + /* Queues */ + /// Queue to manage scheduled early blocks. + gossip_block_delay_queue: DelayQueue, + /// Queue to manage scheduled early blocks. + rpc_block_delay_queue: DelayQueue, + /// Queue to manage scheduled attestations. + attestations_delay_queue: DelayQueue, + /// Queue to manage scheduled light client updates. + lc_updates_delay_queue: DelayQueue, + /// Queue to manage scheduled sampling requests + sampling_requests_delay_queue: DelayQueue, + + /* Queued items */ + /// Queued blocks. + queued_gossip_block_roots: HashSet, + /// Queued aggregated attestations. + queued_aggregates: FnvHashMap, + /// Queued attestations. + queued_unaggregates: FnvHashMap, + /// Attestations (aggregated and unaggregated) per root. + awaiting_attestations_per_root: HashMap>, + /// Queued Light Client Updates. + queued_lc_updates: FnvHashMap, + /// Light Client Updates per parent_root. + awaiting_lc_updates_per_parent_root: HashMap>, + /// Queued sampling requests. + queued_sampling_requests: FnvHashMap, + /// Sampling requests per block root. + awaiting_sampling_requests_per_block_root: HashMap>, + /// Queued backfill batches + queued_backfill_batches: Vec, + + /* Aux */ + /// Next attestation id, used for both aggregated and unaggregated attestations + next_attestation: usize, + next_lc_update: usize, + next_sampling_request_update: usize, + early_block_debounce: TimeLatch, + rpc_block_debounce: TimeLatch, + attestation_delay_debounce: TimeLatch, + lc_update_delay_debounce: TimeLatch, + sampling_request_delay_debounce: TimeLatch, + next_backfill_batch_event: Option>>, + slot_clock: Arc, +} + +pub type QueuedLightClientUpdateId = usize; +pub type QueuedSamplingRequestId = usize; + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +enum QueuedAttestationId { + Aggregate(usize), + Unaggregate(usize), +} + +impl QueuedAggregate { + pub fn beacon_block_root(&self) -> &Hash256 { + &self.beacon_block_root + } +} + +impl QueuedUnaggregate { + pub fn beacon_block_root(&self) -> &Hash256 { + &self.beacon_block_root + } +} + +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 + // order fine-grained and separate from the logic of handling each message, which is nice. + + // Poll for expired blocks *before* we try to process new blocks. + // + // The sequential nature of blockchains means it is generally better to try and import all + // existing blocks before new ones. + match self.gossip_block_delay_queue.poll_expired(cx) { + Poll::Ready(Some(queued_block)) => { + return Poll::Ready(Some(InboundEvent::ReadyGossipBlock( + queued_block.into_inner(), + ))); + } + // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // will continue to get this result until something else is added into the queue. + Poll::Ready(None) | Poll::Pending => (), + } + + match self.rpc_block_delay_queue.poll_expired(cx) { + Poll::Ready(Some(queued_block)) => { + return Poll::Ready(Some(InboundEvent::ReadyRpcBlock(queued_block.into_inner()))); + } + // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // will continue to get this result until something else is added into the queue. + Poll::Ready(None) | Poll::Pending => (), + } + + match self.attestations_delay_queue.poll_expired(cx) { + Poll::Ready(Some(attestation_id)) => { + return Poll::Ready(Some(InboundEvent::ReadyAttestation( + attestation_id.into_inner(), + ))); + } + // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // will continue to get this result until something else is added into the queue. + Poll::Ready(None) | Poll::Pending => (), + } + + match self.lc_updates_delay_queue.poll_expired(cx) { + Poll::Ready(Some(lc_id)) => { + return Poll::Ready(Some(InboundEvent::ReadyLightClientUpdate( + lc_id.into_inner(), + ))); + } + // `Poll::Ready(None)` means that there are no more entries in the delay queue and we + // will continue to get this result until something else is added into the queue. + Poll::Ready(None) | Poll::Pending => (), + } + + if let Some(next_backfill_batch_event) = self.next_backfill_batch_event.as_mut() { + match next_backfill_batch_event.as_mut().poll(cx) { + Poll::Ready(_) => { + let maybe_batch = self.queued_backfill_batches.pop(); + self.recompute_next_backfill_batch_event(); + + if let Some(batch) = maybe_batch { + return Poll::Ready(Some(InboundEvent::ReadyBackfillSync(batch))); + } + } + Poll::Pending => (), + } + } + + // Last empty the messages channel. + match self.work_reprocessing_rx.poll_recv(cx) { + Poll::Ready(Some(message)) => return Poll::Ready(Some(InboundEvent::Msg(message))), + Poll::Ready(None) | Poll::Pending => {} + } + + Poll::Pending + } +} + +/// 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, + work_reprocessing_rx: Receiver, + executor: &TaskExecutor, + slot_clock: Arc, + maximum_gossip_clock_disparity: Duration, +) -> Result<(), String> { + // Sanity check + if ADDITIONAL_QUEUED_BLOCK_DELAY >= maximum_gossip_clock_disparity { + return Err("The block delay and gossip disparity don't match.".to_string()); + } + let mut queue = ReprocessQueue::new(ready_work_tx, work_reprocessing_rx, slot_clock); + + executor.spawn( + async move { + while let Some(msg) = queue.next().await { + queue.handle_message(msg); + } + + debug!(msg = "shutting down", "Re-process queue stopped",); + }, + TASK_NAME, + ); + Ok(()) +} + +impl ReprocessQueue { + fn new( + ready_work_tx: Sender, + work_reprocessing_rx: Receiver, + slot_clock: Arc, + ) -> Self { + ReprocessQueue { + work_reprocessing_rx, + ready_work_tx, + gossip_block_delay_queue: DelayQueue::new(), + rpc_block_delay_queue: DelayQueue::new(), + attestations_delay_queue: DelayQueue::new(), + lc_updates_delay_queue: DelayQueue::new(), + sampling_requests_delay_queue: <_>::default(), + queued_gossip_block_roots: HashSet::new(), + queued_lc_updates: FnvHashMap::default(), + queued_aggregates: FnvHashMap::default(), + queued_unaggregates: FnvHashMap::default(), + queued_sampling_requests: <_>::default(), + awaiting_attestations_per_root: HashMap::new(), + awaiting_lc_updates_per_parent_root: HashMap::new(), + awaiting_sampling_requests_per_block_root: <_>::default(), + queued_backfill_batches: Vec::new(), + next_attestation: 0, + next_lc_update: 0, + next_sampling_request_update: 0, + early_block_debounce: TimeLatch::default(), + rpc_block_debounce: TimeLatch::default(), + attestation_delay_debounce: TimeLatch::default(), + lc_update_delay_debounce: TimeLatch::default(), + sampling_request_delay_debounce: <_>::default(), + next_backfill_batch_event: None, + slot_clock, + } + } + + fn handle_message(&mut self, msg: InboundEvent) { + 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.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) { + return; + } + + if let Some(duration_till_slot) = self.slot_clock.duration_to_slot(block_slot) { + // Check to ensure this won't over-fill the queue. + if self.queued_gossip_block_roots.len() >= MAXIMUM_QUEUED_BLOCKS { + if self.early_block_debounce.elapsed() { + warn!( + queue_size = MAXIMUM_QUEUED_BLOCKS, + msg = "check system clock", + "Early blocks queue is full", + ); + } + // Drop the block. + return; + } + + self.queued_gossip_block_roots.insert(block_root); + // Queue the block until the start of the appropriate slot, plus + // `ADDITIONAL_QUEUED_BLOCK_DELAY`. + self.gossip_block_delay_queue.insert( + early_block, + duration_till_slot + ADDITIONAL_QUEUED_BLOCK_DELAY, + ); + } else { + // If there is no duration till the next slot, check to see if the slot + // has already arrived. If it has already arrived, send it out for + // immediate processing. + // + // If we can't read the slot or the slot hasn't arrived, simply drop the + // block. + // + // This logic is slightly awkward since `SlotClock::duration_to_slot` + // doesn't distinguish between a slot that has already arrived and an + // error reading the slot clock. + if let Some(now) = self.slot_clock.now() { + if block_slot <= now + && self + .ready_work_tx + .try_send(ReadyWork::Block(early_block)) + .is_err() + { + error!("Failed to send block"); + } + } + } + } + // A rpc block arrived for processing at the same time when a gossip block + // 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(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() { + warn!( + queue_size = MAXIMUM_QUEUED_BLOCKS, + msg = "check system clock", + "RPC blocks queue is full", + ); + } + // Return the block to the beacon processor signalling to + // ignore processing for this block + if self + .ready_work_tx + .try_send(ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock { + process_fn: rpc_block.ignore_fn, + })) + .is_err() + { + error!("Failed to send rpc block to beacon processor"); + } + return; + } + + // Queue the block for 1/3rd of a slot + self.rpc_block_delay_queue + .insert(rpc_block, QUEUED_RPC_BLOCK_DELAY); + } + InboundEvent::ReadyRpcBlock(queued_rpc_block) => { + debug!( + block_root = %queued_rpc_block.beacon_block_root, + "Sending rpc block for reprocessing" + ); + if self + .ready_work_tx + .try_send(ReadyWork::RpcBlock(queued_rpc_block)) + .is_err() + { + error!("Failed to send rpc block to beacon processor"); + } + } + InboundEvent::Msg(UnknownBlockAggregate(queued_aggregate)) => { + if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { + if self.attestation_delay_debounce.elapsed() { + error!( + queue_size = MAXIMUM_QUEUED_ATTESTATIONS, + msg = "check system clock", + "Aggregate attestation delay queue is full" + ); + } + // Drop the attestation. + return; + } + + let att_id = QueuedAttestationId::Aggregate(self.next_attestation); + + // Register the delay. + let delay_key = self + .attestations_delay_queue + .insert(att_id, QUEUED_ATTESTATION_DELAY); + + // Register this attestation for the corresponding root. + self.awaiting_attestations_per_root + .entry(*queued_aggregate.beacon_block_root()) + .or_default() + .push(att_id); + + // Store the attestation and its info. + self.queued_aggregates + .insert(self.next_attestation, (queued_aggregate, delay_key)); + + self.next_attestation += 1; + } + InboundEvent::Msg(UnknownBlockUnaggregate(queued_unaggregate)) => { + if self.attestations_delay_queue.len() >= MAXIMUM_QUEUED_ATTESTATIONS { + if self.attestation_delay_debounce.elapsed() { + error!( + queue_size = MAXIMUM_QUEUED_ATTESTATIONS, + msg = "check system clock", + "Attestation delay queue is full" + ); + } + // Drop the attestation. + return; + } + + let att_id = QueuedAttestationId::Unaggregate(self.next_attestation); + + // Register the delay. + let delay_key = self + .attestations_delay_queue + .insert(att_id, QUEUED_ATTESTATION_DELAY); + + // Register this attestation for the corresponding root. + self.awaiting_attestations_per_root + .entry(*queued_unaggregate.beacon_block_root()) + .or_default() + .push(att_id); + + // Store the attestation and its info. + self.queued_unaggregates + .insert(self.next_attestation, (queued_unaggregate, delay_key)); + + self.next_attestation += 1; + } + InboundEvent::Msg(UnknownLightClientOptimisticUpdate( + queued_light_client_optimistic_update, + )) => { + if self.lc_updates_delay_queue.len() >= MAXIMUM_QUEUED_LIGHT_CLIENT_UPDATES { + if self.lc_update_delay_debounce.elapsed() { + error!( + queue_size = MAXIMUM_QUEUED_LIGHT_CLIENT_UPDATES, + msg = "check system clock", + "Light client updates delay queue is full" + ); + } + // Drop the light client update. + return; + } + + let lc_id: QueuedLightClientUpdateId = self.next_lc_update; + + // Register the delay. + let delay_key = self + .lc_updates_delay_queue + .insert(lc_id, QUEUED_LIGHT_CLIENT_UPDATE_DELAY); + + // Register the light client update for the corresponding root. + self.awaiting_lc_updates_per_parent_root + .entry(queued_light_client_optimistic_update.parent_root) + .or_default() + .push(lc_id); + + // Store the light client update and its info. + self.queued_lc_updates.insert( + self.next_lc_update, + (queued_light_client_optimistic_update, delay_key), + ); + + self.next_lc_update += 1; + } + InboundEvent::Msg(UnknownBlockSamplingRequest(queued_sampling_request)) => { + if self.sampling_requests_delay_queue.len() >= MAXIMUM_QUEUED_SAMPLING_REQUESTS { + if self.sampling_request_delay_debounce.elapsed() { + error!( + queue_size = MAXIMUM_QUEUED_SAMPLING_REQUESTS, + "Sampling requests delay queue is full", + ); + } + // Drop the inbound message. + return; + } + + let id: QueuedSamplingRequestId = self.next_sampling_request_update; + self.next_sampling_request_update += 1; + + // Register the delay. + let delay_key = self + .sampling_requests_delay_queue + .insert(id, QUEUED_SAMPLING_REQUESTS_DELAY); + + self.awaiting_sampling_requests_per_block_root + .entry(queued_sampling_request.beacon_block_root) + .or_default() + .push(id); + + self.queued_sampling_requests + .insert(id, (queued_sampling_request, delay_key)); + } + InboundEvent::Msg(BlockImported { + block_root, + parent_root, + }) => { + // Unqueue the attestations we have for this root, if any. + if let Some(queued_ids) = self.awaiting_attestations_per_root.remove(&block_root) { + let mut sent_count = 0; + let mut failed_to_send_count = 0; + + for id in queued_ids { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_ATTESTATIONS, + ); + + if let Some((work, delay_key)) = match id { + QueuedAttestationId::Aggregate(id) => self + .queued_aggregates + .remove(&id) + .map(|(aggregate, delay_key)| { + (ReadyWork::Aggregate(aggregate), delay_key) + }), + QueuedAttestationId::Unaggregate(id) => self + .queued_unaggregates + .remove(&id) + .map(|(unaggregate, delay_key)| { + (ReadyWork::Unaggregate(unaggregate), delay_key) + }), + } { + // Remove the delay. + self.attestations_delay_queue.remove(&delay_key); + + // Send the work. + if self.ready_work_tx.try_send(work).is_err() { + failed_to_send_count += 1; + } else { + sent_count += 1; + } + } else { + // There is a mismatch between the attestation ids registered for this + // root and the queued attestations. This should never happen. + error!( + ?block_root, + att_id = ?id, + "Unknown queued attestation for block root", + ); + } + } + + if failed_to_send_count > 0 { + error!( + hint = "system may be overloaded", + ?parent_root, + ?block_root, + %failed_to_send_count, + %sent_count, + "Ignored scheduled attestation(s) for block" + ); + } + } + // Unqueue the sampling requests we have for this root, if any. + if let Some(queued_ids) = self + .awaiting_sampling_requests_per_block_root + .remove(&block_root) + { + let mut sent_count = 0; + let mut failed_to_send_count = 0; + + for id in queued_ids { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_SAMPLING_REQUESTS, + ); + + if let Some((queued, delay_key)) = self.queued_sampling_requests.remove(&id) + { + // Remove the delay. + self.sampling_requests_delay_queue.remove(&delay_key); + + // Send the work. + let work = ReadyWork::SamplingRequest(queued); + + if self.ready_work_tx.try_send(work).is_err() { + failed_to_send_count += 1; + } else { + sent_count += 1; + } + } else { + // This should never happen. + error!(?block_root, ?id, "Unknown sampling request for block root"); + } + } + + if failed_to_send_count > 0 { + error!( + ?block_root, + failed_to_send_count, + sent_count, + hint = "system may be overloaded", + "Ignored scheduled sampling requests for block", + ); + } + } + } + InboundEvent::Msg(NewLightClientOptimisticUpdate { parent_root }) => { + // Unqueue the light client optimistic updates we have for this root, if any. + if let Some(queued_lc_id) = self + .awaiting_lc_updates_per_parent_root + .remove(&parent_root) + { + debug!( + %parent_root, + count = queued_lc_id.len(), + "Dequeuing light client optimistic updates", + ); + + for lc_id in queued_lc_id { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_OPTIMISTIC_UPDATES, + ); + if let Some((work, delay_key)) = self.queued_lc_updates.remove(&lc_id).map( + |(light_client_optimistic_update, delay_key)| { + ( + ReadyWork::LightClientUpdate(light_client_optimistic_update), + delay_key, + ) + }, + ) { + // Remove the delay + self.lc_updates_delay_queue.remove(&delay_key); + + // Send the work + match self.ready_work_tx.try_send(work) { + Ok(_) => trace!("reprocessing light client update sent"), + Err(_) => error!("Failed to send scheduled light client update"), + } + } else { + // There is a mismatch between the light client update ids registered for this + // root and the queued light client updates. This should never happen. + error!( + ?parent_root, + ?lc_id, + "Unknown queued light client update for parent root" + ); + } + } + } + } + InboundEvent::Msg(BackfillSync(queued_backfill_batch)) => { + self.queued_backfill_batches + .insert(0, queued_backfill_batch); + // only recompute if there is no `next_backfill_batch_event` already scheduled + if self.next_backfill_batch_event.is_none() { + self.recompute_next_backfill_batch_event(); + } + } + // A block that was queued for later processing is now ready to be processed. + InboundEvent::ReadyGossipBlock(ready_block) => { + 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 + // program works, but still process the block anyway. + error!(?block_root, "Unknown block in delay queue"); + } + + if self + .ready_work_tx + .try_send(ReadyWork::Block(ready_block)) + .is_err() + { + error!("Failed to pop queued block"); + } + } + InboundEvent::ReadyAttestation(queued_id) => { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS, + ); + + if let Some((root, work)) = match queued_id { + QueuedAttestationId::Aggregate(id) => { + self.queued_aggregates + .remove(&id) + .map(|(aggregate, _delay_key)| { + ( + *aggregate.beacon_block_root(), + ReadyWork::Aggregate(aggregate), + ) + }) + } + QueuedAttestationId::Unaggregate(id) => self + .queued_unaggregates + .remove(&id) + .map(|(unaggregate, _delay_key)| { + ( + *unaggregate.beacon_block_root(), + ReadyWork::Unaggregate(unaggregate), + ) + }), + } { + if self.ready_work_tx.try_send(work).is_err() { + error!( + block_root = ?root, + hint = "system may be overloaded", + "Ignored scheduled attestation" + ); + } + + if let Some(queued_atts) = self.awaiting_attestations_per_root.get_mut(&root) { + if let Some(index) = queued_atts.iter().position(|&id| id == queued_id) { + queued_atts.swap_remove(index); + } + } + } + } + InboundEvent::ReadyLightClientUpdate(queued_id) => { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_OPTIMISTIC_UPDATES, + ); + + if let Some((parent_root, work)) = self.queued_lc_updates.remove(&queued_id).map( + |(queued_lc_update, _delay_key)| { + ( + queued_lc_update.parent_root, + ReadyWork::LightClientUpdate(queued_lc_update), + ) + }, + ) { + if self.ready_work_tx.try_send(work).is_err() { + error!("Failed to send scheduled light client optimistic update"); + } + + if let Some(queued_lc_updates) = self + .awaiting_lc_updates_per_parent_root + .get_mut(&parent_root) + { + if let Some(index) = + queued_lc_updates.iter().position(|&id| id == queued_id) + { + queued_lc_updates.swap_remove(index); + } + } + } + } + InboundEvent::ReadyBackfillSync(queued_backfill_batch) => { + let millis_from_slot_start = self + .slot_clock + .millis_from_current_slot_start() + .map_or("null".to_string(), |duration| { + duration.as_millis().to_string() + }); + + debug!(millis_from_slot_start, "Sending scheduled backfill work"); + + match self + .ready_work_tx + .try_send(ReadyWork::BackfillSync(queued_backfill_batch)) + { + // 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!( + info = "sending work back to queue", + "Failed to send scheduled backfill work" + ); + self.queued_backfill_batches.insert(0, batch); + + // only recompute if there is no `next_backfill_batch_event` already scheduled + if self.next_backfill_batch_event.is_none() { + self.recompute_next_backfill_batch_event(); + } + } + // The message was not sent and we didn't get the correct + // return result. This is a logic error. + _ => crit!("Unexpected return from try_send error"), + } + } + } + + metrics::set_gauge_vec( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, + &[GOSSIP_BLOCKS], + self.gossip_block_delay_queue.len() as i64, + ); + metrics::set_gauge_vec( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, + &[RPC_BLOCKS], + self.rpc_block_delay_queue.len() as i64, + ); + metrics::set_gauge_vec( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, + &[ATTESTATIONS], + self.attestations_delay_queue.len() as i64, + ); + metrics::set_gauge_vec( + &metrics::BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL, + &[LIGHT_CLIENT_UPDATES], + self.lc_updates_delay_queue.len() as i64, + ); + } + + fn recompute_next_backfill_batch_event(&mut self) { + // 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), + ))); + } else { + self.next_backfill_batch_event = None + } + } + + /// 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: &S) -> Duration { + let slot_duration = slot_clock.slot_duration(); + slot_clock + .millis_from_current_slot_start() + .and_then(|duration_from_slot_start| { + BACKFILL_SCHEDULE_IN_SLOT + .into_iter() + // Convert fractions to seconds from slot start. + .map(|(multiplier, divisor)| (slot_duration / divisor) * multiplier) + .find_or_first(|&event_duration_from_slot_start| { + event_duration_from_slot_start > duration_from_slot_start + }) + .map(|next_event_time| { + if duration_from_slot_start >= next_event_time { + // event is in the next slot, add duration to next slot + let duration_to_next_slot = slot_duration - duration_from_slot_start; + duration_to_next_slot + next_event_time + } else { + next_event_time - duration_from_slot_start + } + }) + }) + // If we can't read the slot clock, just wait another slot. + .unwrap_or(slot_duration) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use slot_clock::{ManualSlotClock, TestingSlotClock}; + use std::ops::Add; + use std::sync::Arc; + use task_executor::test_utils::TestRuntime; + use types::Slot; + + #[test] + fn backfill_processing_schedule_calculation() { + let slot_duration = Duration::from_secs(12); + let slot_clock = TestingSlotClock::new(Slot::new(0), Duration::from_secs(0), slot_duration); + let current_slot_start = slot_clock.start_of(Slot::new(100)).unwrap(); + slot_clock.set_current_time(current_slot_start); + + let event_times = BACKFILL_SCHEDULE_IN_SLOT + .map(|(multiplier, divisor)| (slot_duration / divisor) * multiplier); + + for &event_duration_from_slot_start in event_times.iter() { + let duration_to_next_event = + ReprocessQueue::::duration_until_next_backfill_batch_event( + &slot_clock, + ); + + let current_time = slot_clock.millis_from_current_slot_start().unwrap(); + + assert_eq!( + duration_to_next_event, + event_duration_from_slot_start - current_time + ); + + slot_clock.set_current_time(current_slot_start + event_duration_from_slot_start) + } + + // 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( + &slot_clock, + ); + assert_eq!( + duration_to_next_event, + duration_to_next_slot + event_times[0] + ); + } + + // Regression test for issue #5504. + // See: https://github.com/sigp/lighthouse/issues/5504#issuecomment-2050930045 + #[tokio::test] + async fn backfill_schedule_failed_should_reschedule() { + let runtime = TestRuntime::default(); + let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(1); + let (ready_work_tx, mut ready_work_rx) = mpsc::channel(1); + let slot_duration = 12; + let slot_clock = Arc::new(testing_slot_clock(slot_duration)); + + spawn_reprocess_scheduler( + ready_work_tx.clone(), + work_reprocessing_rx, + &runtime.task_executor, + slot_clock.clone(), + Duration::from_millis(500), + ) + .unwrap(); + + // Pause time so it only advances manually + tokio::time::pause(); + + // Send some random work to `ready_work_tx` to fill up the capacity first. + ready_work_tx + .try_send(ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock { + process_fn: Box::new(|| {}), + })) + .unwrap(); + + // Now queue a backfill sync batch. + work_reprocessing_tx + .try_send(ReprocessQueueMessage::BackfillSync(QueuedBackfillBatch( + Box::pin(async {}), + ))) + .unwrap(); + tokio::task::yield_now().await; + + // Advance the time by more than 1/2 the slot to trigger a scheduled backfill batch to be sent. + // This should fail as the `ready_work` channel is at capacity, and it should be rescheduled. + let duration_to_next_event = + ReprocessQueue::duration_until_next_backfill_batch_event(slot_clock.as_ref()); + let one_ms = Duration::from_millis(1); + advance_time(&slot_clock, duration_to_next_event.add(one_ms)).await; + + // Now drain the `ready_work` channel. + assert!(matches!( + ready_work_rx.try_recv(), + Ok(ReadyWork::IgnoredRpcBlock { .. }) + )); + assert!(ready_work_rx.try_recv().is_err()); + + // Advance time again, and assert that the re-scheduled batch is successfully sent. + let duration_to_next_event = + ReprocessQueue::duration_until_next_backfill_batch_event(slot_clock.as_ref()); + advance_time(&slot_clock, duration_to_next_event.add(one_ms)).await; + assert!(matches!( + ready_work_rx.try_recv(), + Ok(ReadyWork::BackfillSync { .. }) + )); + } + + /// Advances slot clock and test clock time by the same duration. + async fn advance_time(slot_clock: &ManualSlotClock, duration: Duration) { + slot_clock.advance_time(duration); + tokio::time::advance(duration).await; + // NOTE: The `tokio::time::advance` fn actually calls `yield_now()` after advancing the + // clock. Why do we need an extra `yield_now`? + tokio::task::yield_now().await; + } + + fn testing_slot_clock(slot_duration: u64) -> ManualSlotClock { + TestingSlotClock::new( + Slot::new(0), + Duration::from_secs(0), + Duration::from_secs(slot_duration), + ) + } +} diff --git a/beacon_node/client/src/builder.rs b/beacon_node/client/src/builder.rs index 3cb7b33aae2..0ca615b2ccb 100644 --- a/beacon_node/client/src/builder.rs +++ b/beacon_node/client/src/builder.rs @@ -18,8 +18,8 @@ use beacon_chain::{ BeaconChain, BeaconChainTypes, Eth1ChainBackend, MigratorConfig, ServerSentEventHandler, }; use beacon_chain::{Kzg, LightClientProducerEvent}; +use beacon_processor::BeaconProcessorConfig; use beacon_processor::{BeaconProcessor, BeaconProcessorChannels}; -use beacon_processor::{BeaconProcessorConfig, BeaconProcessorQueueLengths}; use environment::RuntimeContext; use eth1::{Config as Eth1Config, Service as Eth1Service}; use eth2::{ @@ -544,7 +544,6 @@ where network_senders: None, network_globals: None, beacon_processor_send: None, - beacon_processor_reprocess_send: None, eth1_service: Some(genesis_service.eth1_service.clone()), sse_logging_components: runtime_context.sse_logging_components.clone(), }); @@ -636,7 +635,6 @@ where context.executor, libp2p_registry.as_mut(), beacon_processor_channels.beacon_processor_tx.clone(), - beacon_processor_channels.work_reprocessing_tx.clone(), ) .await .map_err(|e| format!("Failed to start network: {:?}", e))?; @@ -775,9 +773,6 @@ where network_globals: self.network_globals.clone(), eth1_service: self.eth1_service.clone(), beacon_processor_send: Some(beacon_processor_channels.beacon_processor_tx.clone()), - beacon_processor_reprocess_send: Some( - beacon_processor_channels.work_reprocessing_tx.clone(), - ), sse_logging_components: runtime_context.sse_logging_components.clone(), }); @@ -833,6 +828,11 @@ 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()); + let beacon_state = &beacon_chain + .canonical_head + .cached_head() + .snapshot + .beacon_state; BeaconProcessor { network_globals: network_globals.clone(), executor: beacon_processor_context.executor.clone(), @@ -840,20 +840,11 @@ where config: beacon_processor_config, } .spawn_manager( + beacon_state, beacon_processor_channels.beacon_processor_rx, - beacon_processor_channels.work_reprocessing_tx.clone(), - beacon_processor_channels.work_reprocessing_rx, None, beacon_chain.slot_clock.clone(), - beacon_chain.spec.maximum_gossip_clock_disparity(), - BeaconProcessorQueueLengths::from_state( - &beacon_chain - .canonical_head - .cached_head() - .snapshot - .beacon_state, - &beacon_chain.spec, - )?, + &beacon_chain.spec, )?; } @@ -916,7 +907,7 @@ where compute_light_client_updates( &inner_chain, light_client_server_rv, - beacon_processor_channels.work_reprocessing_tx, + beacon_processor_channels.beacon_processor_tx, ) .await }, diff --git a/beacon_node/client/src/compute_light_client_updates.rs b/beacon_node/client/src/compute_light_client_updates.rs index fab284c4285..d82fc85fd05 100644 --- a/beacon_node/client/src/compute_light_client_updates.rs +++ b/beacon_node/client/src/compute_light_client_updates.rs @@ -1,8 +1,7 @@ use beacon_chain::{BeaconChain, BeaconChainTypes, LightClientProducerEvent}; -use beacon_processor::work_reprocessing_queue::ReprocessQueueMessage; +use beacon_processor::{BeaconProcessorSend, ReprocessQueueMessage, Work, WorkEvent}; use futures::channel::mpsc::Receiver; use futures::StreamExt; -use tokio::sync::mpsc::Sender; use tracing::error; // Each `LightClientProducerEvent` is ~200 bytes. With the light_client server producing only recent @@ -14,7 +13,7 @@ pub(crate) const LIGHT_CLIENT_SERVER_CHANNEL_CAPACITY: usize = 32; pub async fn compute_light_client_updates( chain: &BeaconChain, mut light_client_server_rv: Receiver>, - reprocess_tx: Sender, + beacon_processor_send: BeaconProcessorSend, ) { // Should only receive events for recent blocks, import_block filters by blocks close to clock. // @@ -31,7 +30,14 @@ pub async fn compute_light_client_updates( }); let msg = ReprocessQueueMessage::NewLightClientOptimisticUpdate { parent_root }; - if reprocess_tx.try_send(msg).is_err() { + + if beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: true, + work: Work::Reprocess(msg), + }) + .is_err() + { error!(%parent_root,"Failed to inform light client update") }; } diff --git a/beacon_node/http_api/src/lib.rs b/beacon_node/http_api/src/lib.rs index 386d9fe33aa..8b2ab4fed08 100644 --- a/beacon_node/http_api/src/lib.rs +++ b/beacon_node/http_api/src/lib.rs @@ -40,7 +40,7 @@ use beacon_chain::{ validator_monitor::timestamp_now, AttestationError as AttnError, BeaconChain, BeaconChainError, BeaconChainTypes, WhenSlotSkipped, }; -use beacon_processor::{work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend}; +use beacon_processor::BeaconProcessorSend; pub use block_id::BlockId; use builder_states::get_next_withdrawals; use bytes::Bytes; @@ -133,7 +133,6 @@ pub struct Context { pub network_senders: Option>, pub network_globals: Option>>, pub beacon_processor_send: Option>, - pub beacon_processor_reprocess_send: Option>, pub eth1_service: Option, pub sse_logging_components: Option, } @@ -557,12 +556,6 @@ pub fn serve( .filter(|_| config.enable_beacon_processor); let task_spawner_filter = warp::any().map(move || TaskSpawner::new(beacon_processor_send.clone())); - let beacon_processor_reprocess_send = ctx - .beacon_processor_reprocess_send - .clone() - .filter(|_| config.enable_beacon_processor); - let reprocess_send_filter = warp::any().map(move || beacon_processor_reprocess_send.clone()); - let duplicate_block_status_code = ctx.config.duplicate_block_status_code; /* @@ -1944,20 +1937,18 @@ pub fn serve( .and(warp::path::end()) .and(warp_utils::json::json()) .and(network_tx_filter.clone()) - .and(reprocess_send_filter.clone()) .then( |task_spawner: TaskSpawner, chain: Arc>, attestations: Vec>, - network_tx: UnboundedSender>, - reprocess_tx: Option>| async move { + network_tx: UnboundedSender>| async move { let attestations = attestations.into_iter().map(Either::Left).collect(); let result = crate::publish_attestations::publish_attestations( task_spawner, chain, attestations, network_tx, - reprocess_tx, + true, ) .await .map(|()| warp::reply::json(&())); @@ -1972,14 +1963,12 @@ pub fn serve( .and(warp_utils::json::json::()) .and(optional_consensus_version_header_filter) .and(network_tx_filter.clone()) - .and(reprocess_send_filter.clone()) .then( |task_spawner: TaskSpawner, chain: Arc>, payload: Value, fork_name: Option, - network_tx: UnboundedSender>, - reprocess_tx: Option>| async move { + network_tx: UnboundedSender>| async move { let attestations = match crate::publish_attestations::deserialize_attestation_payload::( payload, fork_name, @@ -2005,7 +1994,7 @@ pub fn serve( chain, attestations, network_tx, - reprocess_tx, + true, // TODO(beacon-processor) always allow reprocessing? ) .await .map(|()| warp::reply::json(&())); diff --git a/beacon_node/http_api/src/publish_attestations.rs b/beacon_node/http_api/src/publish_attestations.rs index cd5e912bdf4..b6af3c6c0e9 100644 --- a/beacon_node/http_api/src/publish_attestations.rs +++ b/beacon_node/http_api/src/publish_attestations.rs @@ -39,7 +39,7 @@ use beacon_chain::{ single_attestation::single_attestation_to_attestation, validator_monitor::timestamp_now, AttestationError, BeaconChain, BeaconChainError, BeaconChainTypes, }; -use beacon_processor::work_reprocessing_queue::{QueuedUnaggregate, ReprocessQueueMessage}; +use beacon_processor::{QueuedUnaggregate, ReprocessQueueMessage, Work, WorkEvent}; use either::Either; use eth2::types::Failure; use lighthouse_network::PubsubMessage; @@ -48,10 +48,7 @@ use serde_json::Value; use std::borrow::Cow; use std::sync::Arc; use std::time::Duration; -use tokio::sync::{ - mpsc::{Sender, UnboundedSender}, - oneshot, -}; +use tokio::sync::{mpsc::UnboundedSender, oneshot}; use tracing::{debug, error, warn}; use types::{Attestation, EthSpec, ForkName, SingleAttestation}; @@ -224,7 +221,7 @@ pub async fn publish_attestations( chain: Arc>, attestations: Vec, SingleAttestation>>, network_tx: UnboundedSender>, - reprocess_send: Option>, + allow_reprocess: bool, ) -> Result<(), warp::Rejection> { // Collect metadata about attestations which we'll use to report failures. We need to // move the `attestations` vec into the blocking task, so this small overhead is unavoidable. @@ -239,6 +236,7 @@ pub async fn publish_attestations( // Gossip validate and publish attestations that can be immediately processed. let seen_timestamp = timestamp_now(); let mut prelim_results = task_spawner + .clone() .blocking_task(Priority::P0, move || { Ok(attestations .into_iter() @@ -253,7 +251,7 @@ pub async fn publish_attestations( Err(Error::Validation(AttestationError::UnknownHeadBlock { beacon_block_root, })) => { - let Some(reprocess_tx) = &reprocess_send else { + if !allow_reprocess { return PublishAttestationResult::Failure(Error::ReprocessDisabled); }; // Re-process. @@ -277,7 +275,13 @@ pub async fn publish_attestations( beacon_block_root, process_fn: Box::new(reprocess_fn), }); - if reprocess_tx.try_send(reprocess_msg).is_err() { + if task_spawner + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(reprocess_msg), + }) + .is_err() + { PublishAttestationResult::Failure(Error::ReprocessFull) } else { PublishAttestationResult::Reprocessing(rx) diff --git a/beacon_node/http_api/src/task_spawner.rs b/beacon_node/http_api/src/task_spawner.rs index a679b294f65..834cd29971f 100644 --- a/beacon_node/http_api/src/task_spawner.rs +++ b/beacon_node/http_api/src/task_spawner.rs @@ -30,6 +30,7 @@ impl Priority { } /// Spawns tasks on the `BeaconProcessor` or directly on the tokio executor. +#[derive(Clone)] pub struct TaskSpawner { /// Used to send tasks to the `BeaconProcessor`. The tokio executor will be /// used if this is `None`. @@ -155,6 +156,32 @@ impl TaskSpawner { .and_then(|x| x) } } + + pub fn try_send(&self, work_event: WorkEvent) -> Result<(), warp::Rejection> { + if let Some(beacon_processor_send) = &self.beacon_processor_send { + let error_message = match beacon_processor_send.try_send(work_event) { + Ok(()) => None, + Err(TrySendError::Full(_)) => { + Some("The task was dropped. The server is overloaded.") + } + Err(TrySendError::Closed(_)) => { + Some("The task was dropped. The server is shutting down.") + } + }; + + if let Some(error_message) = error_message { + return Err(warp_utils::reject::custom_server_error( + error_message.to_string(), + )); + }; + + Ok(()) + } else { + Err(warp_utils::reject::custom_server_error( + "The beacon processor is unavailable".to_string(), + )) + } + } } /// Send a task to the beacon processor and await execution. diff --git a/beacon_node/http_api/src/test_utils.rs b/beacon_node/http_api/src/test_utils.rs index f78a361dad3..1339bcacc94 100644 --- a/beacon_node/http_api/src/test_utils.rs +++ b/beacon_node/http_api/src/test_utils.rs @@ -3,9 +3,7 @@ use beacon_chain::{ test_utils::{BeaconChainHarness, BoxedMutator, Builder, EphemeralHarnessType}, BeaconChain, BeaconChainTypes, }; -use beacon_processor::{ - BeaconProcessor, BeaconProcessorChannels, BeaconProcessorConfig, BeaconProcessorQueueLengths, -}; +use beacon_processor::{BeaconProcessor, BeaconProcessorChannels, BeaconProcessorConfig}; use directory::DEFAULT_ROOT_DIR; use eth2::{BeaconNodeHttpClient, Timeouts}; use lighthouse_network::rpc::methods::MetaDataV3; @@ -194,19 +192,22 @@ pub async fn create_api_server_with_config( // The number of workers must be greater than one. Tests which use the // builder workflow sometimes require an internal HTTP request in order // to fulfill an already in-flight HTTP request, therefore having only - // one worker will result in a deadlock. - max_workers: 2, + // one worker will result in a deadlock. Since the introduction of the + // earliest deadline priority queue, the beacon processor now requires more + // than one worker to be available for "non-priority" work events. Keeping the + // number of workers to a value greater than two prevents test failures due to + // timeouts + max_workers: 3, ..BeaconProcessorConfig::default() }; let BeaconProcessorChannels { beacon_processor_tx, beacon_processor_rx, - work_reprocessing_tx, - work_reprocessing_rx, } = BeaconProcessorChannels::new(&beacon_processor_config); + let beacon_state = &chain.canonical_head.cached_head().snapshot.beacon_state; + let beacon_processor_send = beacon_processor_tx; - let reprocess_send = work_reprocessing_tx.clone(); BeaconProcessor { network_globals: network_globals.clone(), executor: test_runtime.task_executor.clone(), @@ -214,17 +215,11 @@ pub async fn create_api_server_with_config( config: beacon_processor_config, } .spawn_manager( + beacon_state, beacon_processor_rx, - work_reprocessing_tx, - work_reprocessing_rx, None, chain.slot_clock.clone(), - chain.spec.maximum_gossip_clock_disparity(), - BeaconProcessorQueueLengths::from_state( - &chain.canonical_head.cached_head().snapshot.beacon_state, - &chain.spec, - ) - .unwrap(), + &chain.spec, ) .unwrap(); @@ -241,7 +236,6 @@ pub async fn create_api_server_with_config( network_senders: Some(network_senders), network_globals: Some(network_globals), beacon_processor_send: Some(beacon_processor_send), - beacon_processor_reprocess_send: Some(reprocess_send), eth1_service: Some(eth1_service), sse_logging_components: None, }); diff --git a/beacon_node/http_api/tests/interactive_tests.rs b/beacon_node/http_api/tests/interactive_tests.rs index bb3086945bc..e0db71c6a8f 100644 --- a/beacon_node/http_api/tests/interactive_tests.rs +++ b/beacon_node/http_api/tests/interactive_tests.rs @@ -4,7 +4,8 @@ use beacon_chain::{ test_utils::{AttestationStrategy, BlockStrategy, LightClientStrategy, SyncCommitteeStrategy}, ChainConfig, }; -use beacon_processor::work_reprocessing_queue::ReprocessQueueMessage; +use beacon_processor::ReprocessQueueMessage; +use beacon_processor::{Work, WorkEvent}; use either::Either; use eth2::types::ProduceBlockV3Response; use eth2::types::{DepositContractData, StateId}; @@ -945,14 +946,16 @@ async fn queue_attestations_from_http() { .unwrap(); tester .ctx - .beacon_processor_reprocess_send + .beacon_processor_send .as_ref() .unwrap() - .send(ReprocessQueueMessage::BlockImported { - block_root, - parent_root, + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(ReprocessQueueMessage::BlockImported { + block_root, + parent_root, + }), }) - .await .unwrap(); attestation_future.await.unwrap(); diff --git a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs index d61ea583772..9398570a23f 100644 --- a/beacon_node/network/src/network_beacon_processor/gossip_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/gossip_methods.rs @@ -31,7 +31,6 @@ use std::path::PathBuf; use std::sync::Arc; use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; use store::hot_cold_store::HotColdDBError; -use tokio::sync::mpsc; use tracing::{debug, error, info, trace, warn}; use types::{ beacon_block::BlockImportSource, Attestation, AttestationData, AttestationRef, @@ -43,11 +42,9 @@ use types::{ }; use beacon_processor::{ - work_reprocessing_queue::{ - QueuedAggregate, QueuedGossipBlock, QueuedLightClientUpdate, QueuedUnaggregate, - ReprocessQueueMessage, - }, - DuplicateCache, GossipAggregatePackage, GossipAttestationBatch, + DuplicateCache, GossipAggregatePackage, GossipAttestationBatch, QueuedAggregate, + QueuedGossipBlock, QueuedLightClientUpdate, QueuedUnaggregate, ReprocessQueueMessage, Work, + WorkEvent, }; /// Set to `true` to introduce stricter penalties for peers who send some types of late consensus @@ -211,7 +208,7 @@ impl NetworkBeaconProcessor { attestation: Box>, subnet_id: SubnetId, should_import: bool, - reprocess_tx: Option>, + allow_reprocess: bool, seen_timestamp: Duration, ) { let result = match self @@ -230,16 +227,16 @@ impl NetworkBeaconProcessor { message_id, peer_id, subnet_id, - reprocess_tx, should_import, seen_timestamp, + allow_reprocess, ); } pub fn process_gossip_attestation_batch( self: Arc, packages: GossipAttestationBatch, - reprocess_tx: Option>, + allow_reprocess: bool, ) { let attestations_and_subnets = packages .iter() @@ -295,9 +292,9 @@ impl NetworkBeaconProcessor { package.message_id, package.peer_id, package.subnet_id, - reprocess_tx.clone(), package.should_import, package.seen_timestamp, + allow_reprocess, ); } } @@ -311,9 +308,9 @@ impl NetworkBeaconProcessor { message_id: MessageId, peer_id: PeerId, subnet_id: SubnetId, - reprocess_tx: Option>, should_import: bool, seen_timestamp: Duration, + allow_reprocess: bool, ) { match result { Ok(verified_attestation) => { @@ -395,8 +392,8 @@ impl NetworkBeaconProcessor { should_import, seen_timestamp, }, - reprocess_tx, error, + allow_reprocess, seen_timestamp, ); } @@ -416,7 +413,6 @@ impl NetworkBeaconProcessor { single_attestation: Box, subnet_id: SubnetId, should_import: bool, - reprocess_tx: Option>, seen_timestamp: Duration, ) { let conversion_result = self.chain.with_committee_cache( @@ -469,41 +465,45 @@ impl NetworkBeaconProcessor { // Outermost error (from `with_committee_cache`) indicating that the block is not known // and that this conversion should be retried. Err(BeaconChainError::MissingBeaconBlock(beacon_block_root)) => { - if let Some(sender) = reprocess_tx { - metrics::inc_counter( - &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL, - ); - // We don't know the block, get the sync manager to handle the block lookup, and - // send the attestation to be scheduled for re-processing. - self.sync_tx - .send(SyncMessage::UnknownBlockHashFromAttestation( - peer_id, - beacon_block_root, - )) - .unwrap_or_else(|_| { - warn!(msg = "UnknownBlockHash", "Failed to send to sync service") - }); - let processor = self.clone(); - // Do not allow this attestation to be re-processed beyond this point. - let reprocess_msg = - ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { - beacon_block_root, - process_fn: Box::new(move || { - processor.process_gossip_attestation_to_convert( - message_id, - peer_id, - single_attestation, - subnet_id, - should_import, - None, - seen_timestamp, - ) - }), - }); - if sender.try_send(reprocess_msg).is_err() { - error!("Failed to send attestation for re-processing") - } - } else { + metrics::inc_counter( + &metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL, + ); + // We don't know the block, get the sync manager to handle the block lookup, and + // send the attestation to be scheduled for re-processing. + self.sync_tx + .send(SyncMessage::UnknownBlockHashFromAttestation( + peer_id, + beacon_block_root, + )) + .unwrap_or_else(|_| { + warn!(msg = "UnknownBlockHash", "Failed to send to sync service") + }); + let processor = self.clone(); + let msg_id = message_id.clone(); + // Do not allow this attestation to be re-processed beyond this point. + let reprocess_msg = + ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate { + beacon_block_root, + process_fn: Box::new(move || { + processor.process_gossip_attestation_to_convert( + msg_id, + peer_id, + single_attestation, + subnet_id, + should_import, + seen_timestamp, + ) + }), + }); + if self + .beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(reprocess_msg), + }) + .is_err() + { + error!("Failed to send attestation for re-processing"); // We shouldn't make any further attempts to process this attestation. // // Don't downscore the peer since it's not clear if we requested this head @@ -523,8 +523,8 @@ impl NetworkBeaconProcessor { FailedAtt::SingleUnaggregate { attestation: single_attestation, }, - None, error, + false, seen_timestamp, ); } @@ -536,8 +536,8 @@ impl NetworkBeaconProcessor { FailedAtt::SingleUnaggregate { attestation: single_attestation, }, - None, AttnError::BeaconChainError(error), + false, seen_timestamp, ); } @@ -556,7 +556,7 @@ impl NetworkBeaconProcessor { message_id: MessageId, peer_id: PeerId, aggregate: Box>, - reprocess_tx: Option>, + allow_reprocess: bool, seen_timestamp: Duration, ) { let beacon_block_root = aggregate.message().aggregate().data().beacon_block_root; @@ -580,7 +580,7 @@ impl NetworkBeaconProcessor { beacon_block_root, message_id, peer_id, - reprocess_tx, + allow_reprocess, seen_timestamp, ); } @@ -588,7 +588,7 @@ impl NetworkBeaconProcessor { pub fn process_gossip_aggregate_batch( self: Arc, packages: Vec>, - reprocess_tx: Option>, + allow_reprocess: bool, ) { let aggregates = packages.iter().map(|package| package.aggregate.as_ref()); @@ -642,7 +642,7 @@ impl NetworkBeaconProcessor { package.beacon_block_root, package.message_id, package.peer_id, - reprocess_tx.clone(), + allow_reprocess, package.seen_timestamp, ); } @@ -654,7 +654,7 @@ impl NetworkBeaconProcessor { beacon_block_root: Hash256, message_id: MessageId, peer_id: PeerId, - reprocess_tx: Option>, + allow_reprocess: bool, seen_timestamp: Duration, ) { match result { @@ -733,8 +733,8 @@ impl NetworkBeaconProcessor { attestation: signed_aggregate, seen_timestamp, }, - reprocess_tx, error, + allow_reprocess, seen_timestamp, ); } @@ -1200,20 +1200,12 @@ impl NetworkBeaconProcessor { peer_id: PeerId, peer_client: Client, block: Arc>, - reprocess_tx: mpsc::Sender, duplicate_cache: DuplicateCache, invalid_block_storage: InvalidBlockStorage, seen_duration: Duration, ) { if let Some(gossip_verified_block) = self - .process_gossip_unverified_block( - message_id, - peer_id, - peer_client, - block.clone(), - reprocess_tx.clone(), - seen_duration, - ) + .process_gossip_unverified_block(message_id, peer_id, peer_client, block, seen_duration) .await { let block_root = gossip_verified_block.block_root; @@ -1222,7 +1214,6 @@ impl NetworkBeaconProcessor { self.process_gossip_verified_block( peer_id, gossip_verified_block, - reprocess_tx, invalid_block_storage, seen_duration, ) @@ -1248,7 +1239,6 @@ impl NetworkBeaconProcessor { peer_id: PeerId, peer_client: Client, block: Arc>, - reprocess_tx: mpsc::Sender, seen_duration: Duration, ) -> Option> { let block_delay = @@ -1478,24 +1468,28 @@ impl NetworkBeaconProcessor { 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 { - beacon_block_slot: block_slot, - beacon_block_root: block_root, - process_fn, - })) + if self + .beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(ReprocessQueueMessage::EarlyBlock( + QueuedGossipBlock { + beacon_block_slot: block_slot, + beacon_block_root: block_root, + process_fn, + }, + )), + }) .is_err() { error!( @@ -1528,7 +1522,6 @@ impl NetworkBeaconProcessor { self: Arc, peer_id: PeerId, verified_block: GossipVerifiedBlock, - reprocess_tx: mpsc::Sender, invalid_block_storage: InvalidBlockStorage, _seen_duration: Duration, ) { @@ -1578,10 +1571,16 @@ impl NetworkBeaconProcessor { match &result { Ok(AvailabilityProcessingStatus::Imported(block_root)) => { - if reprocess_tx - .try_send(ReprocessQueueMessage::BlockImported { - block_root: *block_root, - parent_root: block.message().parent_root(), + metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_IMPORTED_TOTAL); + + if self + .beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(ReprocessQueueMessage::BlockImported { + block_root: *block_root, + parent_root: block.message().parent_root(), + }), }) .is_err() { @@ -2106,7 +2105,7 @@ impl NetworkBeaconProcessor { message_id: MessageId, peer_id: PeerId, light_client_optimistic_update: LightClientOptimisticUpdate, - reprocess_tx: Option>, + allow_reprocess: bool, seen_timestamp: Duration, ) { match self.chain.verify_optimistic_update_for_gossip( @@ -2134,7 +2133,7 @@ impl NetworkBeaconProcessor { "Optimistic update for unknown block" ); - if let Some(sender) = reprocess_tx { + if allow_reprocess { let processor = self.clone(); let msg = ReprocessQueueMessage::UnknownLightClientOptimisticUpdate( QueuedLightClientUpdate { @@ -2144,14 +2143,21 @@ impl NetworkBeaconProcessor { message_id, peer_id, light_client_optimistic_update, - None, // Do not reprocess this message again. + false, // Do not reprocess this message again. seen_timestamp, ) }), }, ); - if sender.try_send(msg).is_err() { + if self + .beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: true, + work: Work::Reprocess(msg), + }) + .is_err() + { error!("Failed to send optimistic update for re-processing") } } else { @@ -2221,8 +2227,8 @@ impl NetworkBeaconProcessor { peer_id: PeerId, message_id: MessageId, failed_att: FailedAtt, - reprocess_tx: Option>, error: AttnError, + allow_reprocess: bool, seen_timestamp: Duration, ) { let beacon_block_root = failed_att.beacon_block_root(); @@ -2461,7 +2467,7 @@ impl NetworkBeaconProcessor { block = ?beacon_block_root, "Attestation for unknown block" ); - if let Some(sender) = reprocess_tx { + if allow_reprocess { // We don't know the block, get the sync manager to handle the block lookup, and // send the attestation to be scheduled for re-processing. self.sync_tx @@ -2488,7 +2494,7 @@ impl NetworkBeaconProcessor { message_id, peer_id, attestation, - None, // Do not allow this attestation to be re-processed beyond this point. + false, // Do not allow this attestation to be re-processed beyond this point. seen_timestamp, ) }), @@ -2523,7 +2529,7 @@ impl NetworkBeaconProcessor { attestation, subnet_id, should_import, - None, // Do not allow this attestation to be re-processed beyond this point. + false, // Do not allow this attestation to be re-processed beyond this point. seen_timestamp, ) }), @@ -2531,7 +2537,14 @@ impl NetworkBeaconProcessor { } }; - if sender.try_send(msg).is_err() { + if self + .beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(msg), + }) + .is_err() + { error!("Failed to send attestation for re-processing") } } else { diff --git a/beacon_node/network/src/network_beacon_processor/mod.rs b/beacon_node/network/src/network_beacon_processor/mod.rs index 9a8edbfa4c4..f27b3c1b497 100644 --- a/beacon_node/network/src/network_beacon_processor/mod.rs +++ b/beacon_node/network/src/network_beacon_processor/mod.rs @@ -12,8 +12,8 @@ use beacon_chain::{ AvailabilityProcessingStatus, BeaconChain, BeaconChainTypes, BlockError, NotifyExecutionLayer, }; use beacon_processor::{ - work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend, DuplicateCache, - GossipAggregatePackage, GossipAttestationPackage, Work, WorkEvent as BeaconWorkEvent, + BeaconProcessorSend, DuplicateCache, GossipAggregatePackage, GossipAttestationPackage, Work, + WorkEvent as BeaconWorkEvent, }; use lighthouse_network::rpc::methods::{ BlobsByRangeRequest, BlobsByRootRequest, DataColumnsByRangeRequest, DataColumnsByRootRequest, @@ -61,7 +61,6 @@ pub struct NetworkBeaconProcessor { 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, @@ -87,14 +86,12 @@ impl NetworkBeaconProcessor { ) -> Result<(), Error> { let processor = self.clone(); let process_individual = move |package: GossipAttestationPackage| { - let reprocess_tx = processor.reprocess_tx.clone(); processor.process_gossip_attestation_to_convert( package.message_id, package.peer_id, package.attestation, package.subnet_id, package.should_import, - Some(reprocess_tx), package.seen_timestamp, ) }; @@ -129,24 +126,21 @@ impl NetworkBeaconProcessor { 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), + true, 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)) - }; + let process_batch = + move |attestations| processor.process_gossip_attestation_batch(attestations, true); self.try_send(BeaconWorkEvent { drop_during_sync: true, @@ -176,22 +170,19 @@ impl NetworkBeaconProcessor { // 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), + true, 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 process_batch = + move |aggregates| processor.process_gossip_aggregate_batch(aggregates, true); let beacon_block_root = aggregate.message().aggregate().data().beacon_block_root; self.try_send(BeaconWorkEvent { @@ -221,7 +212,6 @@ impl NetworkBeaconProcessor { ) -> 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 @@ -230,7 +220,6 @@ impl NetworkBeaconProcessor { peer_id, peer_client, block, - reprocess_tx, duplicate_cache, invalid_block_storage, seen_timestamp, @@ -423,12 +412,11 @@ impl NetworkBeaconProcessor { ) -> 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), + true, seen_timestamp, ) }; @@ -1159,8 +1147,6 @@ impl NetworkBeaconProcessor> { let BeaconProcessorChannels { beacon_processor_tx, beacon_processor_rx, - work_reprocessing_tx, - work_reprocessing_rx: _work_reprocessing_rx, } = <_>::default(); let (network_tx, _network_rx) = mpsc::unbounded_channel(); @@ -1171,7 +1157,6 @@ impl NetworkBeaconProcessor> { chain, network_tx, sync_tx, - reprocess_tx: work_reprocessing_tx, network_globals, invalid_block_storage: InvalidBlockStorage::Disabled, executor, diff --git a/beacon_node/network/src/network_beacon_processor/sync_methods.rs b/beacon_node/network/src/network_beacon_processor/sync_methods.rs index 48ae26c8265..dada0bfb0ca 100644 --- a/beacon_node/network/src/network_beacon_processor/sync_methods.rs +++ b/beacon_node/network/src/network_beacon_processor/sync_methods.rs @@ -14,14 +14,12 @@ use beacon_chain::{ BlockError, ChainSegmentResult, HistoricalBlockError, NotifyExecutionLayer, }; use beacon_processor::{ - work_reprocessing_queue::{QueuedRpcBlock, ReprocessQueueMessage}, - AsyncFn, BlockingFn, DuplicateCache, + AsyncFn, BlockingFn, DuplicateCache, QueuedRpcBlock, ReprocessQueueMessage, Work, WorkEvent, }; use lighthouse_network::PeerAction; use std::sync::Arc; use std::time::Duration; use store::KzgCommitment; -use tokio::sync::mpsc; use tracing::{debug, error, info, warn}; use types::beacon_block_body::format_kzg_commitments; use types::blob_sidecar::FixedBlobSidecarList; @@ -57,14 +55,12 @@ impl NetworkBeaconProcessor { process_type: BlockProcessType, ) -> 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; @@ -106,7 +102,6 @@ impl NetworkBeaconProcessor { block: RpcBlock, seen_timestamp: Duration, process_type: BlockProcessType, - reprocess_tx: mpsc::Sender, duplicate_cache: DuplicateCache, ) { // Check if the block is already being imported through another source @@ -131,7 +126,14 @@ impl NetworkBeaconProcessor { ignore_fn, }); - if reprocess_tx.try_send(reprocess_msg).is_err() { + if self + .beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(reprocess_msg), + }) + .is_err() + { error!(source = "rpc", %block_root,"Failed to inform block import") }; return; @@ -176,7 +178,15 @@ impl NetworkBeaconProcessor { block_root: *hash, parent_root, }; - if reprocess_tx.try_send(reprocess_msg).is_err() { + + if self + .beacon_processor_send + .try_send(WorkEvent { + drop_during_sync: false, + work: Work::Reprocess(reprocess_msg), + }) + .is_err() + { error!( source = "rpc", block_root = %hash, diff --git a/beacon_node/network/src/network_beacon_processor/tests.rs b/beacon_node/network/src/network_beacon_processor/tests.rs index 292e894870f..8537a52919a 100644 --- a/beacon_node/network/src/network_beacon_processor/tests.rs +++ b/beacon_node/network/src/network_beacon_processor/tests.rs @@ -15,7 +15,7 @@ use beacon_chain::test_utils::{ EphemeralHarnessType, }; use beacon_chain::{BeaconChain, WhenSlotSkipped}; -use beacon_processor::{work_reprocessing_queue::*, *}; +use beacon_processor::*; use itertools::Itertools; use lighthouse_network::rpc::methods::{BlobsByRangeRequest, MetaDataV3}; use lighthouse_network::rpc::InboundRequestId; @@ -50,6 +50,13 @@ const SEQ_NUMBER: u64 = 0; /// The default time to wait for `BeaconProcessor` events. const STANDARD_TIMEOUT: Duration = Duration::from_secs(10); +// TODO(beacon-processor) import these two instead of defining +/// For how long to queue rpc blocks before sending them back for reprocessing. +pub const QUEUED_RPC_BLOCK_DELAY: Duration = Duration::from_secs(4); + +/// For how long to queue aggregated and unaggregated attestations for re-processing. +pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12); + /// Provides utilities for testing the `BeaconProcessor`. struct TestRig { chain: Arc>, @@ -192,8 +199,6 @@ impl TestRig { let BeaconProcessorChannels { beacon_processor_tx, beacon_processor_rx, - work_reprocessing_tx, - work_reprocessing_rx, } = BeaconProcessorChannels::new(&beacon_processor_config); let (sync_tx, _sync_rx) = mpsc::unbounded_channel(); @@ -237,13 +242,14 @@ impl TestRig { 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(), }; let network_beacon_processor = Arc::new(network_beacon_processor); + let beacon_state = &chain.canonical_head.cached_head().snapshot.beacon_state; + let beacon_processor = BeaconProcessor { network_globals: network_globals.clone(), executor, @@ -251,17 +257,11 @@ impl TestRig { config: beacon_processor_config, } .spawn_manager( + beacon_state, beacon_processor_rx, - work_reprocessing_tx, - work_reprocessing_rx, Some(work_journal_tx), harness.chain.slot_clock.clone(), - chain.spec.maximum_gossip_clock_disparity(), - BeaconProcessorQueueLengths::from_state( - &chain.canonical_head.cached_head().snapshot.beacon_state, - &chain.spec, - ) - .unwrap(), + &chain.spec, ); assert!(beacon_processor.is_ok()); diff --git a/beacon_node/network/src/router.rs b/beacon_node/network/src/router.rs index 05c00b76af1..a6b98000851 100644 --- a/beacon_node/network/src/router.rs +++ b/beacon_node/network/src/router.rs @@ -10,9 +10,7 @@ use crate::service::NetworkMessage; use crate::status::status_message; use crate::sync::SyncMessage; use beacon_chain::{BeaconChain, BeaconChainTypes}; -use beacon_processor::{ - work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend, DuplicateCache, -}; +use beacon_processor::{BeaconProcessorSend, DuplicateCache}; use futures::prelude::*; use lighthouse_network::rpc::*; use lighthouse_network::{ @@ -85,7 +83,6 @@ impl Router { executor: task_executor::TaskExecutor, invalid_block_storage: InvalidBlockStorage, beacon_processor_send: BeaconProcessorSend, - beacon_processor_reprocess_tx: mpsc::Sender, fork_context: Arc, ) -> Result>, String> { trace!("Service starting"); @@ -101,7 +98,6 @@ impl Router { 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(), diff --git a/beacon_node/network/src/service.rs b/beacon_node/network/src/service.rs index 7afd62ab2e7..968adfdbfd2 100644 --- a/beacon_node/network/src/service.rs +++ b/beacon_node/network/src/service.rs @@ -6,7 +6,7 @@ use crate::router::{Router, RouterMessage}; use crate::subnet_service::{SubnetService, SubnetServiceMessage, Subscription}; use crate::NetworkConfig; use beacon_chain::{BeaconChain, BeaconChainTypes}; -use beacon_processor::{work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend}; +use beacon_processor::BeaconProcessorSend; use futures::channel::mpsc::Sender; use futures::future::OptionFuture; use futures::prelude::*; @@ -204,7 +204,6 @@ impl NetworkService { executor: task_executor::TaskExecutor, libp2p_registry: Option<&'_ mut Registry>, beacon_processor_send: BeaconProcessorSend, - beacon_processor_reprocess_tx: mpsc::Sender, ) -> Result< ( NetworkService, @@ -300,7 +299,6 @@ impl NetworkService { executor.clone(), invalid_block_storage, beacon_processor_send, - beacon_processor_reprocess_tx, fork_context.clone(), )?; @@ -352,7 +350,6 @@ impl NetworkService { executor: task_executor::TaskExecutor, libp2p_registry: Option<&'_ mut Registry>, beacon_processor_send: BeaconProcessorSend, - beacon_processor_reprocess_tx: mpsc::Sender, ) -> Result<(Arc>, NetworkSenders), String> { let (network_service, network_globals, network_senders) = Self::build( beacon_chain, @@ -360,7 +357,6 @@ impl NetworkService { executor.clone(), libp2p_registry, beacon_processor_send, - beacon_processor_reprocess_tx, ) .await?; diff --git a/beacon_node/network/src/service/tests.rs b/beacon_node/network/src/service/tests.rs index 15c3321e94d..db342117473 100644 --- a/beacon_node/network/src/service/tests.rs +++ b/beacon_node/network/src/service/tests.rs @@ -58,8 +58,6 @@ fn test_dht_persistence() { let BeaconProcessorChannels { beacon_processor_tx, beacon_processor_rx: _beacon_processor_rx, - work_reprocessing_tx, - work_reprocessing_rx: _work_reprocessing_rx, } = <_>::default(); let _network_service = NetworkService::start( @@ -68,7 +66,6 @@ fn test_dht_persistence() { executor, None, beacon_processor_tx, - work_reprocessing_tx, ) .await .unwrap(); @@ -137,7 +134,6 @@ fn test_removing_topic_weight_on_old_topics() { executor.clone(), None, beacon_processor_channels.beacon_processor_tx, - beacon_processor_channels.work_reprocessing_tx, ) .await .unwrap() diff --git a/lighthouse/tests/beacon_node.rs b/lighthouse/tests/beacon_node.rs index ea4716c0103..cf40141cdd9 100644 --- a/lighthouse/tests/beacon_node.rs +++ b/lighthouse/tests/beacon_node.rs @@ -8,6 +8,7 @@ use beacon_node::{ beacon_chain::store::config::DatabaseBackend as BeaconNodeBackend, ClientConfig as Config, }; use beacon_processor::BeaconProcessorConfig; +use beacon_processor::BeaconProcessorType; use eth1::Eth1Endpoint; use lighthouse_network::PeerId; use std::fs::File; @@ -2702,7 +2703,8 @@ fn beacon_processor() { max_scheduled_work_queue_len: 3, max_gossip_attestation_batch_size: 4, max_gossip_aggregate_batch_size: 5, - enable_backfill_rate_limiting: false + enable_backfill_rate_limiting: false, + beacon_processor_type: BeaconProcessorType::EarliestDeadline, } ) });