Move the BeaconProcessor
into a new crate (#4435)
*Replaces #4434. It is identical, but this PR has a smaller diff due to a curated commit history.* ## Issue Addressed NA ## Proposed Changes This PR moves the scheduling logic for the `BeaconProcessor` into a new crate in `beacon_node/beacon_processor`. Previously it existed in the `beacon_node/network` crate. This addresses a circular-dependency problem where it's not possible to use the `BeaconProcessor` from the `beacon_chain` crate. The `network` crate depends on the `beacon_chain` crate (`network -> beacon_chain`), but importing the `BeaconProcessor` into the `beacon_chain` crate would create a circular dependancy of `beacon_chain -> network`. The `BeaconProcessor` was designed to provide queuing and prioritized scheduling for messages from the network. It has proven to be quite valuable and I believe we'd make Lighthouse more stable and effective by using it elsewhere. In particular, I think we should use the `BeaconProcessor` for: 1. HTTP API requests. 1. Scheduled tasks in the `BeaconChain` (e.g., state advance). Using the `BeaconProcessor` for these tasks would help prevent the BN from becoming overwhelmed and would also help it to prioritize operations (e.g., choosing to process blocks from gossip before responding to low-priority HTTP API requests). ## Additional Info This PR is intended to have zero impact on runtime behaviour. It aims to simply separate the *scheduling* code (i.e., the `BeaconProcessor`) from the *business logic* in the `network` crate (i.e., the `Worker` impls). Future PRs (see #4462) can build upon these works to actually use the `BeaconProcessor` for more operations. I've gone to some effort to use `git mv` to make the diff look more like "file was moved and modified" rather than "file was deleted and a new one added". This should reduce review burden and help maintain commit attribution.
This commit is contained in:
parent
ea2420d193
commit
c25825a539
28
Cargo.lock
generated
28
Cargo.lock
generated
@ -700,6 +700,30 @@ dependencies = [
|
||||
"unused_port",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "beacon_processor"
|
||||
version = "0.1.0"
|
||||
dependencies = [
|
||||
"derivative",
|
||||
"ethereum_ssz",
|
||||
"fnv",
|
||||
"futures",
|
||||
"hex",
|
||||
"itertools",
|
||||
"lazy_static",
|
||||
"lighthouse_metrics",
|
||||
"lighthouse_network",
|
||||
"logging",
|
||||
"parking_lot 0.12.1",
|
||||
"slog",
|
||||
"slot_clock",
|
||||
"strum",
|
||||
"task_executor",
|
||||
"tokio",
|
||||
"tokio-util 0.6.10",
|
||||
"types",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "bincode"
|
||||
version = "1.3.3"
|
||||
@ -1148,6 +1172,7 @@ name = "client"
|
||||
version = "0.2.0"
|
||||
dependencies = [
|
||||
"beacon_chain",
|
||||
"beacon_processor",
|
||||
"directory",
|
||||
"dirs",
|
||||
"environment",
|
||||
@ -1165,6 +1190,7 @@ dependencies = [
|
||||
"logging",
|
||||
"monitoring_api",
|
||||
"network",
|
||||
"num_cpus",
|
||||
"operation_pool",
|
||||
"parking_lot 0.12.1",
|
||||
"sensitive_url",
|
||||
@ -5405,6 +5431,7 @@ name = "network"
|
||||
version = "0.2.0"
|
||||
dependencies = [
|
||||
"beacon_chain",
|
||||
"beacon_processor",
|
||||
"delay_map",
|
||||
"derivative",
|
||||
"environment",
|
||||
@ -5428,6 +5455,7 @@ dependencies = [
|
||||
"matches",
|
||||
"num_cpus",
|
||||
"operation_pool",
|
||||
"parking_lot 0.12.1",
|
||||
"rand 0.8.5",
|
||||
"rlp",
|
||||
"slog",
|
||||
|
@ -4,6 +4,7 @@ members = [
|
||||
|
||||
"beacon_node",
|
||||
"beacon_node/beacon_chain",
|
||||
"beacon_node/beacon_processor",
|
||||
"beacon_node/builder_client",
|
||||
"beacon_node/client",
|
||||
"beacon_node/eth1",
|
||||
|
24
beacon_node/beacon_processor/Cargo.toml
Normal file
24
beacon_node/beacon_processor/Cargo.toml
Normal file
@ -0,0 +1,24 @@
|
||||
[package]
|
||||
name = "beacon_processor"
|
||||
version = "0.1.0"
|
||||
edition = "2021"
|
||||
|
||||
[dependencies]
|
||||
slog = { version = "2.5.2", features = ["max_level_trace"] }
|
||||
itertools = "0.10.0"
|
||||
logging = { path = "../../common/logging" }
|
||||
tokio = { version = "1.14.0", features = ["full"] }
|
||||
tokio-util = { version = "0.6.3", features = ["time"] }
|
||||
futures = "0.3.7"
|
||||
fnv = "1.0.7"
|
||||
strum = "0.24.0"
|
||||
task_executor = { path = "../../common/task_executor" }
|
||||
slot_clock = { path = "../../common/slot_clock" }
|
||||
lighthouse_network = { path = "../lighthouse_network" }
|
||||
hex = "0.4.2"
|
||||
derivative = "2.2.0"
|
||||
types = { path = "../../consensus/types" }
|
||||
ethereum_ssz = "0.5.0"
|
||||
lazy_static = "1.4.0"
|
||||
lighthouse_metrics = { path = "../../common/lighthouse_metrics" }
|
||||
parking_lot = "0.12.0"
|
File diff suppressed because it is too large
Load Diff
141
beacon_node/beacon_processor/src/metrics.rs
Normal file
141
beacon_node/beacon_processor/src/metrics.rs
Normal file
@ -0,0 +1,141 @@
|
||||
pub use lighthouse_metrics::*;
|
||||
|
||||
lazy_static::lazy_static! {
|
||||
|
||||
/*
|
||||
* Gossip processor
|
||||
*/
|
||||
pub static ref BEACON_PROCESSOR_WORK_EVENTS_RX_COUNT: Result<IntCounterVec> = try_create_int_counter_vec(
|
||||
"beacon_processor_work_events_rx_count",
|
||||
"Count of work events received (but not necessarily processed)",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORK_EVENTS_IGNORED_COUNT: Result<IntCounterVec> = try_create_int_counter_vec(
|
||||
"beacon_processor_work_events_ignored_count",
|
||||
"Count of work events purposefully ignored",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORK_EVENTS_STARTED_COUNT: Result<IntCounterVec> = try_create_int_counter_vec(
|
||||
"beacon_processor_work_events_started_count",
|
||||
"Count of work events which have been started by a worker",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORKER_TIME: Result<HistogramVec> = try_create_histogram_vec(
|
||||
"beacon_processor_worker_time",
|
||||
"Time taken for a worker to fully process some parcel of work.",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORKERS_SPAWNED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_workers_spawned_total",
|
||||
"The number of workers ever spawned by the gossip processing pool."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORKERS_ACTIVE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_workers_active_total",
|
||||
"Count of active workers in the gossip processing pool."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_IDLE_EVENTS_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_idle_events_total",
|
||||
"Count of idle events processed by the gossip processor manager."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_EVENT_HANDLING_SECONDS: Result<Histogram> = try_create_histogram(
|
||||
"beacon_processor_event_handling_seconds",
|
||||
"Time spent handling a new message and allocating it to a queue or worker."
|
||||
);
|
||||
// Gossip blocks.
|
||||
pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_gossip_block_queue_total",
|
||||
"Count of blocks from gossip waiting to be verified."
|
||||
);
|
||||
// Gossip Exits.
|
||||
pub static ref BEACON_PROCESSOR_EXIT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_exit_queue_total",
|
||||
"Count of exits from gossip waiting to be verified."
|
||||
);
|
||||
// Gossip proposer slashings.
|
||||
pub static ref BEACON_PROCESSOR_PROPOSER_SLASHING_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_proposer_slashing_queue_total",
|
||||
"Count of proposer slashings from gossip waiting to be verified."
|
||||
);
|
||||
// Gossip attester slashings.
|
||||
pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_attester_slashing_queue_total",
|
||||
"Count of attester slashings from gossip waiting to be verified."
|
||||
);
|
||||
// Gossip BLS to execution changes.
|
||||
pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_bls_to_execution_change_queue_total",
|
||||
"Count of address changes from gossip waiting to be verified."
|
||||
);
|
||||
// Rpc blocks.
|
||||
pub static ref BEACON_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_rpc_block_queue_total",
|
||||
"Count of blocks from the rpc waiting to be verified."
|
||||
);
|
||||
// Chain segments.
|
||||
pub static ref BEACON_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_chain_segment_queue_total",
|
||||
"Count of chain segments from the rpc waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_BACKFILL_CHAIN_SEGMENT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_backfill_chain_segment_queue_total",
|
||||
"Count of backfill chain segments from the rpc waiting to be verified."
|
||||
);
|
||||
// Unaggregated attestations.
|
||||
pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_unaggregated_attestation_queue_total",
|
||||
"Count of unagg. attestations waiting to be processed."
|
||||
);
|
||||
// Aggregated attestations.
|
||||
pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_aggregated_attestation_queue_total",
|
||||
"Count of agg. attestations waiting to be processed."
|
||||
);
|
||||
// Sync committee messages.
|
||||
pub static ref BEACON_PROCESSOR_SYNC_MESSAGE_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_sync_message_queue_total",
|
||||
"Count of sync committee messages waiting to be processed."
|
||||
);
|
||||
// Sync contribution.
|
||||
pub static ref BEACON_PROCESSOR_SYNC_CONTRIBUTION_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_sync_contribution_queue_total",
|
||||
"Count of sync committee contributions waiting to be processed."
|
||||
);
|
||||
|
||||
/*
|
||||
* Attestation reprocessing queue metrics.
|
||||
*/
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL: Result<IntGaugeVec> =
|
||||
try_create_int_gauge_vec(
|
||||
"beacon_processor_reprocessing_queue_total",
|
||||
"Count of items in a reprocessing queue.",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_expired_attestations",
|
||||
"Number of queued attestations which have expired before a matching block has been found."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_ATTESTATIONS: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_matched_attestations",
|
||||
"Number of queued attestations where as matching block has been imported."
|
||||
);
|
||||
|
||||
/*
|
||||
* Light client update reprocessing queue metrics.
|
||||
*/
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_OPTIMISTIC_UPDATES: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_expired_optimistic_updates",
|
||||
"Number of queued light client optimistic updates which have expired before a matching block has been found."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_OPTIMISTIC_UPDATES: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_matched_optimistic_updates",
|
||||
"Number of queued light client optimistic updates where as matching block has been imported."
|
||||
);
|
||||
|
||||
/// Errors and Debugging Stats
|
||||
pub static ref BEACON_PROCESSOR_SEND_ERROR_PER_WORK_TYPE: Result<IntCounterVec> =
|
||||
try_create_int_counter_vec(
|
||||
"beacon_processor_send_error_per_work_type",
|
||||
"Total number of beacon processor send error per work type",
|
||||
&["type"]
|
||||
);
|
||||
}
|
@ -10,23 +10,18 @@
|
||||
//!
|
||||
//! Aggregated and unaggregated attestations that failed verification due to referencing an unknown
|
||||
//! block will be re-queued until their block is imported, or until they expire.
|
||||
use super::MAX_SCHEDULED_WORK_QUEUE_LEN;
|
||||
use crate::beacon_processor::{ChainSegmentProcessId, Work, WorkEvent};
|
||||
use crate::metrics;
|
||||
use crate::sync::manager::BlockProcessType;
|
||||
use beacon_chain::{BeaconChainTypes, GossipVerifiedBlock, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
||||
use crate::{AsyncFn, BlockingFn, Work, WorkEvent};
|
||||
use fnv::FnvHashMap;
|
||||
use futures::task::Poll;
|
||||
use futures::{Stream, StreamExt};
|
||||
use itertools::Itertools;
|
||||
use lighthouse_network::{MessageId, PeerId};
|
||||
use logging::TimeLatch;
|
||||
use slog::{crit, debug, error, trace, warn, Logger};
|
||||
use slot_clock::SlotClock;
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::future::Future;
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
use std::task::Context;
|
||||
use std::time::Duration;
|
||||
use strum::AsRefStr;
|
||||
@ -34,10 +29,7 @@ use task_executor::TaskExecutor;
|
||||
use tokio::sync::mpsc::{self, Receiver, Sender};
|
||||
use tokio::time::error::Error as TimeError;
|
||||
use tokio_util::time::delay_queue::{DelayQueue, Key as DelayKey};
|
||||
use types::{
|
||||
Attestation, EthSpec, Hash256, LightClientOptimisticUpdate, SignedAggregateAndProof,
|
||||
SignedBeaconBlock, SubnetId,
|
||||
};
|
||||
use types::{EthSpec, Hash256, Slot};
|
||||
|
||||
const TASK_NAME: &str = "beacon_processor_reprocess_queue";
|
||||
const GOSSIP_BLOCKS: &str = "gossip_blocks";
|
||||
@ -47,7 +39,7 @@ const LIGHT_CLIENT_UPDATES: &str = "lc_updates";
|
||||
|
||||
/// Queue blocks for re-processing with an `ADDITIONAL_QUEUED_BLOCK_DELAY` after the slot starts.
|
||||
/// This is to account for any slight drift in the system clock.
|
||||
const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5);
|
||||
pub const ADDITIONAL_QUEUED_BLOCK_DELAY: Duration = Duration::from_millis(5);
|
||||
|
||||
/// For how long to queue aggregated and unaggregated attestations for re-processing.
|
||||
pub const QUEUED_ATTESTATION_DELAY: Duration = Duration::from_secs(12);
|
||||
@ -84,12 +76,12 @@ pub const BACKFILL_SCHEDULE_IN_SLOT: [(u32, u32); 3] = [
|
||||
|
||||
/// Messages that the scheduler can receive.
|
||||
#[derive(AsRefStr)]
|
||||
pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
||||
pub enum ReprocessQueueMessage {
|
||||
/// A block that has been received early and we should queue for later processing.
|
||||
EarlyBlock(QueuedGossipBlock<T>),
|
||||
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<T::EthSpec>),
|
||||
RpcBlock(QueuedRpcBlock),
|
||||
/// A block that was successfully processed. We use this to handle attestations and light client updates
|
||||
/// for unknown blocks.
|
||||
BlockImported {
|
||||
@ -97,139 +89,127 @@ pub enum ReprocessQueueMessage<T: BeaconChainTypes> {
|
||||
parent_root: Hash256,
|
||||
},
|
||||
/// An unaggregated attestation that references an unknown block.
|
||||
UnknownBlockUnaggregate(QueuedUnaggregate<T::EthSpec>),
|
||||
UnknownBlockUnaggregate(QueuedUnaggregate),
|
||||
/// An aggregated attestation that references an unknown block.
|
||||
UnknownBlockAggregate(QueuedAggregate<T::EthSpec>),
|
||||
UnknownBlockAggregate(QueuedAggregate),
|
||||
/// A light client optimistic update that references a parent root that has not been seen as a parent.
|
||||
UnknownLightClientOptimisticUpdate(QueuedLightClientUpdate<T::EthSpec>),
|
||||
UnknownLightClientOptimisticUpdate(QueuedLightClientUpdate),
|
||||
/// A new backfill batch that needs to be scheduled for processing.
|
||||
BackfillSync(QueuedBackfillBatch<T::EthSpec>),
|
||||
BackfillSync(QueuedBackfillBatch),
|
||||
}
|
||||
|
||||
/// Events sent by the scheduler once they are ready for re-processing.
|
||||
pub enum ReadyWork<T: BeaconChainTypes> {
|
||||
Block(QueuedGossipBlock<T>),
|
||||
RpcBlock(QueuedRpcBlock<T::EthSpec>),
|
||||
Unaggregate(QueuedUnaggregate<T::EthSpec>),
|
||||
Aggregate(QueuedAggregate<T::EthSpec>),
|
||||
LightClientUpdate(QueuedLightClientUpdate<T::EthSpec>),
|
||||
BackfillSync(QueuedBackfillBatch<T::EthSpec>),
|
||||
pub enum ReadyWork {
|
||||
Block(QueuedGossipBlock),
|
||||
RpcBlock(QueuedRpcBlock),
|
||||
IgnoredRpcBlock(IgnoredRpcBlock),
|
||||
Unaggregate(QueuedUnaggregate),
|
||||
Aggregate(QueuedAggregate),
|
||||
LightClientUpdate(QueuedLightClientUpdate),
|
||||
BackfillSync(QueuedBackfillBatch),
|
||||
}
|
||||
|
||||
/// An Attestation for which the corresponding block was not seen while processing, queued for
|
||||
/// later.
|
||||
pub struct QueuedUnaggregate<T: EthSpec> {
|
||||
pub peer_id: PeerId,
|
||||
pub message_id: MessageId,
|
||||
pub attestation: Box<Attestation<T>>,
|
||||
pub subnet_id: SubnetId,
|
||||
pub should_import: bool,
|
||||
pub seen_timestamp: Duration,
|
||||
pub struct QueuedUnaggregate {
|
||||
pub beacon_block_root: Hash256,
|
||||
pub process_fn: BlockingFn,
|
||||
}
|
||||
|
||||
/// An aggregated attestation for which the corresponding block was not seen while processing, queued for
|
||||
/// later.
|
||||
pub struct QueuedAggregate<T: EthSpec> {
|
||||
pub peer_id: PeerId,
|
||||
pub message_id: MessageId,
|
||||
pub attestation: Box<SignedAggregateAndProof<T>>,
|
||||
pub seen_timestamp: Duration,
|
||||
pub struct QueuedAggregate {
|
||||
pub beacon_block_root: Hash256,
|
||||
pub process_fn: BlockingFn,
|
||||
}
|
||||
|
||||
/// A light client update for which the corresponding parent block was not seen while processing,
|
||||
/// queued for later.
|
||||
pub struct QueuedLightClientUpdate<T: EthSpec> {
|
||||
pub peer_id: PeerId,
|
||||
pub message_id: MessageId,
|
||||
pub light_client_optimistic_update: Box<LightClientOptimisticUpdate<T>>,
|
||||
pub struct QueuedLightClientUpdate {
|
||||
pub parent_root: Hash256,
|
||||
pub seen_timestamp: Duration,
|
||||
pub process_fn: BlockingFn,
|
||||
}
|
||||
|
||||
/// A block that arrived early and has been queued for later import.
|
||||
pub struct QueuedGossipBlock<T: BeaconChainTypes> {
|
||||
pub peer_id: PeerId,
|
||||
pub block: Box<GossipVerifiedBlock<T>>,
|
||||
pub seen_timestamp: Duration,
|
||||
pub struct QueuedGossipBlock {
|
||||
pub beacon_block_slot: Slot,
|
||||
pub beacon_block_root: Hash256,
|
||||
pub process_fn: AsyncFn,
|
||||
}
|
||||
|
||||
/// A block that arrived for processing when the same block was being imported over gossip.
|
||||
/// It is queued for later import.
|
||||
pub struct QueuedRpcBlock<T: EthSpec> {
|
||||
pub block_root: Hash256,
|
||||
pub block: Arc<SignedBeaconBlock<T>>,
|
||||
pub process_type: BlockProcessType,
|
||||
pub seen_timestamp: Duration,
|
||||
/// Indicates if the beacon chain should process this block or not.
|
||||
/// We use this to ignore block processing when rpc block queues are full.
|
||||
pub should_process: bool,
|
||||
pub struct QueuedRpcBlock {
|
||||
pub beacon_block_root: Hash256,
|
||||
/// Processes/imports the block.
|
||||
pub process_fn: AsyncFn,
|
||||
/// Ignores the block.
|
||||
pub ignore_fn: BlockingFn,
|
||||
}
|
||||
|
||||
/// A 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.
|
||||
#[derive(Clone)]
|
||||
pub struct QueuedBackfillBatch<E: EthSpec> {
|
||||
pub process_id: ChainSegmentProcessId,
|
||||
pub blocks: Vec<Arc<SignedBeaconBlock<E>>>,
|
||||
}
|
||||
pub struct QueuedBackfillBatch(pub AsyncFn);
|
||||
|
||||
impl<T: BeaconChainTypes> TryFrom<WorkEvent<T>> for QueuedBackfillBatch<T::EthSpec> {
|
||||
impl<T: EthSpec> TryFrom<WorkEvent<T>> for QueuedBackfillBatch {
|
||||
type Error = WorkEvent<T>;
|
||||
|
||||
fn try_from(event: WorkEvent<T>) -> Result<Self, WorkEvent<T>> {
|
||||
match event {
|
||||
WorkEvent {
|
||||
work:
|
||||
Work::ChainSegment {
|
||||
process_id: process_id @ ChainSegmentProcessId::BackSyncBatchId(_),
|
||||
blocks,
|
||||
},
|
||||
work: Work::ChainSegmentBackfill(process_fn),
|
||||
..
|
||||
} => Ok(QueuedBackfillBatch { process_id, blocks }),
|
||||
} => Ok(QueuedBackfillBatch(process_fn)),
|
||||
_ => Err(event),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> From<QueuedBackfillBatch<T::EthSpec>> for WorkEvent<T> {
|
||||
fn from(queued_backfill_batch: QueuedBackfillBatch<T::EthSpec>) -> WorkEvent<T> {
|
||||
WorkEvent::chain_segment(
|
||||
queued_backfill_batch.process_id,
|
||||
queued_backfill_batch.blocks,
|
||||
)
|
||||
impl<T: EthSpec> From<QueuedBackfillBatch> for WorkEvent<T> {
|
||||
fn from(queued_backfill_batch: QueuedBackfillBatch) -> WorkEvent<T> {
|
||||
WorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::ChainSegmentBackfill(queued_backfill_batch.0),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Unifies the different messages processed by the block delay queue.
|
||||
enum InboundEvent<T: BeaconChainTypes> {
|
||||
enum InboundEvent {
|
||||
/// A gossip block that was queued for later processing and is ready for import.
|
||||
ReadyGossipBlock(QueuedGossipBlock<T>),
|
||||
ReadyGossipBlock(QueuedGossipBlock),
|
||||
/// A rpc block that was queued because the same gossip block was being imported
|
||||
/// will now be retried for import.
|
||||
ReadyRpcBlock(QueuedRpcBlock<T::EthSpec>),
|
||||
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<T::EthSpec>),
|
||||
ReadyBackfillSync(QueuedBackfillBatch),
|
||||
/// A `DelayQueue` returned an error.
|
||||
DelayQueueError(TimeError, &'static str),
|
||||
/// A message sent to the `ReprocessQueue`
|
||||
Msg(ReprocessQueueMessage<T>),
|
||||
Msg(ReprocessQueueMessage),
|
||||
}
|
||||
|
||||
/// Manages scheduling works that need to be later re-processed.
|
||||
struct ReprocessQueue<T: BeaconChainTypes> {
|
||||
struct ReprocessQueue<S> {
|
||||
/// Receiver of messages relevant to schedule works for reprocessing.
|
||||
work_reprocessing_rx: Receiver<ReprocessQueueMessage<T>>,
|
||||
work_reprocessing_rx: Receiver<ReprocessQueueMessage>,
|
||||
/// Sender of works once they become ready
|
||||
ready_work_tx: Sender<ReadyWork<T>>,
|
||||
ready_work_tx: Sender<ReadyWork>,
|
||||
|
||||
/* Queues */
|
||||
/// Queue to manage scheduled early blocks.
|
||||
gossip_block_delay_queue: DelayQueue<QueuedGossipBlock<T>>,
|
||||
gossip_block_delay_queue: DelayQueue<QueuedGossipBlock>,
|
||||
/// Queue to manage scheduled early blocks.
|
||||
rpc_block_delay_queue: DelayQueue<QueuedRpcBlock<T::EthSpec>>,
|
||||
rpc_block_delay_queue: DelayQueue<QueuedRpcBlock>,
|
||||
/// Queue to manage scheduled attestations.
|
||||
attestations_delay_queue: DelayQueue<QueuedAttestationId>,
|
||||
/// Queue to manage scheduled light client updates.
|
||||
@ -239,17 +219,17 @@ struct ReprocessQueue<T: BeaconChainTypes> {
|
||||
/// Queued blocks.
|
||||
queued_gossip_block_roots: HashSet<Hash256>,
|
||||
/// Queued aggregated attestations.
|
||||
queued_aggregates: FnvHashMap<usize, (QueuedAggregate<T::EthSpec>, DelayKey)>,
|
||||
queued_aggregates: FnvHashMap<usize, (QueuedAggregate, DelayKey)>,
|
||||
/// Queued attestations.
|
||||
queued_unaggregates: FnvHashMap<usize, (QueuedUnaggregate<T::EthSpec>, DelayKey)>,
|
||||
queued_unaggregates: FnvHashMap<usize, (QueuedUnaggregate, DelayKey)>,
|
||||
/// Attestations (aggregated and unaggregated) per root.
|
||||
awaiting_attestations_per_root: HashMap<Hash256, Vec<QueuedAttestationId>>,
|
||||
/// Queued Light Client Updates.
|
||||
queued_lc_updates: FnvHashMap<usize, (QueuedLightClientUpdate<T::EthSpec>, DelayKey)>,
|
||||
queued_lc_updates: FnvHashMap<usize, (QueuedLightClientUpdate, DelayKey)>,
|
||||
/// Light Client Updates per parent_root.
|
||||
awaiting_lc_updates_per_parent_root: HashMap<Hash256, Vec<QueuedLightClientUpdateId>>,
|
||||
/// Queued backfill batches
|
||||
queued_backfill_batches: Vec<QueuedBackfillBatch<T::EthSpec>>,
|
||||
queued_backfill_batches: Vec<QueuedBackfillBatch>,
|
||||
|
||||
/* Aux */
|
||||
/// Next attestation id, used for both aggregated and unaggregated attestations
|
||||
@ -260,7 +240,7 @@ struct ReprocessQueue<T: BeaconChainTypes> {
|
||||
attestation_delay_debounce: TimeLatch,
|
||||
lc_update_delay_debounce: TimeLatch,
|
||||
next_backfill_batch_event: Option<Pin<Box<tokio::time::Sleep>>>,
|
||||
slot_clock: Pin<Box<T::SlotClock>>,
|
||||
slot_clock: Pin<Box<S>>,
|
||||
}
|
||||
|
||||
pub type QueuedLightClientUpdateId = usize;
|
||||
@ -271,20 +251,20 @@ enum QueuedAttestationId {
|
||||
Unaggregate(usize),
|
||||
}
|
||||
|
||||
impl<T: EthSpec> QueuedAggregate<T> {
|
||||
impl QueuedAggregate {
|
||||
pub fn beacon_block_root(&self) -> &Hash256 {
|
||||
&self.attestation.message.aggregate.data.beacon_block_root
|
||||
&self.beacon_block_root
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: EthSpec> QueuedUnaggregate<T> {
|
||||
impl QueuedUnaggregate {
|
||||
pub fn beacon_block_root(&self) -> &Hash256 {
|
||||
&self.attestation.data.beacon_block_root
|
||||
&self.beacon_block_root
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> {
|
||||
type Item = InboundEvent<T>;
|
||||
impl<S: SlotClock> Stream for ReprocessQueue<S> {
|
||||
type Item = InboundEvent;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
// NOTE: implementing `Stream` is not necessary but allows to maintain the future selection
|
||||
@ -375,16 +355,13 @@ impl<T: BeaconChainTypes> Stream for ReprocessQueue<T> {
|
||||
/// 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<T: BeaconChainTypes>(
|
||||
ready_work_tx: Sender<ReadyWork<T>>,
|
||||
pub fn spawn_reprocess_scheduler<S: SlotClock + 'static>(
|
||||
ready_work_tx: Sender<ReadyWork>,
|
||||
work_reprocessing_rx: Receiver<ReprocessQueueMessage>,
|
||||
executor: &TaskExecutor,
|
||||
slot_clock: T::SlotClock,
|
||||
slot_clock: S,
|
||||
log: Logger,
|
||||
) -> Sender<ReprocessQueueMessage<T>> {
|
||||
let (work_reprocessing_tx, work_reprocessing_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
|
||||
// Basic sanity check.
|
||||
assert!(ADDITIONAL_QUEUED_BLOCK_DELAY < MAXIMUM_GOSSIP_CLOCK_DISPARITY);
|
||||
|
||||
) {
|
||||
let mut queue = ReprocessQueue {
|
||||
work_reprocessing_rx,
|
||||
ready_work_tx,
|
||||
@ -423,19 +400,17 @@ pub fn spawn_reprocess_scheduler<T: BeaconChainTypes>(
|
||||
},
|
||||
TASK_NAME,
|
||||
);
|
||||
|
||||
work_reprocessing_tx
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
fn handle_message(&mut self, msg: InboundEvent<T>, slot_clock: &T::SlotClock, log: &Logger) {
|
||||
impl<S: SlotClock> ReprocessQueue<S> {
|
||||
fn handle_message(&mut self, msg: InboundEvent, slot_clock: &S, log: &Logger) {
|
||||
use ReprocessQueueMessage::*;
|
||||
match msg {
|
||||
// Some block has been indicated as "early" and should be processed when the
|
||||
// appropriate slot arrives.
|
||||
InboundEvent::Msg(EarlyBlock(early_block)) => {
|
||||
let block_slot = early_block.block.block.slot();
|
||||
let block_root = early_block.block.block_root;
|
||||
let block_slot = early_block.beacon_block_slot;
|
||||
let block_root = early_block.beacon_block_root;
|
||||
|
||||
// Don't add the same block to the queue twice. This prevents DoS attacks.
|
||||
if self.queued_gossip_block_roots.contains(&block_root) {
|
||||
@ -494,7 +469,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
// for the same block hash is being imported. We wait for `QUEUED_RPC_BLOCK_DELAY`
|
||||
// and then send the rpc block back for processing assuming the gossip import
|
||||
// has completed by then.
|
||||
InboundEvent::Msg(RpcBlock(mut rpc_block)) => {
|
||||
InboundEvent::Msg(RpcBlock(rpc_block)) => {
|
||||
// Check to ensure this won't over-fill the queue.
|
||||
if self.rpc_block_delay_queue.len() >= MAXIMUM_QUEUED_BLOCKS {
|
||||
if self.rpc_block_debounce.elapsed() {
|
||||
@ -507,10 +482,11 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
}
|
||||
// Return the block to the beacon processor signalling to
|
||||
// ignore processing for this block
|
||||
rpc_block.should_process = false;
|
||||
if self
|
||||
.ready_work_tx
|
||||
.try_send(ReadyWork::RpcBlock(rpc_block))
|
||||
.try_send(ReadyWork::IgnoredRpcBlock(IgnoredRpcBlock {
|
||||
process_fn: rpc_block.ignore_fn,
|
||||
}))
|
||||
.is_err()
|
||||
{
|
||||
error!(
|
||||
@ -529,7 +505,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
debug!(
|
||||
log,
|
||||
"Sending rpc block for reprocessing";
|
||||
"block_root" => %queued_rpc_block.block.canonical_root()
|
||||
"block_root" => %queued_rpc_block.beacon_block_root
|
||||
);
|
||||
if self
|
||||
.ready_work_tx
|
||||
@ -767,7 +743,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
}
|
||||
// A block that was queued for later processing is now ready to be processed.
|
||||
InboundEvent::ReadyGossipBlock(ready_block) => {
|
||||
let block_root = ready_block.block.block_root;
|
||||
let block_root = ready_block.beacon_block_root;
|
||||
|
||||
if !self.queued_gossip_block_roots.remove(&block_root) {
|
||||
// Log an error to alert that we've made a bad assumption about how this
|
||||
@ -885,18 +861,28 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
"millis_from_slot_start" => millis_from_slot_start
|
||||
);
|
||||
|
||||
if self
|
||||
match self
|
||||
.ready_work_tx
|
||||
.try_send(ReadyWork::BackfillSync(queued_backfill_batch.clone()))
|
||||
.is_err()
|
||||
.try_send(ReadyWork::BackfillSync(queued_backfill_batch))
|
||||
{
|
||||
// The message was sent successfully.
|
||||
Ok(()) => (),
|
||||
// The message was not sent, recover it from the returned `Err`.
|
||||
Err(mpsc::error::TrySendError::Full(ReadyWork::BackfillSync(batch)))
|
||||
| Err(mpsc::error::TrySendError::Closed(ReadyWork::BackfillSync(batch))) => {
|
||||
error!(
|
||||
log,
|
||||
"Failed to send scheduled backfill work";
|
||||
"info" => "sending work back to queue"
|
||||
);
|
||||
self.queued_backfill_batches
|
||||
.insert(0, queued_backfill_batch);
|
||||
self.queued_backfill_batches.insert(0, batch)
|
||||
}
|
||||
// The message was not sent and we didn't get the correct
|
||||
// return result. This is a logic error.
|
||||
_ => crit!(
|
||||
log,
|
||||
"Unexpected return from try_send error";
|
||||
),
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -927,7 +913,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
// 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::<T>::duration_until_next_backfill_batch_event(&self.slot_clock),
|
||||
ReprocessQueue::<S>::duration_until_next_backfill_batch_event(&self.slot_clock),
|
||||
)));
|
||||
} else {
|
||||
self.next_backfill_batch_event = None
|
||||
@ -936,7 +922,7 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
|
||||
/// Returns duration until the next scheduled processing time. The schedule ensure that backfill
|
||||
/// processing is done in windows of time that aren't critical
|
||||
fn duration_until_next_backfill_batch_event(slot_clock: &T::SlotClock) -> Duration {
|
||||
fn duration_until_next_backfill_batch_event(slot_clock: &S) -> Duration {
|
||||
let slot_duration = slot_clock.slot_duration();
|
||||
slot_clock
|
||||
.millis_from_current_slot_start()
|
||||
@ -966,16 +952,9 @@ impl<T: BeaconChainTypes> ReprocessQueue<T> {
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use beacon_chain::builder::Witness;
|
||||
use beacon_chain::eth1_chain::CachingEth1Backend;
|
||||
use slot_clock::TestingSlotClock;
|
||||
use store::MemoryStore;
|
||||
use types::MainnetEthSpec as E;
|
||||
use types::Slot;
|
||||
|
||||
type TestBeaconChainType =
|
||||
Witness<TestingSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
|
||||
|
||||
#[test]
|
||||
fn backfill_processing_schedule_calculation() {
|
||||
let slot_duration = Duration::from_secs(12);
|
||||
@ -988,7 +967,7 @@ mod tests {
|
||||
|
||||
for &event_duration_from_slot_start in event_times.iter() {
|
||||
let duration_to_next_event =
|
||||
ReprocessQueue::<TestBeaconChainType>::duration_until_next_backfill_batch_event(
|
||||
ReprocessQueue::<TestingSlotClock>::duration_until_next_backfill_batch_event(
|
||||
&slot_clock,
|
||||
);
|
||||
|
||||
@ -1005,7 +984,7 @@ mod tests {
|
||||
// check for next event beyond the current slot
|
||||
let duration_to_next_slot = slot_clock.duration_to_next_slot().unwrap();
|
||||
let duration_to_next_event =
|
||||
ReprocessQueue::<TestBeaconChainType>::duration_until_next_backfill_batch_event(
|
||||
ReprocessQueue::<TestingSlotClock>::duration_until_next_backfill_batch_event(
|
||||
&slot_clock,
|
||||
);
|
||||
assert_eq!(
|
@ -43,3 +43,5 @@ slasher = { path = "../../slasher" }
|
||||
slasher_service = { path = "../../slasher/service" }
|
||||
monitoring_api = {path = "../../common/monitoring_api"}
|
||||
execution_layer = { path = "../execution_layer" }
|
||||
beacon_processor = { path = "../beacon_processor" }
|
||||
num_cpus = "1.13.0"
|
||||
|
@ -13,6 +13,10 @@ use beacon_chain::{
|
||||
store::{HotColdDB, ItemStore, LevelDB, StoreConfig},
|
||||
BeaconChain, BeaconChainTypes, Eth1ChainBackend, ServerSentEventHandler,
|
||||
};
|
||||
use beacon_processor::{
|
||||
work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessor, BeaconProcessorSend,
|
||||
WorkEvent, MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN,
|
||||
};
|
||||
use environment::RuntimeContext;
|
||||
use eth1::{Config as Eth1Config, Service as Eth1Service};
|
||||
use eth2::{
|
||||
@ -27,12 +31,13 @@ use network::{NetworkConfig, NetworkSenders, NetworkService};
|
||||
use slasher::Slasher;
|
||||
use slasher_service::SlasherService;
|
||||
use slog::{debug, info, warn, Logger};
|
||||
use std::cmp;
|
||||
use std::net::TcpListener;
|
||||
use std::path::{Path, PathBuf};
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
use timer::spawn_timer;
|
||||
use tokio::sync::oneshot;
|
||||
use tokio::sync::{mpsc, oneshot};
|
||||
use types::{
|
||||
test_utils::generate_deterministic_keypairs, BeaconState, ChainSpec, EthSpec,
|
||||
ExecutionBlockHash, Hash256, SignedBeaconBlock,
|
||||
@ -72,6 +77,10 @@ pub struct ClientBuilder<T: BeaconChainTypes> {
|
||||
http_metrics_config: http_metrics::Config,
|
||||
slasher: Option<Arc<Slasher<T::EthSpec>>>,
|
||||
eth_spec_instance: T::EthSpec,
|
||||
beacon_processor_send: BeaconProcessorSend<T::EthSpec>,
|
||||
beacon_processor_receive: mpsc::Receiver<WorkEvent<T::EthSpec>>,
|
||||
work_reprocessing_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
work_reprocessing_rx: mpsc::Receiver<ReprocessQueueMessage>,
|
||||
}
|
||||
|
||||
impl<TSlotClock, TEth1Backend, TEthSpec, THotStore, TColdStore>
|
||||
@ -87,6 +96,10 @@ where
|
||||
///
|
||||
/// The `eth_spec_instance` parameter is used to concretize `TEthSpec`.
|
||||
pub fn new(eth_spec_instance: TEthSpec) -> Self {
|
||||
let (beacon_processor_send, beacon_processor_receive) =
|
||||
mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
|
||||
let (work_reprocessing_tx, work_reprocessing_rx) =
|
||||
mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
|
||||
Self {
|
||||
slot_clock: None,
|
||||
store: None,
|
||||
@ -104,6 +117,10 @@ where
|
||||
http_metrics_config: <_>::default(),
|
||||
slasher: None,
|
||||
eth_spec_instance,
|
||||
beacon_processor_send: BeaconProcessorSend(beacon_processor_send),
|
||||
beacon_processor_receive,
|
||||
work_reprocessing_tx,
|
||||
work_reprocessing_rx,
|
||||
}
|
||||
}
|
||||
|
||||
@ -568,6 +585,8 @@ where
|
||||
gossipsub_registry
|
||||
.as_mut()
|
||||
.map(|registry| registry.sub_registry_with_prefix("gossipsub")),
|
||||
self.beacon_processor_send.clone(),
|
||||
self.work_reprocessing_tx.clone(),
|
||||
)
|
||||
.await
|
||||
.map_err(|e| format!("Failed to start network: {:?}", e))?;
|
||||
@ -755,6 +774,27 @@ where
|
||||
}
|
||||
|
||||
if let Some(beacon_chain) = self.beacon_chain.as_ref() {
|
||||
if let Some(network_globals) = &self.network_globals {
|
||||
let beacon_processor_context = runtime_context.service_context("bproc".into());
|
||||
BeaconProcessor {
|
||||
network_globals: network_globals.clone(),
|
||||
executor: beacon_processor_context.executor.clone(),
|
||||
max_workers: cmp::max(1, num_cpus::get()),
|
||||
current_workers: 0,
|
||||
enable_backfill_rate_limiting: beacon_chain
|
||||
.config
|
||||
.enable_backfill_rate_limiting,
|
||||
log: beacon_processor_context.log().clone(),
|
||||
}
|
||||
.spawn_manager(
|
||||
self.beacon_processor_receive,
|
||||
self.work_reprocessing_tx,
|
||||
self.work_reprocessing_rx,
|
||||
None,
|
||||
beacon_chain.slot_clock.clone(),
|
||||
);
|
||||
}
|
||||
|
||||
let state_advance_context = runtime_context.service_context("state_advance".into());
|
||||
let state_advance_log = state_advance_context.log().clone();
|
||||
spawn_state_advance_timer(
|
||||
|
@ -11,7 +11,6 @@ matches = "0.1.8"
|
||||
exit-future = "0.2.0"
|
||||
slog-term = "2.6.0"
|
||||
slog-async = "2.5.0"
|
||||
environment = { path = "../../lighthouse/environment" }
|
||||
|
||||
[dependencies]
|
||||
beacon_chain = { path = "../beacon_chain" }
|
||||
@ -47,3 +46,6 @@ delay_map = "0.3.0"
|
||||
ethereum-types = { version = "0.14.1", optional = true }
|
||||
operation_pool = { path = "../operation_pool" }
|
||||
execution_layer = { path = "../execution_layer" }
|
||||
beacon_processor = { path = "../beacon_processor" }
|
||||
parking_lot = "0.12.0"
|
||||
environment = { path = "../../lighthouse/environment" }
|
@ -1,51 +0,0 @@
|
||||
use super::work_reprocessing_queue::ReprocessQueueMessage;
|
||||
use crate::{service::NetworkMessage, sync::SyncMessage};
|
||||
use beacon_chain::{BeaconChain, BeaconChainTypes};
|
||||
use slog::{debug, Logger};
|
||||
use std::sync::Arc;
|
||||
use tokio::sync::mpsc;
|
||||
|
||||
mod gossip_methods;
|
||||
mod rpc_methods;
|
||||
mod sync_methods;
|
||||
|
||||
pub use gossip_methods::{GossipAggregatePackage, GossipAttestationPackage};
|
||||
pub use sync_methods::ChainSegmentProcessId;
|
||||
|
||||
pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1;
|
||||
|
||||
/// Contains the context necessary to import blocks, attestations, etc to the beacon chain.
|
||||
pub struct Worker<T: BeaconChainTypes> {
|
||||
pub chain: Arc<BeaconChain<T>>,
|
||||
pub network_tx: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
|
||||
pub sync_tx: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
|
||||
pub log: Logger,
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> Worker<T> {
|
||||
/// Send a message to `sync_tx`.
|
||||
///
|
||||
/// Creates a log if there is an internal error.
|
||||
fn send_sync_message(&self, message: SyncMessage<T::EthSpec>) {
|
||||
self.sync_tx.send(message).unwrap_or_else(|e| {
|
||||
debug!(self.log, "Could not send message to the sync service";
|
||||
"error" => %e)
|
||||
});
|
||||
}
|
||||
|
||||
/// Send a message to `network_tx`.
|
||||
///
|
||||
/// Creates a log if there is an internal error.
|
||||
fn send_network_message(&self, message: NetworkMessage<T::EthSpec>) {
|
||||
self.network_tx.send(message).unwrap_or_else(|e| {
|
||||
debug!(self.log, "Could not send message to the network service. Likely shutdown";
|
||||
"error" => %e)
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
/// Contains the necessary items for a worker to do their job.
|
||||
pub struct Toolbox<T: BeaconChainTypes> {
|
||||
pub idle_tx: mpsc::Sender<()>,
|
||||
pub work_reprocessing_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
|
||||
}
|
@ -6,10 +6,10 @@ pub mod error;
|
||||
#[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy
|
||||
pub mod service;
|
||||
|
||||
mod beacon_processor;
|
||||
#[allow(clippy::mutable_key_type)] // PeerId in hashmaps are no longer permitted by clippy
|
||||
mod metrics;
|
||||
mod nat;
|
||||
mod network_beacon_processor;
|
||||
mod persisted_dht;
|
||||
mod router;
|
||||
mod status;
|
||||
|
@ -49,47 +49,8 @@ lazy_static! {
|
||||
/*
|
||||
* Gossip processor
|
||||
*/
|
||||
pub static ref BEACON_PROCESSOR_WORK_EVENTS_RX_COUNT: Result<IntCounterVec> = try_create_int_counter_vec(
|
||||
"beacon_processor_work_events_rx_count",
|
||||
"Count of work events received (but not necessarily processed)",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORK_EVENTS_IGNORED_COUNT: Result<IntCounterVec> = try_create_int_counter_vec(
|
||||
"beacon_processor_work_events_ignored_count",
|
||||
"Count of work events purposefully ignored",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORK_EVENTS_STARTED_COUNT: Result<IntCounterVec> = try_create_int_counter_vec(
|
||||
"beacon_processor_work_events_started_count",
|
||||
"Count of work events which have been started by a worker",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORKER_TIME: Result<HistogramVec> = try_create_histogram_vec(
|
||||
"beacon_processor_worker_time",
|
||||
"Time taken for a worker to fully process some parcel of work.",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORKERS_SPAWNED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_workers_spawned_total",
|
||||
"The number of workers ever spawned by the gossip processing pool."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_WORKERS_ACTIVE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_workers_active_total",
|
||||
"Count of active workers in the gossip processing pool."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_IDLE_EVENTS_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_idle_events_total",
|
||||
"Count of idle events processed by the gossip processor manager."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_EVENT_HANDLING_SECONDS: Result<Histogram> = try_create_histogram(
|
||||
"beacon_processor_event_handling_seconds",
|
||||
"Time spent handling a new message and allocating it to a queue or worker."
|
||||
);
|
||||
|
||||
// Gossip blocks.
|
||||
pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_gossip_block_queue_total",
|
||||
"Count of blocks from gossip waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_GOSSIP_BLOCK_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_gossip_block_verified_total",
|
||||
"Total number of gossip blocks verified for propagation."
|
||||
@ -107,10 +68,6 @@ lazy_static! {
|
||||
"Whenever a gossip block is received early this metrics is set to how early that block was."
|
||||
);
|
||||
// Gossip Exits.
|
||||
pub static ref BEACON_PROCESSOR_EXIT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_exit_queue_total",
|
||||
"Count of exits from gossip waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_EXIT_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_exit_verified_total",
|
||||
"Total number of voluntary exits verified for propagation."
|
||||
@ -120,10 +77,6 @@ lazy_static! {
|
||||
"Total number of voluntary exits imported to the op pool."
|
||||
);
|
||||
// Gossip proposer slashings.
|
||||
pub static ref BEACON_PROCESSOR_PROPOSER_SLASHING_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_proposer_slashing_queue_total",
|
||||
"Count of proposer slashings from gossip waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_PROPOSER_SLASHING_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_proposer_slashing_verified_total",
|
||||
"Total number of proposer slashings verified for propagation."
|
||||
@ -133,10 +86,6 @@ lazy_static! {
|
||||
"Total number of proposer slashings imported to the op pool."
|
||||
);
|
||||
// Gossip attester slashings.
|
||||
pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_attester_slashing_queue_total",
|
||||
"Count of attester slashings from gossip waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_ATTESTER_SLASHING_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_attester_slashing_verified_total",
|
||||
"Total number of attester slashings verified for propagation."
|
||||
@ -146,10 +95,6 @@ lazy_static! {
|
||||
"Total number of attester slashings imported to the op pool."
|
||||
);
|
||||
// Gossip BLS to execution changes.
|
||||
pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_bls_to_execution_change_queue_total",
|
||||
"Count of address changes from gossip waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_bls_to_execution_change_verified_total",
|
||||
"Total number of address changes verified for propagation."
|
||||
@ -159,23 +104,11 @@ lazy_static! {
|
||||
"Total number of address changes imported to the op pool."
|
||||
);
|
||||
// Rpc blocks.
|
||||
pub static ref BEACON_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_rpc_block_queue_total",
|
||||
"Count of blocks from the rpc waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_RPC_BLOCK_IMPORTED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_rpc_block_imported_total",
|
||||
"Total number of gossip blocks imported to fork choice, etc."
|
||||
);
|
||||
// Chain segments.
|
||||
pub static ref BEACON_PROCESSOR_CHAIN_SEGMENT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_chain_segment_queue_total",
|
||||
"Count of chain segments from the rpc waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_BACKFILL_CHAIN_SEGMENT_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_backfill_chain_segment_queue_total",
|
||||
"Count of backfill chain segments from the rpc waiting to be verified."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_CHAIN_SEGMENT_SUCCESS_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_chain_segment_success_total",
|
||||
"Total number of chain segments successfully processed."
|
||||
@ -193,10 +126,6 @@ lazy_static! {
|
||||
"Total number of backfill chain segments that failed processing."
|
||||
);
|
||||
// Unaggregated attestations.
|
||||
pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_unaggregated_attestation_queue_total",
|
||||
"Count of unagg. attestations waiting to be processed."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_unaggregated_attestation_verified_total",
|
||||
"Total number of unaggregated attestations verified for gossip."
|
||||
@ -210,10 +139,6 @@ lazy_static! {
|
||||
"Total number of unaggregated attestations that referenced an unknown block and were re-queued."
|
||||
);
|
||||
// Aggregated attestations.
|
||||
pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_aggregated_attestation_queue_total",
|
||||
"Count of agg. attestations waiting to be processed."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_AGGREGATED_ATTESTATION_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_aggregated_attestation_verified_total",
|
||||
"Total number of aggregated attestations verified for gossip."
|
||||
@ -227,10 +152,6 @@ lazy_static! {
|
||||
"Total number of aggregated attestations that referenced an unknown block and were re-queued."
|
||||
);
|
||||
// Sync committee messages.
|
||||
pub static ref BEACON_PROCESSOR_SYNC_MESSAGE_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_sync_message_queue_total",
|
||||
"Count of sync committee messages waiting to be processed."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_SYNC_MESSAGE_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_sync_message_verified_total",
|
||||
"Total number of sync committee messages verified for gossip."
|
||||
@ -240,10 +161,6 @@ lazy_static! {
|
||||
"Total number of sync committee messages imported to fork choice, etc."
|
||||
);
|
||||
// Sync contribution.
|
||||
pub static ref BEACON_PROCESSOR_SYNC_CONTRIBUTION_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
|
||||
"beacon_processor_sync_contribution_queue_total",
|
||||
"Count of sync committee contributions waiting to be processed."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_SYNC_CONTRIBUTION_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_sync_contribution_verified_total",
|
||||
"Total number of sync committee contributions verified for gossip."
|
||||
@ -279,12 +196,6 @@ lazy_static! {
|
||||
"Gossipsub light_client_optimistic_update errors per error type",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_SEND_ERROR_PER_WORK_TYPE: Result<IntCounterVec> =
|
||||
try_create_int_counter_vec(
|
||||
"beacon_processor_send_error_per_work_type",
|
||||
"Total number of beacon processor send error per work type",
|
||||
&["type"]
|
||||
);
|
||||
|
||||
|
||||
/*
|
||||
@ -371,35 +282,9 @@ lazy_static! {
|
||||
"Count of times when a gossip block arrived from the network later than the attestation deadline.",
|
||||
);
|
||||
|
||||
/*
|
||||
* Attestation reprocessing queue metrics.
|
||||
*/
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_TOTAL: Result<IntGaugeVec> =
|
||||
try_create_int_gauge_vec(
|
||||
"beacon_processor_reprocessing_queue_total",
|
||||
"Count of items in a reprocessing queue.",
|
||||
&["type"]
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_ATTESTATIONS: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_expired_attestations",
|
||||
"Number of queued attestations which have expired before a matching block has been found."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_ATTESTATIONS: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_matched_attestations",
|
||||
"Number of queued attestations where as matching block has been imported."
|
||||
);
|
||||
|
||||
/*
|
||||
* Light client update reprocessing queue metrics.
|
||||
*/
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_EXPIRED_OPTIMISTIC_UPDATES: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_expired_optimistic_updates",
|
||||
"Number of queued light client optimistic updates which have expired before a matching block has been found."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_MATCHED_OPTIMISTIC_UPDATES: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_matched_optimistic_updates",
|
||||
"Number of queued light client optimistic updates where as matching block has been imported."
|
||||
);
|
||||
pub static ref BEACON_PROCESSOR_REPROCESSING_QUEUE_SENT_OPTIMISTIC_UPDATES: Result<IntCounter> = try_create_int_counter(
|
||||
"beacon_processor_reprocessing_queue_sent_optimistic_updates",
|
||||
"Number of queued light client optimistic updates where as matching block has been imported."
|
||||
|
@ -1,4 +1,9 @@
|
||||
use crate::{metrics, service::NetworkMessage, sync::SyncMessage};
|
||||
use crate::{
|
||||
metrics,
|
||||
network_beacon_processor::{InvalidBlockStorage, NetworkBeaconProcessor},
|
||||
service::NetworkMessage,
|
||||
sync::SyncMessage,
|
||||
};
|
||||
|
||||
use beacon_chain::store::Error;
|
||||
use beacon_chain::{
|
||||
@ -30,14 +35,13 @@ use types::{
|
||||
SyncCommitteeMessage, SyncSubnetId,
|
||||
};
|
||||
|
||||
use super::{
|
||||
super::work_reprocessing_queue::{
|
||||
use beacon_processor::{
|
||||
work_reprocessing_queue::{
|
||||
QueuedAggregate, QueuedGossipBlock, QueuedLightClientUpdate, QueuedUnaggregate,
|
||||
ReprocessQueueMessage,
|
||||
},
|
||||
Worker,
|
||||
DuplicateCache, GossipAggregatePackage, GossipAttestationPackage,
|
||||
};
|
||||
use crate::beacon_processor::{DuplicateCache, InvalidBlockStorage};
|
||||
|
||||
/// Set to `true` to introduce stricter penalties for peers who send some types of late consensus
|
||||
/// messages.
|
||||
@ -144,65 +148,7 @@ impl<T: EthSpec> FailedAtt<T> {
|
||||
}
|
||||
}
|
||||
|
||||
/// Items required to verify a batch of unaggregated gossip attestations.
|
||||
#[derive(Debug)]
|
||||
pub struct GossipAttestationPackage<E: EthSpec> {
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
attestation: Box<Attestation<E>>,
|
||||
subnet_id: SubnetId,
|
||||
should_import: bool,
|
||||
seen_timestamp: Duration,
|
||||
}
|
||||
|
||||
impl<E: EthSpec> GossipAttestationPackage<E> {
|
||||
pub fn new(
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
attestation: Box<Attestation<E>>,
|
||||
subnet_id: SubnetId,
|
||||
should_import: bool,
|
||||
seen_timestamp: Duration,
|
||||
) -> Self {
|
||||
Self {
|
||||
message_id,
|
||||
peer_id,
|
||||
attestation,
|
||||
subnet_id,
|
||||
should_import,
|
||||
seen_timestamp,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Items required to verify a batch of aggregated gossip attestations.
|
||||
#[derive(Debug)]
|
||||
pub struct GossipAggregatePackage<E: EthSpec> {
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
aggregate: Box<SignedAggregateAndProof<E>>,
|
||||
beacon_block_root: Hash256,
|
||||
seen_timestamp: Duration,
|
||||
}
|
||||
|
||||
impl<E: EthSpec> GossipAggregatePackage<E> {
|
||||
pub fn new(
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
aggregate: Box<SignedAggregateAndProof<E>>,
|
||||
seen_timestamp: Duration,
|
||||
) -> Self {
|
||||
Self {
|
||||
message_id,
|
||||
peer_id,
|
||||
beacon_block_root: aggregate.message.aggregate.data.beacon_block_root,
|
||||
aggregate,
|
||||
seen_timestamp,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> Worker<T> {
|
||||
impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
|
||||
/* Auxiliary functions */
|
||||
|
||||
/// Penalizes a peer for misbehaviour.
|
||||
@ -245,13 +191,13 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
/// Raises a log if there are errors.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn process_gossip_attestation(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
attestation: Box<Attestation<T::EthSpec>>,
|
||||
subnet_id: SubnetId,
|
||||
should_import: bool,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
let result = match self
|
||||
@ -277,9 +223,9 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_attestation_batch(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
packages: Vec<GossipAttestationPackage<T::EthSpec>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
) {
|
||||
let attestations_and_subnets = packages
|
||||
.iter()
|
||||
@ -348,12 +294,12 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
// cant' be mixed-up) and creating a struct would result in more complexity.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
fn process_gossip_attestation_result(
|
||||
&self,
|
||||
self: &Arc<Self>,
|
||||
result: Result<VerifiedUnaggregate<T>, RejectedUnaggregate<T::EthSpec>>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
subnet_id: SubnetId,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
should_import: bool,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
@ -456,11 +402,11 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
///
|
||||
/// Raises a log if there are errors.
|
||||
pub fn process_gossip_aggregate(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
aggregate: Box<SignedAggregateAndProof<T::EthSpec>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root;
|
||||
@ -490,9 +436,9 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_aggregate_batch(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
packages: Vec<GossipAggregatePackage<T::EthSpec>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
) {
|
||||
let aggregates = packages.iter().map(|package| package.aggregate.as_ref());
|
||||
|
||||
@ -555,12 +501,12 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
fn process_gossip_aggregate_result(
|
||||
&self,
|
||||
self: &Arc<Self>,
|
||||
result: Result<VerifiedAggregate<T>, RejectedAggregate<T::EthSpec>>,
|
||||
beacon_block_root: Hash256,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
match result {
|
||||
@ -659,12 +605,12 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
/// Raises a log if there are errors.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn process_gossip_block(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
peer_client: Client,
|
||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
duplicate_cache: DuplicateCache,
|
||||
invalid_block_storage: InvalidBlockStorage,
|
||||
seen_duration: Duration,
|
||||
@ -708,12 +654,12 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
///
|
||||
/// Returns the `GossipVerifiedBlock` if verification passes and raises a log if there are errors.
|
||||
pub async fn process_gossip_unverified_block(
|
||||
&self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
peer_client: Client,
|
||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
seen_duration: Duration,
|
||||
) -> Option<GossipVerifiedBlock<T>> {
|
||||
let block_delay =
|
||||
@ -911,11 +857,25 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
|
||||
metrics::inc_counter(&metrics::BEACON_PROCESSOR_GOSSIP_BLOCK_REQUEUED_TOTAL);
|
||||
|
||||
let inner_self = self.clone();
|
||||
let process_fn = Box::pin(async move {
|
||||
let reprocess_tx = inner_self.reprocess_tx.clone();
|
||||
let invalid_block_storage = inner_self.invalid_block_storage.clone();
|
||||
inner_self
|
||||
.process_gossip_verified_block(
|
||||
peer_id,
|
||||
verified_block,
|
||||
reprocess_tx,
|
||||
invalid_block_storage,
|
||||
seen_duration,
|
||||
)
|
||||
.await;
|
||||
});
|
||||
if reprocess_tx
|
||||
.try_send(ReprocessQueueMessage::EarlyBlock(QueuedGossipBlock {
|
||||
peer_id,
|
||||
block: Box::new(verified_block),
|
||||
seen_timestamp: seen_duration,
|
||||
beacon_block_slot: block_slot,
|
||||
beacon_block_root: block_root,
|
||||
process_fn,
|
||||
}))
|
||||
.is_err()
|
||||
{
|
||||
@ -948,10 +908,10 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
///
|
||||
/// Raises a log if there are errors.
|
||||
pub async fn process_gossip_verified_block(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
peer_id: PeerId,
|
||||
verified_block: GossipVerifiedBlock<T>,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
invalid_block_storage: InvalidBlockStorage,
|
||||
// This value is not used presently, but it might come in handy for debugging.
|
||||
_seen_duration: Duration,
|
||||
@ -1051,7 +1011,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_voluntary_exit(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
voluntary_exit: SignedVoluntaryExit,
|
||||
@ -1109,7 +1069,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_proposer_slashing(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
proposer_slashing: ProposerSlashing,
|
||||
@ -1171,7 +1131,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_attester_slashing(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
attester_slashing: AttesterSlashing<T::EthSpec>,
|
||||
@ -1225,7 +1185,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_bls_to_execution_change(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
bls_to_execution_change: SignedBlsToExecutionChange,
|
||||
@ -1308,7 +1268,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
///
|
||||
/// Raises a log if there are errors.
|
||||
pub fn process_gossip_sync_committee_signature(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
sync_signature: SyncCommitteeMessage,
|
||||
@ -1371,7 +1331,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
///
|
||||
/// Raises a log if there are errors.
|
||||
pub fn process_sync_committee_contribution(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
sync_contribution: SignedContributionAndProof<T::EthSpec>,
|
||||
@ -1426,7 +1386,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_finality_update(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
light_client_finality_update: LightClientFinalityUpdate<T::EthSpec>,
|
||||
@ -1492,11 +1452,11 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
}
|
||||
|
||||
pub fn process_gossip_optimistic_update(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
light_client_optimistic_update: LightClientOptimisticUpdate<T::EthSpec>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
match self.chain.verify_optimistic_update_for_gossip(
|
||||
@ -1527,15 +1487,19 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
);
|
||||
|
||||
if let Some(sender) = reprocess_tx {
|
||||
let processor = self.clone();
|
||||
let msg = ReprocessQueueMessage::UnknownLightClientOptimisticUpdate(
|
||||
QueuedLightClientUpdate {
|
||||
peer_id,
|
||||
message_id,
|
||||
light_client_optimistic_update: Box::new(
|
||||
light_client_optimistic_update,
|
||||
),
|
||||
parent_root,
|
||||
process_fn: Box::new(move || {
|
||||
processor.process_gossip_optimistic_update(
|
||||
message_id,
|
||||
peer_id,
|
||||
light_client_optimistic_update,
|
||||
None, // Do not reprocess this message again.
|
||||
seen_timestamp,
|
||||
)
|
||||
}),
|
||||
},
|
||||
);
|
||||
|
||||
@ -1624,11 +1588,11 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
/// Handle an error whilst verifying an `Attestation` or `SignedAggregateAndProof` from the
|
||||
/// network.
|
||||
fn handle_attestation_verification_failure(
|
||||
&self,
|
||||
self: &Arc<Self>,
|
||||
peer_id: PeerId,
|
||||
message_id: MessageId,
|
||||
failed_att: FailedAtt<T::EthSpec>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage<T>>>,
|
||||
reprocess_tx: Option<mpsc::Sender<ReprocessQueueMessage>>,
|
||||
error: AttnError,
|
||||
seen_timestamp: Duration,
|
||||
) {
|
||||
@ -1860,11 +1824,18 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
metrics::inc_counter(
|
||||
&metrics::BEACON_PROCESSOR_AGGREGATED_ATTESTATION_REQUEUED_TOTAL,
|
||||
);
|
||||
let processor = self.clone();
|
||||
ReprocessQueueMessage::UnknownBlockAggregate(QueuedAggregate {
|
||||
peer_id,
|
||||
beacon_block_root: *beacon_block_root,
|
||||
process_fn: Box::new(move || {
|
||||
processor.process_gossip_aggregate(
|
||||
message_id,
|
||||
peer_id,
|
||||
attestation,
|
||||
None, // Do not allow this attestation to be re-processed beyond this point.
|
||||
seen_timestamp,
|
||||
)
|
||||
}),
|
||||
})
|
||||
}
|
||||
FailedAtt::Unaggregate {
|
||||
@ -1876,13 +1847,20 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
metrics::inc_counter(
|
||||
&metrics::BEACON_PROCESSOR_UNAGGREGATED_ATTESTATION_REQUEUED_TOTAL,
|
||||
);
|
||||
let processor = self.clone();
|
||||
ReprocessQueueMessage::UnknownBlockUnaggregate(QueuedUnaggregate {
|
||||
peer_id,
|
||||
beacon_block_root: *beacon_block_root,
|
||||
process_fn: Box::new(move || {
|
||||
processor.process_gossip_attestation(
|
||||
message_id,
|
||||
peer_id,
|
||||
attestation,
|
||||
subnet_id,
|
||||
should_import,
|
||||
None, // Do not allow this attestation to be re-processed beyond this point.
|
||||
seen_timestamp,
|
||||
)
|
||||
}),
|
||||
})
|
||||
}
|
||||
};
|
590
beacon_node/network/src/network_beacon_processor/mod.rs
Normal file
590
beacon_node/network/src/network_beacon_processor/mod.rs
Normal file
@ -0,0 +1,590 @@
|
||||
use crate::{
|
||||
service::NetworkMessage,
|
||||
sync::{manager::BlockProcessType, SyncMessage},
|
||||
};
|
||||
use beacon_chain::{
|
||||
builder::Witness, eth1_chain::CachingEth1Backend, test_utils::BeaconChainHarness, BeaconChain,
|
||||
};
|
||||
use beacon_chain::{BeaconChainTypes, NotifyExecutionLayer};
|
||||
use beacon_processor::{
|
||||
work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend, DuplicateCache,
|
||||
GossipAggregatePackage, GossipAttestationPackage, Work, WorkEvent as BeaconWorkEvent,
|
||||
MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN,
|
||||
};
|
||||
use environment::null_logger;
|
||||
use lighthouse_network::{
|
||||
rpc::{BlocksByRangeRequest, BlocksByRootRequest, LightClientBootstrapRequest, StatusMessage},
|
||||
Client, MessageId, NetworkGlobals, PeerId, PeerRequestId,
|
||||
};
|
||||
use slog::{debug, Logger};
|
||||
use slot_clock::ManualSlotClock;
|
||||
use std::path::PathBuf;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
use store::MemoryStore;
|
||||
use task_executor::test_utils::TestRuntime;
|
||||
use task_executor::TaskExecutor;
|
||||
use tokio::sync::mpsc::{self, error::TrySendError};
|
||||
use types::*;
|
||||
|
||||
pub use sync_methods::ChainSegmentProcessId;
|
||||
|
||||
pub type Error<T> = TrySendError<BeaconWorkEvent<T>>;
|
||||
|
||||
mod gossip_methods;
|
||||
mod rpc_methods;
|
||||
mod sync_methods;
|
||||
mod tests;
|
||||
|
||||
pub(crate) const FUTURE_SLOT_TOLERANCE: u64 = 1;
|
||||
|
||||
/// Defines if and where we will store the SSZ files of invalid blocks.
|
||||
#[derive(Clone)]
|
||||
pub enum InvalidBlockStorage {
|
||||
Enabled(PathBuf),
|
||||
Disabled,
|
||||
}
|
||||
|
||||
/// Provides an interface to a `BeaconProcessor` running in some other thread.
|
||||
/// The wider `networking` crate should use this struct to interface with the
|
||||
/// beacon processor.
|
||||
pub struct NetworkBeaconProcessor<T: BeaconChainTypes> {
|
||||
pub beacon_processor_send: BeaconProcessorSend<T::EthSpec>,
|
||||
pub duplicate_cache: DuplicateCache,
|
||||
pub chain: Arc<BeaconChain<T>>,
|
||||
pub network_tx: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
|
||||
pub sync_tx: mpsc::UnboundedSender<SyncMessage<T::EthSpec>>,
|
||||
pub reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
pub network_globals: Arc<NetworkGlobals<T::EthSpec>>,
|
||||
pub invalid_block_storage: InvalidBlockStorage,
|
||||
pub executor: TaskExecutor,
|
||||
pub log: Logger,
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
|
||||
fn try_send(&self, event: BeaconWorkEvent<T::EthSpec>) -> Result<(), Error<T::EthSpec>> {
|
||||
self.beacon_processor_send
|
||||
.try_send(event)
|
||||
.map_err(Into::into)
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some unaggregated attestation.
|
||||
pub fn send_unaggregated_attestation(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
attestation: Attestation<T::EthSpec>,
|
||||
subnet_id: SubnetId,
|
||||
should_import: bool,
|
||||
seen_timestamp: Duration,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
// Define a closure for processing individual attestations.
|
||||
let processor = self.clone();
|
||||
let process_individual = move |package: GossipAttestationPackage<T::EthSpec>| {
|
||||
let reprocess_tx = processor.reprocess_tx.clone();
|
||||
processor.process_gossip_attestation(
|
||||
package.message_id,
|
||||
package.peer_id,
|
||||
package.attestation,
|
||||
package.subnet_id,
|
||||
package.should_import,
|
||||
Some(reprocess_tx),
|
||||
package.seen_timestamp,
|
||||
)
|
||||
};
|
||||
|
||||
// Define a closure for processing batches of attestations.
|
||||
let processor = self.clone();
|
||||
let process_batch = move |attestations| {
|
||||
let reprocess_tx = processor.reprocess_tx.clone();
|
||||
processor.process_gossip_attestation_batch(attestations, Some(reprocess_tx))
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: true,
|
||||
work: Work::GossipAttestation {
|
||||
attestation: GossipAttestationPackage {
|
||||
message_id,
|
||||
peer_id,
|
||||
attestation: Box::new(attestation),
|
||||
subnet_id,
|
||||
should_import,
|
||||
seen_timestamp,
|
||||
},
|
||||
process_individual: Box::new(process_individual),
|
||||
process_batch: Box::new(process_batch),
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some aggregated attestation.
|
||||
pub fn send_aggregated_attestation(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
aggregate: SignedAggregateAndProof<T::EthSpec>,
|
||||
seen_timestamp: Duration,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
// Define a closure for processing individual attestations.
|
||||
let processor = self.clone();
|
||||
let process_individual = move |package: GossipAggregatePackage<T::EthSpec>| {
|
||||
let reprocess_tx = processor.reprocess_tx.clone();
|
||||
processor.process_gossip_aggregate(
|
||||
package.message_id,
|
||||
package.peer_id,
|
||||
package.aggregate,
|
||||
Some(reprocess_tx),
|
||||
package.seen_timestamp,
|
||||
)
|
||||
};
|
||||
|
||||
// Define a closure for processing batches of attestations.
|
||||
let processor = self.clone();
|
||||
let process_batch = move |aggregates| {
|
||||
let reprocess_tx = processor.reprocess_tx.clone();
|
||||
processor.process_gossip_aggregate_batch(aggregates, Some(reprocess_tx))
|
||||
};
|
||||
|
||||
let beacon_block_root = aggregate.message.aggregate.data.beacon_block_root;
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: true,
|
||||
work: Work::GossipAggregate {
|
||||
aggregate: GossipAggregatePackage {
|
||||
message_id,
|
||||
peer_id,
|
||||
aggregate: Box::new(aggregate),
|
||||
beacon_block_root,
|
||||
seen_timestamp,
|
||||
},
|
||||
process_individual: Box::new(process_individual),
|
||||
process_batch: Box::new(process_batch),
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some block.
|
||||
pub fn send_gossip_beacon_block(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
peer_client: Client,
|
||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = async move {
|
||||
let reprocess_tx = processor.reprocess_tx.clone();
|
||||
let invalid_block_storage = processor.invalid_block_storage.clone();
|
||||
let duplicate_cache = processor.duplicate_cache.clone();
|
||||
processor
|
||||
.process_gossip_block(
|
||||
message_id,
|
||||
peer_id,
|
||||
peer_client,
|
||||
block,
|
||||
reprocess_tx,
|
||||
duplicate_cache,
|
||||
invalid_block_storage,
|
||||
seen_timestamp,
|
||||
)
|
||||
.await
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::GossipBlock(Box::pin(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some sync committee signature.
|
||||
pub fn send_gossip_sync_signature(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
sync_signature: SyncCommitteeMessage,
|
||||
subnet_id: SyncSubnetId,
|
||||
seen_timestamp: Duration,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || {
|
||||
processor.process_gossip_sync_committee_signature(
|
||||
message_id,
|
||||
peer_id,
|
||||
sync_signature,
|
||||
subnet_id,
|
||||
seen_timestamp,
|
||||
)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: true,
|
||||
work: Work::GossipSyncSignature(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some sync committee contribution.
|
||||
pub fn send_gossip_sync_contribution(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
sync_contribution: SignedContributionAndProof<T::EthSpec>,
|
||||
seen_timestamp: Duration,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || {
|
||||
processor.process_sync_committee_contribution(
|
||||
message_id,
|
||||
peer_id,
|
||||
sync_contribution,
|
||||
seen_timestamp,
|
||||
)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: true,
|
||||
work: Work::GossipSyncContribution(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some exit.
|
||||
pub fn send_gossip_voluntary_exit(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
voluntary_exit: Box<SignedVoluntaryExit>,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn =
|
||||
move || processor.process_gossip_voluntary_exit(message_id, peer_id, *voluntary_exit);
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::GossipVoluntaryExit(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some proposer slashing.
|
||||
pub fn send_gossip_proposer_slashing(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
proposer_slashing: Box<ProposerSlashing>,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || {
|
||||
processor.process_gossip_proposer_slashing(message_id, peer_id, *proposer_slashing)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::GossipProposerSlashing(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some light client finality update.
|
||||
pub fn send_gossip_light_client_finality_update(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
light_client_finality_update: LightClientFinalityUpdate<T::EthSpec>,
|
||||
seen_timestamp: Duration,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || {
|
||||
processor.process_gossip_finality_update(
|
||||
message_id,
|
||||
peer_id,
|
||||
light_client_finality_update,
|
||||
seen_timestamp,
|
||||
)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: true,
|
||||
work: Work::GossipLightClientFinalityUpdate(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some light client optimistic update.
|
||||
pub fn send_gossip_light_client_optimistic_update(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
light_client_optimistic_update: LightClientOptimisticUpdate<T::EthSpec>,
|
||||
seen_timestamp: Duration,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || {
|
||||
let reprocess_tx = processor.reprocess_tx.clone();
|
||||
processor.process_gossip_optimistic_update(
|
||||
message_id,
|
||||
peer_id,
|
||||
light_client_optimistic_update,
|
||||
Some(reprocess_tx),
|
||||
seen_timestamp,
|
||||
)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: true,
|
||||
work: Work::GossipLightClientOptimisticUpdate(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some attester slashing.
|
||||
pub fn send_gossip_attester_slashing(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
attester_slashing: Box<AttesterSlashing<T::EthSpec>>,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || {
|
||||
processor.process_gossip_attester_slashing(message_id, peer_id, *attester_slashing)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::GossipAttesterSlashing(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some BLS to execution change.
|
||||
pub fn send_gossip_bls_to_execution_change(
|
||||
self: &Arc<Self>,
|
||||
message_id: MessageId,
|
||||
peer_id: PeerId,
|
||||
bls_to_execution_change: Box<SignedBlsToExecutionChange>,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || {
|
||||
processor.process_gossip_bls_to_execution_change(
|
||||
message_id,
|
||||
peer_id,
|
||||
*bls_to_execution_change,
|
||||
)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::GossipBlsToExecutionChange(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new `Work` event for some block, where the result from computation (if any) is
|
||||
/// sent to the other side of `result_tx`.
|
||||
pub fn send_rpc_beacon_block(
|
||||
self: &Arc<Self>,
|
||||
block_root: Hash256,
|
||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
process_type: BlockProcessType,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let process_fn = self.clone().generate_rpc_beacon_block_process_fn(
|
||||
block_root,
|
||||
block,
|
||||
seen_timestamp,
|
||||
process_type,
|
||||
);
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::RpcBlock { process_fn },
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new work event to import `blocks` as a beacon chain segment.
|
||||
pub fn send_chain_segment(
|
||||
self: &Arc<Self>,
|
||||
process_id: ChainSegmentProcessId,
|
||||
blocks: Vec<Arc<SignedBeaconBlock<T::EthSpec>>>,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let is_backfill = matches!(&process_id, ChainSegmentProcessId::BackSyncBatchId { .. });
|
||||
let processor = self.clone();
|
||||
let process_fn = async move {
|
||||
let notify_execution_layer = if processor
|
||||
.network_globals
|
||||
.sync_state
|
||||
.read()
|
||||
.is_syncing_finalized()
|
||||
{
|
||||
NotifyExecutionLayer::No
|
||||
} else {
|
||||
NotifyExecutionLayer::Yes
|
||||
};
|
||||
processor
|
||||
.process_chain_segment(process_id, blocks, notify_execution_layer)
|
||||
.await;
|
||||
};
|
||||
let process_fn = Box::pin(process_fn);
|
||||
|
||||
// Back-sync batches are dispatched with a different `Work` variant so
|
||||
// they can be rate-limited.
|
||||
let work = if is_backfill {
|
||||
Work::ChainSegmentBackfill(process_fn)
|
||||
} else {
|
||||
Work::ChainSegment(process_fn)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work,
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new work event to process `StatusMessage`s from the RPC network.
|
||||
pub fn send_status_message(
|
||||
self: &Arc<Self>,
|
||||
peer_id: PeerId,
|
||||
message: StatusMessage,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move || processor.process_status(peer_id, message);
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::Status(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new work event to process `BlocksByRangeRequest`s from the RPC network.
|
||||
pub fn send_blocks_by_range_request(
|
||||
self: &Arc<Self>,
|
||||
peer_id: PeerId,
|
||||
request_id: PeerRequestId,
|
||||
request: BlocksByRangeRequest,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move |send_idle_on_drop| {
|
||||
let executor = processor.executor.clone();
|
||||
processor.handle_blocks_by_range_request(
|
||||
executor,
|
||||
send_idle_on_drop,
|
||||
peer_id,
|
||||
request_id,
|
||||
request,
|
||||
)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::BlocksByRangeRequest(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new work event to process `BlocksByRootRequest`s from the RPC network.
|
||||
pub fn send_blocks_by_roots_request(
|
||||
self: &Arc<Self>,
|
||||
peer_id: PeerId,
|
||||
request_id: PeerRequestId,
|
||||
request: BlocksByRootRequest,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn = move |send_idle_on_drop| {
|
||||
let executor = processor.executor.clone();
|
||||
processor.handle_blocks_by_root_request(
|
||||
executor,
|
||||
send_idle_on_drop,
|
||||
peer_id,
|
||||
request_id,
|
||||
request,
|
||||
)
|
||||
};
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: false,
|
||||
work: Work::BlocksByRootsRequest(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Create a new work event to process `LightClientBootstrap`s from the RPC network.
|
||||
pub fn send_lightclient_bootstrap_request(
|
||||
self: &Arc<Self>,
|
||||
peer_id: PeerId,
|
||||
request_id: PeerRequestId,
|
||||
request: LightClientBootstrapRequest,
|
||||
) -> Result<(), Error<T::EthSpec>> {
|
||||
let processor = self.clone();
|
||||
let process_fn =
|
||||
move || processor.handle_light_client_bootstrap(peer_id, request_id, request);
|
||||
|
||||
self.try_send(BeaconWorkEvent {
|
||||
drop_during_sync: true,
|
||||
work: Work::LightClientBootstrapRequest(Box::new(process_fn)),
|
||||
})
|
||||
}
|
||||
|
||||
/// Send a message to `sync_tx`.
|
||||
///
|
||||
/// Creates a log if there is an internal error.
|
||||
fn send_sync_message(&self, message: SyncMessage<T::EthSpec>) {
|
||||
self.sync_tx.send(message).unwrap_or_else(|e| {
|
||||
debug!(self.log, "Could not send message to the sync service";
|
||||
"error" => %e)
|
||||
});
|
||||
}
|
||||
|
||||
/// Send a message to `network_tx`.
|
||||
///
|
||||
/// Creates a log if there is an internal error.
|
||||
fn send_network_message(&self, message: NetworkMessage<T::EthSpec>) {
|
||||
self.network_tx.send(message).unwrap_or_else(|e| {
|
||||
debug!(self.log, "Could not send message to the network service. Likely shutdown";
|
||||
"error" => %e)
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
type TestBeaconChainType<E> =
|
||||
Witness<ManualSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
|
||||
|
||||
impl<E: EthSpec> NetworkBeaconProcessor<TestBeaconChainType<E>> {
|
||||
// Instantiates a mostly non-functional version of `Self` and returns the
|
||||
// event receiver that would normally go to the beacon processor. This is
|
||||
// useful for testing that messages are actually being sent to the beacon
|
||||
// processor (but not much else).
|
||||
pub fn null_for_testing(
|
||||
network_globals: Arc<NetworkGlobals<E>>,
|
||||
) -> (Self, mpsc::Receiver<BeaconWorkEvent<E>>) {
|
||||
let (beacon_processor_send, beacon_processor_receive) =
|
||||
mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
|
||||
let (network_tx, _network_rx) = mpsc::unbounded_channel();
|
||||
let (sync_tx, _sync_rx) = mpsc::unbounded_channel();
|
||||
let (reprocess_tx, _reprocess_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
|
||||
let log = null_logger().unwrap();
|
||||
let harness: BeaconChainHarness<TestBeaconChainType<E>> =
|
||||
BeaconChainHarness::builder(E::default())
|
||||
.spec(E::default_spec())
|
||||
.deterministic_keypairs(8)
|
||||
.logger(log.clone())
|
||||
.fresh_ephemeral_store()
|
||||
.mock_execution_layer()
|
||||
.build();
|
||||
let runtime = TestRuntime::default();
|
||||
|
||||
let network_beacon_processor = Self {
|
||||
beacon_processor_send: BeaconProcessorSend(beacon_processor_send),
|
||||
duplicate_cache: DuplicateCache::default(),
|
||||
chain: harness.chain,
|
||||
network_tx,
|
||||
sync_tx,
|
||||
reprocess_tx,
|
||||
network_globals,
|
||||
invalid_block_storage: InvalidBlockStorage::Disabled,
|
||||
executor: runtime.task_executor.clone(),
|
||||
log,
|
||||
};
|
||||
|
||||
(network_beacon_processor, beacon_processor_receive)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
#[test]
|
||||
fn queued_block_delay_is_sane() {
|
||||
assert!(
|
||||
beacon_processor::work_reprocessing_queue::ADDITIONAL_QUEUED_BLOCK_DELAY
|
||||
< beacon_chain::MAXIMUM_GOSSIP_CLOCK_DISPARITY
|
||||
);
|
||||
}
|
||||
}
|
@ -1,21 +1,21 @@
|
||||
use crate::beacon_processor::{worker::FUTURE_SLOT_TOLERANCE, SendOnDrop};
|
||||
use crate::network_beacon_processor::{NetworkBeaconProcessor, FUTURE_SLOT_TOLERANCE};
|
||||
use crate::service::NetworkMessage;
|
||||
use crate::status::ToStatusMessage;
|
||||
use crate::sync::SyncMessage;
|
||||
use beacon_chain::{BeaconChainError, BeaconChainTypes, HistoricalBlockError, WhenSlotSkipped};
|
||||
use beacon_processor::SendOnDrop;
|
||||
use itertools::process_results;
|
||||
use lighthouse_network::rpc::StatusMessage;
|
||||
use lighthouse_network::rpc::*;
|
||||
use lighthouse_network::{PeerId, PeerRequestId, ReportSource, Response, SyncInfo};
|
||||
use slog::{debug, error, warn};
|
||||
use slot_clock::SlotClock;
|
||||
use std::sync::Arc;
|
||||
use task_executor::TaskExecutor;
|
||||
use tokio_stream::StreamExt;
|
||||
use types::{light_client_bootstrap::LightClientBootstrap, Epoch, EthSpec, Hash256, Slot};
|
||||
|
||||
use super::Worker;
|
||||
|
||||
impl<T: BeaconChainTypes> Worker<T> {
|
||||
impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
|
||||
/* Auxiliary functions */
|
||||
|
||||
/// Disconnects and ban's a peer, sending a Goodbye request with the associated reason.
|
||||
@ -124,7 +124,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
|
||||
/// Handle a `BlocksByRoot` request from the peer.
|
||||
pub fn handle_blocks_by_root_request(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
executor: TaskExecutor,
|
||||
send_on_drop: SendOnDrop,
|
||||
peer_id: PeerId,
|
||||
@ -210,7 +210,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
|
||||
/// Handle a `BlocksByRoot` request from the peer.
|
||||
pub fn handle_light_client_bootstrap(
|
||||
self,
|
||||
self: &Arc<Self>,
|
||||
peer_id: PeerId,
|
||||
request_id: PeerRequestId,
|
||||
request: LightClientBootstrapRequest,
|
||||
@ -283,7 +283,7 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
|
||||
/// Handle a `BlocksByRange` request from the peer.
|
||||
pub fn handle_blocks_by_range_request(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
executor: TaskExecutor,
|
||||
send_on_drop: SendOnDrop,
|
||||
peer_id: PeerId,
|
@ -1,17 +1,21 @@
|
||||
use std::time::Duration;
|
||||
|
||||
use super::{super::work_reprocessing_queue::ReprocessQueueMessage, Worker};
|
||||
use crate::beacon_processor::work_reprocessing_queue::QueuedRpcBlock;
|
||||
use crate::beacon_processor::worker::FUTURE_SLOT_TOLERANCE;
|
||||
use crate::beacon_processor::DuplicateCache;
|
||||
use crate::metrics;
|
||||
use crate::sync::manager::{BlockProcessType, SyncMessage};
|
||||
use crate::sync::{BatchProcessResult, ChainId};
|
||||
use crate::network_beacon_processor::{NetworkBeaconProcessor, FUTURE_SLOT_TOLERANCE};
|
||||
use crate::sync::BatchProcessResult;
|
||||
use crate::sync::{
|
||||
manager::{BlockProcessType, SyncMessage},
|
||||
ChainId,
|
||||
};
|
||||
use beacon_chain::{
|
||||
observed_block_producers::Error as ObserveError, validator_monitor::get_block_delay_ms,
|
||||
BeaconChainError, BeaconChainTypes, BlockError, ChainSegmentResult, HistoricalBlockError,
|
||||
NotifyExecutionLayer,
|
||||
};
|
||||
use beacon_processor::{
|
||||
work_reprocessing_queue::{QueuedRpcBlock, ReprocessQueueMessage},
|
||||
AsyncFn, BlockingFn, DuplicateCache,
|
||||
};
|
||||
use lighthouse_network::PeerAction;
|
||||
use slog::{debug, error, info, warn};
|
||||
use slot_clock::SlotClock;
|
||||
@ -39,27 +43,71 @@ struct ChainSegmentFailed {
|
||||
peer_action: Option<PeerAction>,
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> Worker<T> {
|
||||
/// Attempt to process a block received from a direct RPC request.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn process_rpc_block(
|
||||
self,
|
||||
impl<T: BeaconChainTypes> NetworkBeaconProcessor<T> {
|
||||
/// Returns an async closure which processes a beacon block recieved via RPC.
|
||||
///
|
||||
/// This separate function was required to prevent a cycle during compiler
|
||||
/// type checking.
|
||||
pub fn generate_rpc_beacon_block_process_fn(
|
||||
self: Arc<Self>,
|
||||
block_root: Hash256,
|
||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
process_type: BlockProcessType,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage<T>>,
|
||||
duplicate_cache: DuplicateCache,
|
||||
should_process: bool,
|
||||
) {
|
||||
if !should_process {
|
||||
) -> AsyncFn {
|
||||
let process_fn = async move {
|
||||
let reprocess_tx = self.reprocess_tx.clone();
|
||||
let duplicate_cache = self.duplicate_cache.clone();
|
||||
self.process_rpc_block(
|
||||
block_root,
|
||||
block,
|
||||
seen_timestamp,
|
||||
process_type,
|
||||
reprocess_tx,
|
||||
duplicate_cache,
|
||||
)
|
||||
.await;
|
||||
};
|
||||
Box::pin(process_fn)
|
||||
}
|
||||
|
||||
/// Returns the `process_fn` and `ignore_fn` required when requeuing an RPC block.
|
||||
pub fn generate_rpc_beacon_block_fns(
|
||||
self: Arc<Self>,
|
||||
block_root: Hash256,
|
||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
process_type: BlockProcessType,
|
||||
) -> (AsyncFn, BlockingFn) {
|
||||
// An async closure which will import the block.
|
||||
let process_fn = self.clone().generate_rpc_beacon_block_process_fn(
|
||||
block_root,
|
||||
block,
|
||||
seen_timestamp,
|
||||
process_type.clone(),
|
||||
);
|
||||
// A closure which will ignore the block.
|
||||
let ignore_fn = move || {
|
||||
// Sync handles these results
|
||||
self.send_sync_message(SyncMessage::BlockProcessed {
|
||||
process_type,
|
||||
result: crate::sync::manager::BlockProcessResult::Ignored,
|
||||
});
|
||||
return;
|
||||
};
|
||||
(process_fn, Box::new(ignore_fn))
|
||||
}
|
||||
|
||||
/// Attempt to process a block received from a direct RPC request.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub async fn process_rpc_block(
|
||||
self: Arc<NetworkBeaconProcessor<T>>,
|
||||
block_root: Hash256,
|
||||
block: Arc<SignedBeaconBlock<T::EthSpec>>,
|
||||
seen_timestamp: Duration,
|
||||
process_type: BlockProcessType,
|
||||
reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
duplicate_cache: DuplicateCache,
|
||||
) {
|
||||
// Check if the block is already being imported through another source
|
||||
let handle = match duplicate_cache.check_and_insert(block_root) {
|
||||
Some(handle) => handle,
|
||||
@ -70,13 +118,18 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
"action" => "sending rpc block to reprocessing queue",
|
||||
"block_root" => %block_root,
|
||||
);
|
||||
|
||||
// Send message to work reprocess queue to retry the block
|
||||
let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock {
|
||||
let (process_fn, ignore_fn) = self.clone().generate_rpc_beacon_block_fns(
|
||||
block_root,
|
||||
block: block.clone(),
|
||||
process_type,
|
||||
block,
|
||||
seen_timestamp,
|
||||
should_process: true,
|
||||
process_type,
|
||||
);
|
||||
let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock {
|
||||
beacon_block_root: block_root,
|
||||
process_fn,
|
||||
ignore_fn,
|
||||
});
|
||||
|
||||
if reprocess_tx.try_send(reprocess_msg).is_err() {
|
||||
@ -130,12 +183,16 @@ impl<T: BeaconChainTypes> Worker<T> {
|
||||
);
|
||||
|
||||
// Send message to work reprocess queue to retry the block
|
||||
let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock {
|
||||
let (process_fn, ignore_fn) = self.clone().generate_rpc_beacon_block_fns(
|
||||
block_root,
|
||||
block: block.clone(),
|
||||
process_type,
|
||||
block,
|
||||
seen_timestamp,
|
||||
should_process: true,
|
||||
process_type,
|
||||
);
|
||||
let reprocess_msg = ReprocessQueueMessage::RpcBlock(QueuedRpcBlock {
|
||||
beacon_block_root: block_root,
|
||||
process_fn,
|
||||
ignore_fn,
|
||||
});
|
||||
|
||||
if reprocess_tx.try_send(reprocess_msg).is_err() {
|
@ -1,20 +1,23 @@
|
||||
#![cfg(not(debug_assertions))] // Tests are too slow in debug.
|
||||
#![cfg(test)]
|
||||
|
||||
use crate::beacon_processor::work_reprocessing_queue::{
|
||||
QUEUED_ATTESTATION_DELAY, QUEUED_RPC_BLOCK_DELAY,
|
||||
use crate::{
|
||||
network_beacon_processor::{
|
||||
ChainSegmentProcessId, DuplicateCache, InvalidBlockStorage, NetworkBeaconProcessor,
|
||||
},
|
||||
service::NetworkMessage,
|
||||
sync::{manager::BlockProcessType, SyncMessage},
|
||||
};
|
||||
use crate::beacon_processor::*;
|
||||
use crate::{service::NetworkMessage, sync::SyncMessage};
|
||||
use beacon_chain::test_utils::{
|
||||
AttestationStrategy, BeaconChainHarness, BlockStrategy, EphemeralHarnessType,
|
||||
};
|
||||
use beacon_chain::{BeaconChain, ChainConfig, MAXIMUM_GOSSIP_CLOCK_DISPARITY};
|
||||
use beacon_processor::{work_reprocessing_queue::*, *};
|
||||
use lighthouse_network::{
|
||||
discv5::enr::{CombinedKey, EnrBuilder},
|
||||
rpc::methods::{MetaData, MetaDataV2},
|
||||
types::{EnrAttestationBitfield, EnrSyncCommitteeBitfield},
|
||||
MessageId, NetworkGlobals, PeerId,
|
||||
Client, MessageId, NetworkGlobals, PeerId,
|
||||
};
|
||||
use slot_clock::SlotClock;
|
||||
use std::cmp;
|
||||
@ -23,8 +26,8 @@ use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
use tokio::sync::mpsc;
|
||||
use types::{
|
||||
Attestation, AttesterSlashing, Epoch, EthSpec, MainnetEthSpec, ProposerSlashing,
|
||||
SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
|
||||
Attestation, AttesterSlashing, Epoch, EthSpec, Hash256, MainnetEthSpec, ProposerSlashing,
|
||||
SignedAggregateAndProof, SignedBeaconBlock, SignedVoluntaryExit, SubnetId,
|
||||
};
|
||||
|
||||
type E = MainnetEthSpec;
|
||||
@ -51,11 +54,12 @@ struct TestRig {
|
||||
attester_slashing: AttesterSlashing<E>,
|
||||
proposer_slashing: ProposerSlashing,
|
||||
voluntary_exit: SignedVoluntaryExit,
|
||||
beacon_processor_tx: mpsc::Sender<WorkEvent<T>>,
|
||||
beacon_processor_tx: BeaconProcessorSend<E>,
|
||||
work_journal_rx: mpsc::Receiver<&'static str>,
|
||||
_network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
||||
_sync_rx: mpsc::UnboundedReceiver<SyncMessage<E>>,
|
||||
duplicate_cache: DuplicateCache,
|
||||
network_beacon_processor: Arc<NetworkBeaconProcessor<T>>,
|
||||
_harness: BeaconChainHarness<T>,
|
||||
}
|
||||
|
||||
@ -64,7 +68,7 @@ struct TestRig {
|
||||
impl Drop for TestRig {
|
||||
fn drop(&mut self) {
|
||||
// Causes the beacon processor to shutdown.
|
||||
self.beacon_processor_tx = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN).0;
|
||||
self.beacon_processor_tx = BeaconProcessorSend(mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN).0);
|
||||
}
|
||||
}
|
||||
|
||||
@ -169,6 +173,7 @@ impl TestRig {
|
||||
let log = harness.logger().clone();
|
||||
|
||||
let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
|
||||
let beacon_processor_tx = BeaconProcessorSend(beacon_processor_tx);
|
||||
let (sync_tx, _sync_rx) = mpsc::unbounded_channel();
|
||||
|
||||
// Default metadata
|
||||
@ -191,22 +196,40 @@ impl TestRig {
|
||||
|
||||
let executor = harness.runtime.task_executor.clone();
|
||||
|
||||
let (work_reprocessing_tx, work_reprocessing_rx) =
|
||||
mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
|
||||
let (work_journal_tx, work_journal_rx) = mpsc::channel(16_364);
|
||||
|
||||
let duplicate_cache = DuplicateCache::default();
|
||||
BeaconProcessor {
|
||||
beacon_chain: Arc::downgrade(&chain),
|
||||
let network_beacon_processor = NetworkBeaconProcessor {
|
||||
beacon_processor_send: beacon_processor_tx.clone(),
|
||||
duplicate_cache: duplicate_cache.clone(),
|
||||
chain: harness.chain.clone(),
|
||||
network_tx,
|
||||
sync_tx,
|
||||
reprocess_tx: work_reprocessing_tx.clone(),
|
||||
network_globals: network_globals.clone(),
|
||||
invalid_block_storage: InvalidBlockStorage::Disabled,
|
||||
executor: executor.clone(),
|
||||
log: log.clone(),
|
||||
};
|
||||
let network_beacon_processor = Arc::new(network_beacon_processor);
|
||||
|
||||
BeaconProcessor {
|
||||
network_globals,
|
||||
executor,
|
||||
max_workers: cmp::max(1, num_cpus::get()),
|
||||
current_workers: 0,
|
||||
importing_blocks: duplicate_cache.clone(),
|
||||
invalid_block_storage: InvalidBlockStorage::Disabled,
|
||||
enable_backfill_rate_limiting: harness.chain.config.enable_backfill_rate_limiting,
|
||||
log: log.clone(),
|
||||
}
|
||||
.spawn_manager(beacon_processor_rx, Some(work_journal_tx));
|
||||
.spawn_manager(
|
||||
beacon_processor_rx,
|
||||
work_reprocessing_tx,
|
||||
work_reprocessing_rx,
|
||||
Some(work_journal_tx),
|
||||
harness.chain.slot_clock.clone(),
|
||||
);
|
||||
|
||||
Self {
|
||||
chain,
|
||||
@ -222,6 +245,7 @@ impl TestRig {
|
||||
_network_rx,
|
||||
_sync_rx,
|
||||
duplicate_cache,
|
||||
network_beacon_processor,
|
||||
_harness: harness,
|
||||
}
|
||||
}
|
||||
@ -235,102 +259,105 @@ impl TestRig {
|
||||
}
|
||||
|
||||
pub fn enqueue_gossip_block(&self) {
|
||||
self.beacon_processor_tx
|
||||
.try_send(WorkEvent::gossip_beacon_block(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_beacon_block(
|
||||
junk_message_id(),
|
||||
junk_peer_id(),
|
||||
Client::default(),
|
||||
self.next_block.clone(),
|
||||
Duration::from_secs(0),
|
||||
))
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_rpc_block(&self) {
|
||||
let event = WorkEvent::rpc_beacon_block(
|
||||
self.network_beacon_processor
|
||||
.send_rpc_beacon_block(
|
||||
self.next_block.canonical_root(),
|
||||
self.next_block.clone(),
|
||||
std::time::Duration::default(),
|
||||
BlockProcessType::ParentLookup {
|
||||
chain_hash: Hash256::random(),
|
||||
},
|
||||
);
|
||||
self.beacon_processor_tx.try_send(event).unwrap();
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_single_lookup_rpc_block(&self) {
|
||||
let event = WorkEvent::rpc_beacon_block(
|
||||
self.network_beacon_processor
|
||||
.send_rpc_beacon_block(
|
||||
self.next_block.canonical_root(),
|
||||
self.next_block.clone(),
|
||||
std::time::Duration::default(),
|
||||
BlockProcessType::SingleBlock { id: 1 },
|
||||
);
|
||||
self.beacon_processor_tx.try_send(event).unwrap();
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_backfill_batch(&self) {
|
||||
let event = WorkEvent::chain_segment(
|
||||
self.network_beacon_processor
|
||||
.send_chain_segment(
|
||||
ChainSegmentProcessId::BackSyncBatchId(Epoch::default()),
|
||||
Vec::default(),
|
||||
);
|
||||
self.beacon_processor_tx.try_send(event).unwrap();
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_unaggregated_attestation(&self) {
|
||||
let (attestation, subnet_id) = self.attestations.first().unwrap().clone();
|
||||
self.beacon_processor_tx
|
||||
.try_send(WorkEvent::unaggregated_attestation(
|
||||
self.network_beacon_processor
|
||||
.send_unaggregated_attestation(
|
||||
junk_message_id(),
|
||||
junk_peer_id(),
|
||||
attestation,
|
||||
subnet_id,
|
||||
true,
|
||||
Duration::from_secs(0),
|
||||
))
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_gossip_attester_slashing(&self) {
|
||||
self.beacon_processor_tx
|
||||
.try_send(WorkEvent::gossip_attester_slashing(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_attester_slashing(
|
||||
junk_message_id(),
|
||||
junk_peer_id(),
|
||||
Box::new(self.attester_slashing.clone()),
|
||||
))
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_gossip_proposer_slashing(&self) {
|
||||
self.beacon_processor_tx
|
||||
.try_send(WorkEvent::gossip_proposer_slashing(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_proposer_slashing(
|
||||
junk_message_id(),
|
||||
junk_peer_id(),
|
||||
Box::new(self.proposer_slashing.clone()),
|
||||
))
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_gossip_voluntary_exit(&self) {
|
||||
self.beacon_processor_tx
|
||||
.try_send(WorkEvent::gossip_voluntary_exit(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_voluntary_exit(
|
||||
junk_message_id(),
|
||||
junk_peer_id(),
|
||||
Box::new(self.voluntary_exit.clone()),
|
||||
))
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
pub fn enqueue_next_block_unaggregated_attestation(&self) {
|
||||
let (attestation, subnet_id) = self.next_block_attestations.first().unwrap().clone();
|
||||
self.beacon_processor_tx
|
||||
.try_send(WorkEvent::unaggregated_attestation(
|
||||
self.network_beacon_processor
|
||||
.send_unaggregated_attestation(
|
||||
junk_message_id(),
|
||||
junk_peer_id(),
|
||||
attestation,
|
||||
subnet_id,
|
||||
true,
|
||||
Duration::from_secs(0),
|
||||
))
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
@ -340,13 +367,13 @@ impl TestRig {
|
||||
.first()
|
||||
.unwrap()
|
||||
.clone();
|
||||
self.beacon_processor_tx
|
||||
.try_send(WorkEvent::aggregated_attestation(
|
||||
self.network_beacon_processor
|
||||
.send_aggregated_attestation(
|
||||
junk_message_id(),
|
||||
junk_peer_id(),
|
||||
aggregate,
|
||||
Duration::from_secs(0),
|
||||
))
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
@ -5,16 +5,16 @@
|
||||
//! syncing-related responses to the Sync manager.
|
||||
#![allow(clippy::unit_arg)]
|
||||
|
||||
use crate::beacon_processor::{
|
||||
BeaconProcessor, BeaconProcessorSend, InvalidBlockStorage, WorkEvent as BeaconWorkEvent,
|
||||
MAX_WORK_EVENT_QUEUE_LEN,
|
||||
};
|
||||
use crate::error;
|
||||
use crate::network_beacon_processor::{InvalidBlockStorage, NetworkBeaconProcessor};
|
||||
use crate::service::{NetworkMessage, RequestId};
|
||||
use crate::status::status_message;
|
||||
use crate::sync::manager::RequestId as SyncId;
|
||||
use crate::sync::SyncMessage;
|
||||
use beacon_chain::{BeaconChain, BeaconChainTypes};
|
||||
use beacon_processor::{
|
||||
work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend, DuplicateCache,
|
||||
};
|
||||
use futures::prelude::*;
|
||||
use lighthouse_network::rpc::*;
|
||||
use lighthouse_network::{
|
||||
@ -23,7 +23,6 @@ use lighthouse_network::{
|
||||
use logging::TimeLatch;
|
||||
use slog::{debug, o, trace};
|
||||
use slog::{error, warn};
|
||||
use std::cmp;
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, SystemTime, UNIX_EPOCH};
|
||||
use tokio::sync::mpsc;
|
||||
@ -41,7 +40,7 @@ pub struct Router<T: BeaconChainTypes> {
|
||||
/// A network context to return and handle RPC requests.
|
||||
network: HandlerNetworkContext<T::EthSpec>,
|
||||
/// A multi-threaded, non-blocking processor for applying messages to the beacon chain.
|
||||
beacon_processor_send: BeaconProcessorSend<T>,
|
||||
network_beacon_processor: Arc<NetworkBeaconProcessor<T>>,
|
||||
/// The `Router` logger.
|
||||
log: slog::Logger,
|
||||
/// Provides de-bounce functionality for logging.
|
||||
@ -80,12 +79,15 @@ pub enum RouterMessage<T: EthSpec> {
|
||||
|
||||
impl<T: BeaconChainTypes> Router<T> {
|
||||
/// Initializes and runs the Router.
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub fn spawn(
|
||||
beacon_chain: Arc<BeaconChain<T>>,
|
||||
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
|
||||
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
|
||||
executor: task_executor::TaskExecutor,
|
||||
invalid_block_storage: InvalidBlockStorage,
|
||||
beacon_processor_send: BeaconProcessorSend<T::EthSpec>,
|
||||
beacon_processor_reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
log: slog::Logger,
|
||||
) -> error::Result<mpsc::UnboundedSender<RouterMessage<T::EthSpec>>> {
|
||||
let message_handler_log = log.new(o!("service"=> "router"));
|
||||
@ -93,34 +95,33 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
|
||||
let (handler_send, handler_recv) = mpsc::unbounded_channel();
|
||||
|
||||
let (beacon_processor_send, beacon_processor_receive) =
|
||||
mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
|
||||
|
||||
let sync_logger = log.new(o!("service"=> "sync"));
|
||||
// generate the message channel
|
||||
let (sync_send, sync_recv) = mpsc::unbounded_channel::<SyncMessage<T::EthSpec>>();
|
||||
|
||||
// spawn the sync thread
|
||||
let sync_send = crate::sync::manager::spawn(
|
||||
executor.clone(),
|
||||
beacon_chain.clone(),
|
||||
network_globals.clone(),
|
||||
network_send.clone(),
|
||||
BeaconProcessorSend(beacon_processor_send.clone()),
|
||||
sync_logger,
|
||||
);
|
||||
|
||||
BeaconProcessor {
|
||||
beacon_chain: Arc::downgrade(&beacon_chain),
|
||||
let network_beacon_processor = NetworkBeaconProcessor {
|
||||
beacon_processor_send,
|
||||
duplicate_cache: DuplicateCache::default(),
|
||||
chain: beacon_chain.clone(),
|
||||
network_tx: network_send.clone(),
|
||||
sync_tx: sync_send.clone(),
|
||||
reprocess_tx: beacon_processor_reprocess_tx,
|
||||
network_globals: network_globals.clone(),
|
||||
executor: executor.clone(),
|
||||
max_workers: cmp::max(1, num_cpus::get()),
|
||||
current_workers: 0,
|
||||
importing_blocks: Default::default(),
|
||||
invalid_block_storage,
|
||||
executor: executor.clone(),
|
||||
log: log.clone(),
|
||||
}
|
||||
.spawn_manager(beacon_processor_receive, None);
|
||||
};
|
||||
let network_beacon_processor = Arc::new(network_beacon_processor);
|
||||
|
||||
// spawn the sync thread
|
||||
crate::sync::manager::spawn(
|
||||
executor.clone(),
|
||||
beacon_chain.clone(),
|
||||
network_send.clone(),
|
||||
network_beacon_processor.clone(),
|
||||
sync_recv,
|
||||
sync_logger,
|
||||
);
|
||||
|
||||
// generate the Message handler
|
||||
let mut handler = Router {
|
||||
@ -128,7 +129,7 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
chain: beacon_chain,
|
||||
sync_send,
|
||||
network: HandlerNetworkContext::new(network_send, log.clone()),
|
||||
beacon_processor_send: BeaconProcessorSend(beacon_processor_send),
|
||||
network_beacon_processor,
|
||||
log: message_handler_log,
|
||||
logger_debounce: TimeLatch::default(),
|
||||
};
|
||||
@ -197,14 +198,17 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
Request::Status(status_message) => {
|
||||
self.on_status_request(peer_id, request_id, status_message)
|
||||
}
|
||||
Request::BlocksByRange(request) => self.send_beacon_processor_work(
|
||||
BeaconWorkEvent::blocks_by_range_request(peer_id, request_id, request),
|
||||
Request::BlocksByRange(request) => self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_blocks_by_range_request(peer_id, request_id, request),
|
||||
),
|
||||
Request::BlocksByRoot(request) => self.send_beacon_processor_work(
|
||||
BeaconWorkEvent::blocks_by_roots_request(peer_id, request_id, request),
|
||||
Request::BlocksByRoot(request) => self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_blocks_by_roots_request(peer_id, request_id, request),
|
||||
),
|
||||
Request::LightClientBootstrap(request) => self.send_beacon_processor_work(
|
||||
BeaconWorkEvent::lightclient_bootstrap_request(peer_id, request_id, request),
|
||||
Request::LightClientBootstrap(request) => self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_lightclient_bootstrap_request(peer_id, request_id, request),
|
||||
),
|
||||
}
|
||||
}
|
||||
@ -219,10 +223,10 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
match response {
|
||||
Response::Status(status_message) => {
|
||||
debug!(self.log, "Received Status Response"; "peer_id" => %peer_id, &status_message);
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::status_message(
|
||||
peer_id,
|
||||
status_message,
|
||||
))
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_status_message(peer_id, status_message),
|
||||
)
|
||||
}
|
||||
Response::BlocksByRange(beacon_block) => {
|
||||
self.on_blocks_by_range_response(peer_id, request_id, beacon_block);
|
||||
@ -247,36 +251,40 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
) {
|
||||
match gossip_message {
|
||||
PubsubMessage::AggregateAndProofAttestation(aggregate_and_proof) => self
|
||||
.send_beacon_processor_work(BeaconWorkEvent::aggregated_attestation(
|
||||
.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor.send_aggregated_attestation(
|
||||
message_id,
|
||||
peer_id,
|
||||
*aggregate_and_proof,
|
||||
timestamp_now(),
|
||||
)),
|
||||
PubsubMessage::Attestation(subnet_attestation) => {
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::unaggregated_attestation(
|
||||
),
|
||||
),
|
||||
PubsubMessage::Attestation(subnet_attestation) => self
|
||||
.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor.send_unaggregated_attestation(
|
||||
message_id,
|
||||
peer_id,
|
||||
subnet_attestation.1,
|
||||
subnet_attestation.0,
|
||||
should_process,
|
||||
timestamp_now(),
|
||||
))
|
||||
}
|
||||
PubsubMessage::BeaconBlock(block) => {
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::gossip_beacon_block(
|
||||
),
|
||||
),
|
||||
PubsubMessage::BeaconBlock(block) => self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor.send_gossip_beacon_block(
|
||||
message_id,
|
||||
peer_id,
|
||||
self.network_globals.client(&peer_id),
|
||||
block,
|
||||
timestamp_now(),
|
||||
))
|
||||
}
|
||||
),
|
||||
),
|
||||
PubsubMessage::VoluntaryExit(exit) => {
|
||||
debug!(self.log, "Received a voluntary exit"; "peer_id" => %peer_id);
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::gossip_voluntary_exit(
|
||||
message_id, peer_id, exit,
|
||||
))
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_voluntary_exit(message_id, peer_id, exit),
|
||||
)
|
||||
}
|
||||
PubsubMessage::ProposerSlashing(proposer_slashing) => {
|
||||
debug!(
|
||||
@ -284,11 +292,13 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
"Received a proposer slashing";
|
||||
"peer_id" => %peer_id
|
||||
);
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::gossip_proposer_slashing(
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor.send_gossip_proposer_slashing(
|
||||
message_id,
|
||||
peer_id,
|
||||
proposer_slashing,
|
||||
))
|
||||
),
|
||||
)
|
||||
}
|
||||
PubsubMessage::AttesterSlashing(attester_slashing) => {
|
||||
debug!(
|
||||
@ -296,11 +306,13 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
"Received a attester slashing";
|
||||
"peer_id" => %peer_id
|
||||
);
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::gossip_attester_slashing(
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor.send_gossip_attester_slashing(
|
||||
message_id,
|
||||
peer_id,
|
||||
attester_slashing,
|
||||
))
|
||||
),
|
||||
)
|
||||
}
|
||||
PubsubMessage::SignedContributionAndProof(contribution_and_proof) => {
|
||||
trace!(
|
||||
@ -308,12 +320,14 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
"Received sync committee aggregate";
|
||||
"peer_id" => %peer_id
|
||||
);
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::gossip_sync_contribution(
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor.send_gossip_sync_contribution(
|
||||
message_id,
|
||||
peer_id,
|
||||
*contribution_and_proof,
|
||||
timestamp_now(),
|
||||
))
|
||||
),
|
||||
)
|
||||
}
|
||||
PubsubMessage::SyncCommitteeMessage(sync_committtee_msg) => {
|
||||
trace!(
|
||||
@ -321,13 +335,15 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
"Received sync committee signature";
|
||||
"peer_id" => %peer_id
|
||||
);
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::gossip_sync_signature(
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor.send_gossip_sync_signature(
|
||||
message_id,
|
||||
peer_id,
|
||||
sync_committtee_msg.1,
|
||||
sync_committtee_msg.0,
|
||||
timestamp_now(),
|
||||
))
|
||||
),
|
||||
)
|
||||
}
|
||||
PubsubMessage::LightClientFinalityUpdate(light_client_finality_update) => {
|
||||
trace!(
|
||||
@ -335,11 +351,12 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
"Received light client finality update";
|
||||
"peer_id" => %peer_id
|
||||
);
|
||||
self.send_beacon_processor_work(
|
||||
BeaconWorkEvent::gossip_light_client_finality_update(
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_light_client_finality_update(
|
||||
message_id,
|
||||
peer_id,
|
||||
light_client_finality_update,
|
||||
*light_client_finality_update,
|
||||
timestamp_now(),
|
||||
),
|
||||
)
|
||||
@ -350,21 +367,25 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
"Received light client optimistic update";
|
||||
"peer_id" => %peer_id
|
||||
);
|
||||
self.send_beacon_processor_work(
|
||||
BeaconWorkEvent::gossip_light_client_optimistic_update(
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_light_client_optimistic_update(
|
||||
message_id,
|
||||
peer_id,
|
||||
light_client_optimistic_update,
|
||||
*light_client_optimistic_update,
|
||||
timestamp_now(),
|
||||
),
|
||||
)
|
||||
}
|
||||
PubsubMessage::BlsToExecutionChange(bls_to_execution_change) => self
|
||||
.send_beacon_processor_work(BeaconWorkEvent::gossip_bls_to_execution_change(
|
||||
.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_gossip_bls_to_execution_change(
|
||||
message_id,
|
||||
peer_id,
|
||||
bls_to_execution_change,
|
||||
)),
|
||||
),
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
@ -415,7 +436,10 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
request_id,
|
||||
);
|
||||
|
||||
self.send_beacon_processor_work(BeaconWorkEvent::status_message(peer_id, status))
|
||||
self.handle_beacon_processor_send_result(
|
||||
self.network_beacon_processor
|
||||
.send_status_message(peer_id, status),
|
||||
)
|
||||
}
|
||||
|
||||
/// Handle a `BlocksByRange` response from the peer.
|
||||
@ -480,20 +504,22 @@ impl<T: BeaconChainTypes> Router<T> {
|
||||
});
|
||||
}
|
||||
|
||||
fn send_beacon_processor_work(&mut self, work: BeaconWorkEvent<T>) {
|
||||
self.beacon_processor_send
|
||||
.try_send(work)
|
||||
.unwrap_or_else(|e| {
|
||||
let work_type = match &*e {
|
||||
mpsc::error::TrySendError::Closed(work)
|
||||
| mpsc::error::TrySendError::Full(work) => work.work_type(),
|
||||
fn handle_beacon_processor_send_result(
|
||||
&mut self,
|
||||
result: Result<(), crate::network_beacon_processor::Error<T::EthSpec>>,
|
||||
) {
|
||||
if let Err(e) = result {
|
||||
let work_type = match &e {
|
||||
mpsc::error::TrySendError::Closed(work) | mpsc::error::TrySendError::Full(work) => {
|
||||
work.work_type()
|
||||
}
|
||||
};
|
||||
|
||||
if self.logger_debounce.elapsed() {
|
||||
error!(&self.log, "Unable to send message to the beacon processor";
|
||||
"error" => %e, "type" => work_type)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
use super::sync::manager::RequestId as SyncId;
|
||||
use crate::beacon_processor::InvalidBlockStorage;
|
||||
use crate::network_beacon_processor::InvalidBlockStorage;
|
||||
use crate::persisted_dht::{clear_dht, load_dht, persist_dht};
|
||||
use crate::router::{Router, RouterMessage};
|
||||
use crate::subnet_service::SyncCommitteeService;
|
||||
@ -9,6 +9,7 @@ use crate::{
|
||||
NetworkConfig,
|
||||
};
|
||||
use beacon_chain::{BeaconChain, BeaconChainTypes};
|
||||
use beacon_processor::{work_reprocessing_queue::ReprocessQueueMessage, BeaconProcessorSend};
|
||||
use futures::channel::mpsc::Sender;
|
||||
use futures::future::OptionFuture;
|
||||
use futures::prelude::*;
|
||||
@ -224,6 +225,8 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
||||
config: &NetworkConfig,
|
||||
executor: task_executor::TaskExecutor,
|
||||
gossipsub_registry: Option<&'_ mut Registry>,
|
||||
beacon_processor_send: BeaconProcessorSend<T::EthSpec>,
|
||||
beacon_processor_reprocess_tx: mpsc::Sender<ReprocessQueueMessage>,
|
||||
) -> error::Result<(Arc<NetworkGlobals<T::EthSpec>>, NetworkSenders<T::EthSpec>)> {
|
||||
let network_log = executor.log().clone();
|
||||
// build the channels for external comms
|
||||
@ -311,6 +314,8 @@ impl<T: BeaconChainTypes> NetworkService<T> {
|
||||
network_senders.network_send(),
|
||||
executor.clone(),
|
||||
invalid_block_storage,
|
||||
beacon_processor_send,
|
||||
beacon_processor_reprocess_tx,
|
||||
network_log.clone(),
|
||||
)?;
|
||||
|
||||
|
@ -4,12 +4,15 @@ mod tests {
|
||||
use crate::persisted_dht::load_dht;
|
||||
use crate::{NetworkConfig, NetworkService};
|
||||
use beacon_chain::test_utils::BeaconChainHarness;
|
||||
use beacon_processor::{
|
||||
BeaconProcessorSend, MAX_SCHEDULED_WORK_QUEUE_LEN, MAX_WORK_EVENT_QUEUE_LEN,
|
||||
};
|
||||
use lighthouse_network::Enr;
|
||||
use slog::{o, Drain, Level, Logger};
|
||||
use sloggers::{null::NullLoggerBuilder, Build};
|
||||
use std::str::FromStr;
|
||||
use std::sync::Arc;
|
||||
use tokio::runtime::Runtime;
|
||||
use tokio::{runtime::Runtime, sync::mpsc};
|
||||
use types::MinimalEthSpec;
|
||||
|
||||
fn get_logger(actual_log: bool) -> Logger {
|
||||
@ -67,8 +70,18 @@ mod tests {
|
||||
// Create a new network service which implicitly gets dropped at the
|
||||
// end of the block.
|
||||
|
||||
let _network_service =
|
||||
NetworkService::start(beacon_chain.clone(), &config, executor, None)
|
||||
let (beacon_processor_send, _beacon_processor_receive) =
|
||||
mpsc::channel(MAX_WORK_EVENT_QUEUE_LEN);
|
||||
let (beacon_processor_reprocess_tx, _beacon_processor_reprocess_rx) =
|
||||
mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
|
||||
let _network_service = NetworkService::start(
|
||||
beacon_chain.clone(),
|
||||
&config,
|
||||
executor,
|
||||
None,
|
||||
BeaconProcessorSend(beacon_processor_send),
|
||||
beacon_processor_reprocess_tx,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
drop(signal);
|
||||
|
@ -8,7 +8,7 @@
|
||||
//! If a batch fails, the backfill sync cannot progress. In this scenario, we mark the backfill
|
||||
//! sync as failed, log an error and attempt to retry once a new peer joins the node.
|
||||
|
||||
use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent as BeaconWorkEvent};
|
||||
use crate::network_beacon_processor::ChainSegmentProcessId;
|
||||
use crate::sync::manager::{BatchProcessResult, Id};
|
||||
use crate::sync::network_context::SyncNetworkContext;
|
||||
use crate::sync::range_sync::{
|
||||
@ -537,8 +537,8 @@ impl<T: BeaconChainTypes> BackFillSync<T> {
|
||||
self.current_processing_batch = Some(batch_id);
|
||||
|
||||
if let Err(e) = network
|
||||
.processor_channel()
|
||||
.try_send(BeaconWorkEvent::chain_segment(process_id, blocks))
|
||||
.beacon_processor()
|
||||
.send_chain_segment(process_id, blocks)
|
||||
{
|
||||
crit!(self.log, "Failed to send backfill segment to processor."; "msg" => "process_batch",
|
||||
"error" => %e, "batch" => self.processing_target);
|
||||
|
@ -2,6 +2,7 @@ use std::collections::hash_map::Entry;
|
||||
use std::collections::HashMap;
|
||||
use std::time::Duration;
|
||||
|
||||
use crate::network_beacon_processor::ChainSegmentProcessId;
|
||||
use beacon_chain::{BeaconChainTypes, BlockError};
|
||||
use fnv::FnvHashMap;
|
||||
use lighthouse_network::{PeerAction, PeerId};
|
||||
@ -11,7 +12,6 @@ use smallvec::SmallVec;
|
||||
use std::sync::Arc;
|
||||
use store::{Hash256, SignedBeaconBlock};
|
||||
|
||||
use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent};
|
||||
use crate::metrics;
|
||||
|
||||
use self::parent_lookup::PARENT_FAIL_TOLERANCE;
|
||||
@ -542,8 +542,8 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
||||
BlockProcessResult::Ok
|
||||
| BlockProcessResult::Err(BlockError::BlockIsAlreadyKnown { .. }) => {
|
||||
// Check if the beacon processor is available
|
||||
let beacon_processor_send = match cx.processor_channel_if_enabled() {
|
||||
Some(channel) => channel,
|
||||
let beacon_processor = match cx.beacon_processor_if_enabled() {
|
||||
Some(beacon_processor) => beacon_processor,
|
||||
None => {
|
||||
return trace!(
|
||||
self.log,
|
||||
@ -555,7 +555,7 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
||||
let (chain_hash, blocks, hashes, request) = parent_lookup.parts_for_processing();
|
||||
let process_id = ChainSegmentProcessId::ParentLookup(chain_hash);
|
||||
|
||||
match beacon_processor_send.try_send(WorkEvent::chain_segment(process_id, blocks)) {
|
||||
match beacon_processor.send_chain_segment(process_id, blocks) {
|
||||
Ok(_) => {
|
||||
self.processing_parent_lookups
|
||||
.insert(chain_hash, (hashes, request));
|
||||
@ -664,11 +664,15 @@ impl<T: BeaconChainTypes> BlockLookups<T> {
|
||||
process_type: BlockProcessType,
|
||||
cx: &mut SyncNetworkContext<T>,
|
||||
) -> Result<(), ()> {
|
||||
match cx.processor_channel_if_enabled() {
|
||||
Some(beacon_processor_send) => {
|
||||
match cx.beacon_processor_if_enabled() {
|
||||
Some(beacon_processor) => {
|
||||
trace!(self.log, "Sending block for processing"; "block" => ?block_root, "process" => ?process_type);
|
||||
let event = WorkEvent::rpc_beacon_block(block_root, block, duration, process_type);
|
||||
if let Err(e) = beacon_processor_send.try_send(event) {
|
||||
if let Err(e) = beacon_processor.send_rpc_beacon_block(
|
||||
block_root,
|
||||
block,
|
||||
duration,
|
||||
process_type,
|
||||
) {
|
||||
error!(
|
||||
self.log,
|
||||
"Failed to send sync block to processor";
|
||||
|
@ -1,6 +1,6 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::beacon_processor::BeaconProcessorSend;
|
||||
use crate::network_beacon_processor::NetworkBeaconProcessor;
|
||||
use crate::service::RequestId;
|
||||
use crate::sync::manager::RequestId as SyncId;
|
||||
use crate::NetworkMessage;
|
||||
@ -9,18 +9,19 @@ use super::*;
|
||||
|
||||
use beacon_chain::builder::Witness;
|
||||
use beacon_chain::eth1_chain::CachingEth1Backend;
|
||||
use beacon_processor::WorkEvent;
|
||||
use lighthouse_network::{NetworkGlobals, Request};
|
||||
use slog::{Drain, Level};
|
||||
use slot_clock::SystemTimeSlotClock;
|
||||
use slot_clock::ManualSlotClock;
|
||||
use store::MemoryStore;
|
||||
use tokio::sync::mpsc;
|
||||
use types::test_utils::{SeedableRng, TestRandom, XorShiftRng};
|
||||
use types::MinimalEthSpec as E;
|
||||
|
||||
type T = Witness<SystemTimeSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
|
||||
type T = Witness<ManualSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
|
||||
|
||||
struct TestRig {
|
||||
beacon_processor_rx: mpsc::Receiver<WorkEvent<T>>,
|
||||
beacon_processor_rx: mpsc::Receiver<WorkEvent<E>>,
|
||||
network_rx: mpsc::UnboundedReceiver<NetworkMessage<E>>,
|
||||
rng: XorShiftRng,
|
||||
}
|
||||
@ -41,8 +42,10 @@ impl TestRig {
|
||||
}
|
||||
};
|
||||
|
||||
let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(100);
|
||||
let (network_tx, network_rx) = mpsc::unbounded_channel();
|
||||
let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log));
|
||||
let (network_beacon_processor, beacon_processor_rx) =
|
||||
NetworkBeaconProcessor::null_for_testing(globals);
|
||||
let rng = XorShiftRng::from_seed([42; 16]);
|
||||
let rig = TestRig {
|
||||
beacon_processor_rx,
|
||||
@ -51,11 +54,9 @@ impl TestRig {
|
||||
};
|
||||
let bl = BlockLookups::new(log.new(slog::o!("component" => "block_lookups")));
|
||||
let cx = {
|
||||
let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log));
|
||||
SyncNetworkContext::new(
|
||||
network_tx,
|
||||
globals,
|
||||
BeaconProcessorSend(beacon_processor_tx),
|
||||
Arc::new(network_beacon_processor),
|
||||
log.new(slog::o!("component" => "network_context")),
|
||||
)
|
||||
};
|
||||
@ -102,7 +103,7 @@ impl TestRig {
|
||||
fn expect_block_process(&mut self) {
|
||||
match self.beacon_processor_rx.try_recv() {
|
||||
Ok(work) => {
|
||||
assert_eq!(work.work_type(), crate::beacon_processor::RPC_BLOCK);
|
||||
assert_eq!(work.work_type(), beacon_processor::RPC_BLOCK);
|
||||
}
|
||||
other => panic!("Expected block process, found {:?}", other),
|
||||
}
|
||||
@ -112,7 +113,7 @@ impl TestRig {
|
||||
fn expect_parent_chain_process(&mut self) {
|
||||
match self.beacon_processor_rx.try_recv() {
|
||||
Ok(work) => {
|
||||
assert_eq!(work.work_type(), crate::beacon_processor::CHAIN_SEGMENT);
|
||||
assert_eq!(work.work_type(), beacon_processor::CHAIN_SEGMENT);
|
||||
}
|
||||
other => panic!("Expected chain segment process, found {:?}", other),
|
||||
}
|
||||
|
@ -38,7 +38,7 @@ use super::block_lookups::BlockLookups;
|
||||
use super::network_context::SyncNetworkContext;
|
||||
use super::peer_sync_info::{remote_sync_type, PeerSyncType};
|
||||
use super::range_sync::{RangeSync, RangeSyncType, EPOCHS_PER_BATCH};
|
||||
use crate::beacon_processor::{BeaconProcessorSend, ChainSegmentProcessId};
|
||||
use crate::network_beacon_processor::{ChainSegmentProcessId, NetworkBeaconProcessor};
|
||||
use crate::service::NetworkMessage;
|
||||
use crate::status::ToStatusMessage;
|
||||
use beacon_chain::{BeaconChain, BeaconChainTypes, BlockError, EngineState};
|
||||
@ -159,9 +159,6 @@ pub struct SyncManager<T: BeaconChainTypes> {
|
||||
/// A reference to the underlying beacon chain.
|
||||
chain: Arc<BeaconChain<T>>,
|
||||
|
||||
/// A reference to the network globals and peer-db.
|
||||
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
|
||||
|
||||
/// A receiving channel sent by the message processor thread.
|
||||
input_channel: mpsc::UnboundedReceiver<SyncMessage<T::EthSpec>>,
|
||||
|
||||
@ -186,29 +183,22 @@ pub struct SyncManager<T: BeaconChainTypes> {
|
||||
pub fn spawn<T: BeaconChainTypes>(
|
||||
executor: task_executor::TaskExecutor,
|
||||
beacon_chain: Arc<BeaconChain<T>>,
|
||||
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
|
||||
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
|
||||
beacon_processor_send: BeaconProcessorSend<T>,
|
||||
beacon_processor: Arc<NetworkBeaconProcessor<T>>,
|
||||
sync_recv: mpsc::UnboundedReceiver<SyncMessage<T::EthSpec>>,
|
||||
log: slog::Logger,
|
||||
) -> mpsc::UnboundedSender<SyncMessage<T::EthSpec>> {
|
||||
) {
|
||||
assert!(
|
||||
MAX_REQUEST_BLOCKS >= T::EthSpec::slots_per_epoch() * EPOCHS_PER_BATCH,
|
||||
"Max blocks that can be requested in a single batch greater than max allowed blocks in a single request"
|
||||
);
|
||||
// generate the message channel
|
||||
let (sync_send, sync_recv) = mpsc::unbounded_channel::<SyncMessage<T::EthSpec>>();
|
||||
|
||||
// create an instance of the SyncManager
|
||||
let network_globals = beacon_processor.network_globals.clone();
|
||||
let mut sync_manager = SyncManager {
|
||||
chain: beacon_chain.clone(),
|
||||
network_globals: network_globals.clone(),
|
||||
input_channel: sync_recv,
|
||||
network: SyncNetworkContext::new(
|
||||
network_send,
|
||||
network_globals.clone(),
|
||||
beacon_processor_send,
|
||||
log.clone(),
|
||||
),
|
||||
network: SyncNetworkContext::new(network_send, beacon_processor, log.clone()),
|
||||
range_sync: RangeSync::new(beacon_chain.clone(), log.clone()),
|
||||
backfill_sync: BackFillSync::new(beacon_chain, network_globals, log.clone()),
|
||||
block_lookups: BlockLookups::new(log.clone()),
|
||||
@ -218,10 +208,13 @@ pub fn spawn<T: BeaconChainTypes>(
|
||||
// spawn the sync manager thread
|
||||
debug!(log, "Sync Manager started");
|
||||
executor.spawn(async move { Box::pin(sync_manager.main()).await }, "sync");
|
||||
sync_send
|
||||
}
|
||||
|
||||
impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
fn network_globals(&self) -> &NetworkGlobals<T::EthSpec> {
|
||||
self.network.network_globals()
|
||||
}
|
||||
|
||||
/* Input Handling Functions */
|
||||
|
||||
/// A peer has connected which has blocks that are unknown to us.
|
||||
@ -322,12 +315,12 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
let rpr = new_state.as_str();
|
||||
// Drop the write lock
|
||||
let update_sync_status = self
|
||||
.network_globals
|
||||
.network_globals()
|
||||
.peers
|
||||
.write()
|
||||
.update_sync_status(peer_id, new_state.clone());
|
||||
if let Some(was_updated) = update_sync_status {
|
||||
let is_connected = self.network_globals.peers.read().is_connected(peer_id);
|
||||
let is_connected = self.network_globals().peers.read().is_connected(peer_id);
|
||||
if was_updated {
|
||||
debug!(
|
||||
self.log,
|
||||
@ -383,7 +376,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
let head = self.chain.best_slot();
|
||||
let current_slot = self.chain.slot().unwrap_or_else(|_| Slot::new(0));
|
||||
|
||||
let peers = self.network_globals.peers.read();
|
||||
let peers = self.network_globals().peers.read();
|
||||
if current_slot >= head
|
||||
&& current_slot.sub(head) <= (SLOT_IMPORT_TOLERANCE as u64)
|
||||
&& head > 0
|
||||
@ -445,8 +438,8 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
},
|
||||
};
|
||||
|
||||
let old_state = self.network_globals.set_sync_state(new_state);
|
||||
let new_state = self.network_globals.sync_state.read();
|
||||
let old_state = self.network_globals().set_sync_state(new_state);
|
||||
let new_state = self.network_globals().sync_state.read().clone();
|
||||
if !new_state.eq(&old_state) {
|
||||
info!(self.log, "Sync state updated"; "old_state" => %old_state, "new_state" => %new_state);
|
||||
// If we have become synced - Subscribe to all the core subnet topics
|
||||
@ -505,7 +498,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
}
|
||||
SyncMessage::UnknownBlock(peer_id, block, block_root) => {
|
||||
// If we are not synced or within SLOT_IMPORT_TOLERANCE of the block, ignore
|
||||
if !self.network_globals.sync_state.read().is_synced() {
|
||||
if !self.network_globals().sync_state.read().is_synced() {
|
||||
let head_slot = self.chain.canonical_head.cached_head().head_slot();
|
||||
let unknown_block_slot = block.slot();
|
||||
|
||||
@ -519,7 +512,7 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
return;
|
||||
}
|
||||
}
|
||||
if self.network_globals.peers.read().is_connected(&peer_id)
|
||||
if self.network_globals().peers.read().is_connected(&peer_id)
|
||||
&& self.network.is_execution_engine_online()
|
||||
{
|
||||
self.block_lookups
|
||||
@ -528,8 +521,8 @@ impl<T: BeaconChainTypes> SyncManager<T> {
|
||||
}
|
||||
SyncMessage::UnknownBlockHash(peer_id, block_hash) => {
|
||||
// If we are not synced, ignore this block.
|
||||
if self.network_globals.sync_state.read().is_synced()
|
||||
&& self.network_globals.peers.read().is_connected(&peer_id)
|
||||
if self.network_globals().sync_state.read().is_synced()
|
||||
&& self.network_globals().peers.read().is_connected(&peer_id)
|
||||
&& self.network.is_execution_engine_online()
|
||||
{
|
||||
self.block_lookups
|
||||
|
@ -3,7 +3,7 @@
|
||||
|
||||
use super::manager::{Id, RequestId as SyncRequestId};
|
||||
use super::range_sync::{BatchId, ChainId};
|
||||
use crate::beacon_processor::BeaconProcessorSend;
|
||||
use crate::network_beacon_processor::NetworkBeaconProcessor;
|
||||
use crate::service::{NetworkMessage, RequestId};
|
||||
use crate::status::ToStatusMessage;
|
||||
use beacon_chain::{BeaconChainTypes, EngineState};
|
||||
@ -20,9 +20,6 @@ pub struct SyncNetworkContext<T: BeaconChainTypes> {
|
||||
/// The network channel to relay messages to the Network service.
|
||||
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
|
||||
|
||||
/// Access to the network global vars.
|
||||
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
|
||||
|
||||
/// A sequential ID for all RPC requests.
|
||||
request_id: Id,
|
||||
|
||||
@ -36,8 +33,8 @@ pub struct SyncNetworkContext<T: BeaconChainTypes> {
|
||||
/// `beacon_processor_send`.
|
||||
execution_engine_state: EngineState,
|
||||
|
||||
/// Channel to send work to the beacon processor.
|
||||
beacon_processor_send: BeaconProcessorSend<T>,
|
||||
/// Sends work to the beacon processor via a channel.
|
||||
network_beacon_processor: Arc<NetworkBeaconProcessor<T>>,
|
||||
|
||||
/// Logger for the `SyncNetworkContext`.
|
||||
log: slog::Logger,
|
||||
@ -46,25 +43,27 @@ pub struct SyncNetworkContext<T: BeaconChainTypes> {
|
||||
impl<T: BeaconChainTypes> SyncNetworkContext<T> {
|
||||
pub fn new(
|
||||
network_send: mpsc::UnboundedSender<NetworkMessage<T::EthSpec>>,
|
||||
network_globals: Arc<NetworkGlobals<T::EthSpec>>,
|
||||
beacon_processor_send: BeaconProcessorSend<T>,
|
||||
network_beacon_processor: Arc<NetworkBeaconProcessor<T>>,
|
||||
log: slog::Logger,
|
||||
) -> Self {
|
||||
Self {
|
||||
network_send,
|
||||
execution_engine_state: EngineState::Online, // always assume `Online` at the start
|
||||
network_globals,
|
||||
request_id: 1,
|
||||
range_requests: FnvHashMap::default(),
|
||||
backfill_requests: FnvHashMap::default(),
|
||||
beacon_processor_send,
|
||||
network_beacon_processor,
|
||||
log,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn network_globals(&self) -> &NetworkGlobals<T::EthSpec> {
|
||||
&self.network_beacon_processor.network_globals
|
||||
}
|
||||
|
||||
/// Returns the Client type of the peer if known
|
||||
pub fn client_type(&self, peer_id: &PeerId) -> Client {
|
||||
self.network_globals
|
||||
self.network_globals()
|
||||
.peers
|
||||
.read()
|
||||
.peer_info(peer_id)
|
||||
@ -278,13 +277,13 @@ impl<T: BeaconChainTypes> SyncNetworkContext<T> {
|
||||
})
|
||||
}
|
||||
|
||||
pub fn processor_channel_if_enabled(&self) -> Option<&BeaconProcessorSend<T>> {
|
||||
pub fn beacon_processor_if_enabled(&self) -> Option<&Arc<NetworkBeaconProcessor<T>>> {
|
||||
self.is_execution_engine_online()
|
||||
.then_some(&self.beacon_processor_send)
|
||||
.then_some(&self.network_beacon_processor)
|
||||
}
|
||||
|
||||
pub fn processor_channel(&self) -> &BeaconProcessorSend<T> {
|
||||
&self.beacon_processor_send
|
||||
pub fn beacon_processor(&self) -> &Arc<NetworkBeaconProcessor<T>> {
|
||||
&self.network_beacon_processor
|
||||
}
|
||||
|
||||
fn next_id(&mut self) -> Id {
|
||||
|
@ -1,5 +1,5 @@
|
||||
use super::batch::{BatchInfo, BatchProcessingResult, BatchState};
|
||||
use crate::beacon_processor::{ChainSegmentProcessId, WorkEvent as BeaconWorkEvent};
|
||||
use crate::network_beacon_processor::ChainSegmentProcessId;
|
||||
use crate::sync::{
|
||||
manager::Id, network_context::SyncNetworkContext, BatchOperationOutcome, BatchProcessResult,
|
||||
};
|
||||
@ -294,8 +294,8 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
||||
return Ok(KeepChain);
|
||||
}
|
||||
|
||||
let beacon_processor_send = match network.processor_channel_if_enabled() {
|
||||
Some(channel) => channel,
|
||||
let beacon_processor = match network.beacon_processor_if_enabled() {
|
||||
Some(beacon_processor) => beacon_processor,
|
||||
None => return Ok(KeepChain),
|
||||
};
|
||||
|
||||
@ -317,9 +317,7 @@ impl<T: BeaconChainTypes> SyncingChain<T> {
|
||||
let process_id = ChainSegmentProcessId::RangeBatchId(self.id, batch_id);
|
||||
self.current_processing_batch = Some(batch_id);
|
||||
|
||||
if let Err(e) =
|
||||
beacon_processor_send.try_send(BeaconWorkEvent::chain_segment(process_id, blocks))
|
||||
{
|
||||
if let Err(e) = beacon_processor.send_chain_segment(process_id, blocks) {
|
||||
crit!(self.log, "Failed to send chain segment to processor."; "msg" => "process_batch",
|
||||
"error" => %e, "batch" => self.processing_target);
|
||||
// This is unlikely to happen but it would stall syncing since the batch now has no
|
||||
|
@ -371,22 +371,23 @@ where
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::network_beacon_processor::NetworkBeaconProcessor;
|
||||
use crate::service::RequestId;
|
||||
use crate::NetworkMessage;
|
||||
|
||||
use super::*;
|
||||
use crate::beacon_processor::{BeaconProcessorSend, WorkEvent as BeaconWorkEvent};
|
||||
use beacon_chain::builder::Witness;
|
||||
use beacon_chain::eth1_chain::CachingEth1Backend;
|
||||
use beacon_chain::parking_lot::RwLock;
|
||||
use beacon_chain::EngineState;
|
||||
use beacon_processor::WorkEvent as BeaconWorkEvent;
|
||||
use lighthouse_network::rpc::BlocksByRangeRequest;
|
||||
use lighthouse_network::Request;
|
||||
use lighthouse_network::{rpc::StatusMessage, NetworkGlobals};
|
||||
use slog::{o, Drain};
|
||||
use tokio::sync::mpsc;
|
||||
|
||||
use slot_clock::SystemTimeSlotClock;
|
||||
use slot_clock::ManualSlotClock;
|
||||
use std::collections::HashSet;
|
||||
use std::sync::Arc;
|
||||
use store::MemoryStore;
|
||||
@ -437,7 +438,7 @@ mod tests {
|
||||
}
|
||||
|
||||
type TestBeaconChainType =
|
||||
Witness<SystemTimeSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
|
||||
Witness<ManualSlotClock, CachingEth1Backend<E>, E, MemoryStore<E>, MemoryStore<E>>;
|
||||
|
||||
fn build_log(level: slog::Level, enabled: bool) -> slog::Logger {
|
||||
let decorator = slog_term::TermDecorator::new().build();
|
||||
@ -455,7 +456,7 @@ mod tests {
|
||||
struct TestRig {
|
||||
log: slog::Logger,
|
||||
/// To check what does sync send to the beacon processor.
|
||||
beacon_processor_rx: mpsc::Receiver<BeaconWorkEvent<TestBeaconChainType>>,
|
||||
beacon_processor_rx: mpsc::Receiver<BeaconWorkEvent<E>>,
|
||||
/// To set up different scenarios where sync is told about known/unkown blocks.
|
||||
chain: Arc<FakeStorage>,
|
||||
/// Needed by range to handle communication with the network.
|
||||
@ -583,7 +584,7 @@ mod tests {
|
||||
fn expect_chain_segment(&mut self) {
|
||||
match self.beacon_processor_rx.try_recv() {
|
||||
Ok(work) => {
|
||||
assert_eq!(work.work_type(), crate::beacon_processor::CHAIN_SEGMENT);
|
||||
assert_eq!(work.work_type(), beacon_processor::CHAIN_SEGMENT);
|
||||
}
|
||||
other => panic!("Expected chain segment process, found {:?}", other),
|
||||
}
|
||||
@ -593,17 +594,17 @@ mod tests {
|
||||
fn range(log_enabled: bool) -> (TestRig, RangeSync<TestBeaconChainType, FakeStorage>) {
|
||||
let chain = Arc::new(FakeStorage::default());
|
||||
let log = build_log(slog::Level::Trace, log_enabled);
|
||||
let (beacon_processor_tx, beacon_processor_rx) = mpsc::channel(10);
|
||||
let range_sync = RangeSync::<TestBeaconChainType, FakeStorage>::new(
|
||||
chain.clone(),
|
||||
log.new(o!("component" => "range")),
|
||||
);
|
||||
let (network_tx, network_rx) = mpsc::unbounded_channel();
|
||||
let globals = Arc::new(NetworkGlobals::new_test_globals(Vec::new(), &log));
|
||||
let (network_beacon_processor, beacon_processor_rx) =
|
||||
NetworkBeaconProcessor::null_for_testing(globals.clone());
|
||||
let cx = SyncNetworkContext::new(
|
||||
network_tx,
|
||||
globals.clone(),
|
||||
BeaconProcessorSend(beacon_processor_tx),
|
||||
Arc::new(network_beacon_processor),
|
||||
log.new(o!("component" => "network_context")),
|
||||
);
|
||||
let test_rig = TestRig {
|
||||
|
Loading…
Reference in New Issue
Block a user